From f652783832f8a876365d571da2118724abdb0a66 Mon Sep 17 00:00:00 2001 From: zzzxl <33418555+zzzxl1993@users.noreply.github.com> Date: Thu, 29 Aug 2024 10:38:08 +0800 Subject: [PATCH 01/60] [fix](inverted index) fix error handling in fast_execute (#40024) 1. fix error handling in fast_execute 2. Fixed the issue where fast_execute was still being executed when set enable_inverted_index_query = false. --- be/src/vec/exprs/vexpr.cpp | 13 +++++++++++++ be/src/vec/exprs/vexpr.h | 1 + 2 files changed, 14 insertions(+) diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp index c615c91dc75ef1..b66c8aa80a7e6f 100644 --- a/be/src/vec/exprs/vexpr.cpp +++ b/be/src/vec/exprs/vexpr.cpp @@ -210,6 +210,7 @@ Status VExpr::prepare(RuntimeState* state, const RowDescriptor& row_desc, VExprC RETURN_IF_ERROR(i->prepare(state, row_desc, context)); } --context->_depth_num; + _enable_inverted_index_query = state->query_options().enable_inverted_index_query; return Status::OK(); } @@ -603,6 +604,10 @@ Status VExpr::get_result_from_const(vectorized::Block* block, const std::string& bool VExpr::fast_execute(Block& block, const ColumnNumbers& arguments, size_t result, size_t input_rows_count, const std::string& function_name) { + if (!_enable_inverted_index_query) { + return false; + } + std::string result_column_name = gen_predicate_result_sign(block, arguments, function_name); if (!block.has(result_column_name)) { DBUG_EXECUTE_IF("segment_iterator.fast_execute", { @@ -651,11 +656,19 @@ std::string VExpr::gen_predicate_result_sign(Block& block, const ColumnNumbers& std::set values; for (size_t i = 1; i < arguments.size(); i++) { const auto& entry = block.get_by_position(arguments[i]); + if (!is_column_const(*entry.column)) { + return pred_result_sign; + } values.insert(entry.type->to_string(*entry.column, 0)); } pred_result_sign += boost::join(values, ","); + } else if (function_name == "collection_in" || function_name == "collection_not_in") { + return pred_result_sign; } else { const auto& entry = block.get_by_position(arguments[1]); + if (!is_column_const(*entry.column)) { + return pred_result_sign; + } pred_result_sign += entry.type->to_string(*entry.column, 0); } } diff --git a/be/src/vec/exprs/vexpr.h b/be/src/vec/exprs/vexpr.h index 7ee4e0a5dd4a6a..0409327aef1dc0 100644 --- a/be/src/vec/exprs/vexpr.h +++ b/be/src/vec/exprs/vexpr.h @@ -328,6 +328,7 @@ class VExpr { // ensuring uniqueness during index traversal uint32_t _index_unique_id = 0; bool _can_fast_execute = false; + bool _enable_inverted_index_query = true; }; } // namespace vectorized From 55ed12774fd9ac65257672e83e8fc66f8b9ea3d9 Mon Sep 17 00:00:00 2001 From: seawinde <149132972+seawinde@users.noreply.github.com> Date: Thu, 29 Aug 2024 10:44:15 +0800 Subject: [PATCH 02/60] [improvement](mtmv) Optimize get available mv logic to avoid unnecessary rewrite (#39734) Optimize get available mv logic to avoid unnecessary rewrite such as mv def sql is as following CREATE MATERIALIZED VIEW mv1_0 BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL DISTRIBUTED BY RANDOM BUCKETS 2 PROPERTIES ('replication_num' = '1') AS select lineitem.L_LINENUMBER, orders.O_CUSTKEY from lineitem inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY if we run query as following, mv1_0 should not participate in query rewrite by materialized view to avoid unnecessary rewrite select L_LINENUMBER from mv1_0; --- .../mv/InitMaterializationContextHook.java | 3 +- .../nereids/trees/plans/PlanVisitorTest.java | 16 ++ .../doris/regression/suite/Suite.groovy | 24 +++ .../mv/direct_query/direct_query.groovy | 162 ++++++++++++++++++ 4 files changed, 204 insertions(+), 1 deletion(-) create mode 100644 regression-test/suites/nereids_rules_p0/mv/direct_query/direct_query.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java index c48303dbf0b977..e1f98afae3f779 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java @@ -78,7 +78,8 @@ public void initMaterializationContext(CascadesContext cascadesContext) { * @param cascadesContext current cascadesContext in the planner */ protected void doInitMaterializationContext(CascadesContext cascadesContext) { - TableCollectorContext collectorContext = new TableCollectorContext(Sets.newHashSet(), true); + // Only collect the table or mv which query use directly, to avoid useless mv partition in rewrite + TableCollectorContext collectorContext = new TableCollectorContext(Sets.newHashSet(), false); try { Plan rewritePlan = cascadesContext.getRewritePlan(); // Keep use one connection context when in query, if new connect context, diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java index ec184080bdf401..60f6e19faab635 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java @@ -243,6 +243,22 @@ public BitSet getDisableNereidsRules() { .map(TableIf::getName) .collect(Collectors.toSet()), expectedMvsWithNoExpand); + + TableCollectorContext allTableTypeWithExpand = + new TableCollector.TableCollectorContext( + Sets.newHashSet(TableType.values()), true); + physicalPlan.accept(TableCollector.INSTANCE, allTableTypeWithExpand); + // when collect in plan with expand, should collect table which is expended + Set expectedTablesWithExpand = new HashSet<>(); + expectedTablesWithExpand.add("mv1"); + expectedTablesWithExpand.add("table1"); + expectedTablesWithExpand.add("table2"); + expectedTablesWithExpand.add("table3"); + Assertions.assertEquals( + allTableTypeWithExpand.getCollectedTables().stream() + .map(TableIf::getName) + .collect(Collectors.toSet()), + expectedTablesWithExpand); }); dropMvByNereids("drop materialized view mv1"); } 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 ac8795729e1e94..f5e6e5ab1bab71 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 @@ -1502,6 +1502,30 @@ class Suite implements GroovyInterceptable { return result.values().toList() } + def create_async_mv = { db, mv_name, mv_sql -> + + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS ${mv_sql} + """ + def job_name = getJobName(db, mv_name); + waitingMTMVTaskFinished(job_name) + sql "analyze table ${mv_name} with sync;" + } + + def mv_not_part_in = { query_sql, mv_name -> + explain { + sql(" memo plan ${query_sql}") + notContains("${mv_name} chose") + notContains("${mv_name} not chose") + notContains("${mv_name} fail") + } + } + def mv_rewrite_success = { query_sql, mv_name -> explain { sql(" memo plan ${query_sql}") diff --git a/regression-test/suites/nereids_rules_p0/mv/direct_query/direct_query.groovy b/regression-test/suites/nereids_rules_p0/mv/direct_query/direct_query.groovy new file mode 100644 index 00000000000000..404154300d2724 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/direct_query/direct_query.groovy @@ -0,0 +1,162 @@ +package mv.direct_query +// 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("direct_query_mv") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF" + + sql """ + drop table if exists orders + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + o_comment VARCHAR(79) NOT NULL + ) + DUPLICATE KEY(o_orderkey, o_custkey) + PARTITION BY RANGE(o_orderdate) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30')) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """ + drop table if exists lineitem + """ + + sql""" + CREATE TABLE IF NOT EXISTS lineitem ( + l_orderkey INTEGER NOT NULL, + l_partkey INTEGER NOT NULL, + l_suppkey INTEGER NOT NULL, + l_linenumber INTEGER NOT NULL, + l_quantity DECIMALV3(15,2) NOT NULL, + l_extendedprice DECIMALV3(15,2) NOT NULL, + l_discount DECIMALV3(15,2) NOT NULL, + l_tax DECIMALV3(15,2) NOT NULL, + l_returnflag CHAR(1) NOT NULL, + l_linestatus CHAR(1) NOT NULL, + l_shipdate DATE NOT NULL, + l_commitdate DATE NOT NULL, + l_receiptdate DATE NOT NULL, + l_shipinstruct CHAR(25) NOT NULL, + l_shipmode CHAR(10) NOT NULL, + l_comment VARCHAR(44) NOT NULL + ) + DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) + PARTITION BY RANGE(l_shipdate) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30')) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """ + drop table if exists partsupp + """ + + sql """ + CREATE TABLE IF NOT EXISTS partsupp ( + ps_partkey INTEGER NOT NULL, + ps_suppkey INTEGER NOT NULL, + ps_availqty INTEGER NOT NULL, + ps_supplycost DECIMALV3(15,2) NOT NULL, + ps_comment VARCHAR(199) NOT NULL + ) + DUPLICATE KEY(ps_partkey, ps_suppkey) + DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """ insert into lineitem values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); + """ + + sql """ + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + sql """ + insert into partsupp values + (2, 3, 9, 10.01, 'supply1'), + (2, 3, 10, 11.01, 'supply2'); + """ + + sql """analyze table lineitem with sync;""" + sql """analyze table orders with sync;""" + sql """analyze table partsupp with sync;""" + + + create_async_mv(db, "mv1_0", + """ + select lineitem.L_LINENUMBER, orders.O_CUSTKEY + from lineitem + inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY + """) + // select mv directly, mv should not part in rewrite + mv_not_part_in("""select L_LINENUMBER from mv1_0;""", "mv1_0") + + + create_async_mv(db, "mv2_0", + """ + select L_LINENUMBER, count(O_CUSTKEY) + from mv1_0 + group by L_LINENUMBER; + """) + // mv2 use mv1, though query not use mv1 directly, mv2 should part in rewrite and shoule be chosen + mv_rewrite_success(""" + select L_LINENUMBER, count(O_CUSTKEY) + from lineitem + inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY + group by L_LINENUMBER; + """, + "mv2_0"); + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_0""" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0""" +} From 5f08e5d5c028d4369144b0f58a37fc40ff4b2dbc Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Thu, 29 Aug 2024 10:46:57 +0800 Subject: [PATCH 03/60] [Opt](parquet/orc-reader) Opt get dict ids in _rewrite_dict_predicates(). (#39893) ## Proposed changes The original implementation needed to obtain the dict code through dict value when generating dictionary filter conditions. In order to obtain the dict code, a `dict_value_to_code` hash map needs to be constructed. We found that when the amount of data is large, it is time-consuming to construct the `dict_value_to_code` hash map, and the current implementation will also construct it when there is no filter condition. Because the dict code is the position of the dictionary element, this PR records the position when filtering the dict value, so as to directly obtain the dict code. --- be/src/vec/exec/format/orc/vorc_reader.cpp | 49 +++++++------------ .../parquet/byte_array_dict_decoder.cpp | 11 ----- .../format/parquet/byte_array_dict_decoder.h | 4 -- be/src/vec/exec/format/parquet/decoder.h | 5 -- .../parquet/fix_length_dict_decoder.hpp | 14 ------ .../parquet/vparquet_column_chunk_reader.h | 5 -- .../format/parquet/vparquet_column_reader.cpp | 5 -- .../format/parquet/vparquet_column_reader.h | 7 --- .../format/parquet/vparquet_group_reader.cpp | 43 +++++++--------- 9 files changed, 36 insertions(+), 107 deletions(-) diff --git a/be/src/vec/exec/format/orc/vorc_reader.cpp b/be/src/vec/exec/format/orc/vorc_reader.cpp index e21bc6d67811f6..cffa934cc2c740 100644 --- a/be/src/vec/exec/format/orc/vorc_reader.cpp +++ b/be/src/vec/exec/format/orc/vorc_reader.cpp @@ -2054,7 +2054,6 @@ Status OrcReader::on_string_dicts_loaded( orc::StringDictionary* dict = file_column_name_to_dict_map_iter->second; std::vector dict_values; - std::unordered_map dict_value_to_code; size_t max_value_length = 0; uint64_t dictionaryCount = dict->dictionaryOffset.size() - 1; if (dictionaryCount == 0) { @@ -2074,7 +2073,6 @@ Status OrcReader::on_string_dicts_loaded( max_value_length = length; } dict_values.emplace_back(dict_value); - dict_value_to_code[dict_value] = i; } dict_value_column->insert_many_strings_overflow(&dict_values[0], dict_values.size(), max_value_length); @@ -2113,31 +2111,37 @@ Status OrcReader::on_string_dicts_loaded( ++index; } - // 2.2 Execute conjuncts and filter block. - std::vector columns_to_filter(1, dict_pos); - int column_to_keep = temp_block.columns(); + // 2.2 Execute conjuncts. if (dict_pos != 0) { // VExprContext.execute has an optimization, the filtering is executed when block->rows() > 0 // The following process may be tricky and time-consuming, but we have no other way. temp_block.get_by_position(0).column->assume_mutable()->resize(dict_value_column_size); } - RETURN_IF_CATCH_EXCEPTION(RETURN_IF_ERROR(VExprContext::execute_conjuncts_and_filter_block( - ctxs, &temp_block, columns_to_filter, column_to_keep))); + IColumn::Filter result_filter(temp_block.rows(), 1); + bool can_filter_all; + RETURN_IF_ERROR(VExprContext::execute_conjuncts(ctxs, nullptr, &temp_block, &result_filter, + &can_filter_all)); if (dict_pos != 0) { // We have to clean the first column to insert right data. temp_block.get_by_position(0).column->assume_mutable()->clear(); } - // Check some conditions. - ColumnPtr& dict_column = temp_block.get_by_position(dict_pos).column; - // If dict_column->size() == 0, can filter this stripe. - if (dict_column->size() == 0) { + // If can_filter_all = true, can filter this stripe. + if (can_filter_all) { *is_stripe_filtered = true; return Status::OK(); } + // 3. Get dict codes. + std::vector dict_codes; + for (size_t i = 0; i < result_filter.size(); ++i) { + if (result_filter[i]) { + dict_codes.emplace_back(i); + } + } + // About Performance: if dict_column size is too large, it will generate a large IN filter. - if (dict_column->size() > MAX_DICT_CODE_PREDICATE_TO_REWRITE) { + if (dict_codes.size() > MAX_DICT_CODE_PREDICATE_TO_REWRITE) { it = _dict_filter_cols.erase(it); for (auto& ctx : ctxs) { _non_dict_filter_conjuncts.emplace_back(ctx); @@ -2145,26 +2149,9 @@ Status OrcReader::on_string_dicts_loaded( continue; } - // 3. Get dict codes. - std::vector dict_codes; - if (dict_column->is_nullable()) { - const ColumnNullable* nullable_column = - static_cast(dict_column.get()); - const ColumnString* nested_column = static_cast( - nullable_column->get_nested_column_ptr().get()); - for (int i = 0; i < nested_column->size(); ++i) { - StringRef dict_value = nested_column->get_data_at(i); - dict_codes.emplace_back(dict_value_to_code[dict_value]); - } - } else { - for (int i = 0; i < dict_column->size(); ++i) { - StringRef dict_value = dict_column->get_data_at(i); - dict_codes.emplace_back(dict_value_to_code[dict_value]); - } - } - // 4. Rewrite conjuncts. - RETURN_IF_ERROR(_rewrite_dict_conjuncts(dict_codes, slot_id, dict_column->is_nullable())); + RETURN_IF_ERROR(_rewrite_dict_conjuncts( + dict_codes, slot_id, temp_block.get_by_position(dict_pos).column->is_nullable())); ++it; } return Status::OK(); diff --git a/be/src/vec/exec/format/parquet/byte_array_dict_decoder.cpp b/be/src/vec/exec/format/parquet/byte_array_dict_decoder.cpp index 8b9532e68d0391..7d9f708011c4e5 100644 --- a/be/src/vec/exec/format/parquet/byte_array_dict_decoder.cpp +++ b/be/src/vec/exec/format/parquet/byte_array_dict_decoder.cpp @@ -44,7 +44,6 @@ Status ByteArrayDictDecoder::set_dict(std::unique_ptr& dict, int32_t total_length += l; } - _dict_value_to_code.reserve(num_values); // For insert_many_strings_overflow _dict_data.resize(total_length + ColumnString::MAX_STRINGS_OVERFLOW_SIZE); _max_value_length = 0; @@ -55,7 +54,6 @@ Status ByteArrayDictDecoder::set_dict(std::unique_ptr& dict, int32_t offset_cursor += 4; memcpy(&_dict_data[offset], dict_item_address + offset_cursor, l); _dict_items.emplace_back(&_dict_data[offset], l); - _dict_value_to_code[StringRef(&_dict_data[offset], l)] = i; offset_cursor += l; offset += l; if (offset_cursor > length) { @@ -77,15 +75,6 @@ Status ByteArrayDictDecoder::read_dict_values_to_column(MutableColumnPtr& doris_ return Status::OK(); } -Status ByteArrayDictDecoder::get_dict_codes(const ColumnString* string_column, - std::vector* dict_codes) { - for (int i = 0; i < string_column->size(); ++i) { - StringRef dict_value = string_column->get_data_at(i); - dict_codes->emplace_back(_dict_value_to_code[dict_value]); - } - return Status::OK(); -} - MutableColumnPtr ByteArrayDictDecoder::convert_dict_column_to_string_column( const ColumnInt32* dict_column) { auto res = ColumnString::create(); diff --git a/be/src/vec/exec/format/parquet/byte_array_dict_decoder.h b/be/src/vec/exec/format/parquet/byte_array_dict_decoder.h index 744a62165fb7fe..bb83d41813bb78 100644 --- a/be/src/vec/exec/format/parquet/byte_array_dict_decoder.h +++ b/be/src/vec/exec/format/parquet/byte_array_dict_decoder.h @@ -54,9 +54,6 @@ class ByteArrayDictDecoder final : public BaseDictDecoder { Status read_dict_values_to_column(MutableColumnPtr& doris_column) override; - Status get_dict_codes(const ColumnString* column_string, - std::vector* dict_codes) override; - MutableColumnPtr convert_dict_column_to_string_column(const ColumnInt32* dict_column) override; protected: @@ -64,6 +61,5 @@ class ByteArrayDictDecoder final : public BaseDictDecoder { std::vector _dict_items; std::vector _dict_data; size_t _max_value_length; - std::unordered_map _dict_value_to_code; }; } // namespace doris::vectorized diff --git a/be/src/vec/exec/format/parquet/decoder.h b/be/src/vec/exec/format/parquet/decoder.h index 57fecf4abfb7a2..1654878af80a29 100644 --- a/be/src/vec/exec/format/parquet/decoder.h +++ b/be/src/vec/exec/format/parquet/decoder.h @@ -78,11 +78,6 @@ class Decoder { return Status::NotSupported("read_dict_values_to_column is not supported"); } - virtual Status get_dict_codes(const ColumnString* column_string, - std::vector* dict_codes) { - return Status::NotSupported("get_dict_codes is not supported"); - } - virtual MutableColumnPtr convert_dict_column_to_string_column(const ColumnInt32* dict_column) { LOG(FATAL) << "Method convert_dict_column_to_string_column is not supported"; __builtin_unreachable(); diff --git a/be/src/vec/exec/format/parquet/fix_length_dict_decoder.hpp b/be/src/vec/exec/format/parquet/fix_length_dict_decoder.hpp index 65e329ae89b5a4..0bcc0bd5e73a40 100644 --- a/be/src/vec/exec/format/parquet/fix_length_dict_decoder.hpp +++ b/be/src/vec/exec/format/parquet/fix_length_dict_decoder.hpp @@ -109,10 +109,8 @@ class FixLengthDictDecoder final : public BaseDictDecoder { _dict = std::move(dict); char* dict_item_address = reinterpret_cast(_dict.get()); _dict_items.resize(num_values); - _dict_value_to_code.reserve(num_values); for (size_t i = 0; i < num_values; ++i) { _dict_items[i] = dict_item_address; - _dict_value_to_code[StringRef(_dict_items[i], _type_length)] = i; dict_item_address += _type_length; } return Status::OK(); @@ -128,17 +126,6 @@ class FixLengthDictDecoder final : public BaseDictDecoder { return Status::OK(); } - Status get_dict_codes(const ColumnString* string_column, - std::vector* dict_codes) override { - size_t size = string_column->size(); - dict_codes->reserve(size); - for (int i = 0; i < size; ++i) { - StringRef dict_value = string_column->get_data_at(i); - dict_codes->emplace_back(_dict_value_to_code[dict_value]); - } - return Status::OK(); - } - MutableColumnPtr convert_dict_column_to_string_column(const ColumnInt32* dict_column) override { auto res = ColumnString::create(); std::vector dict_values(dict_column->size()); @@ -149,7 +136,6 @@ class FixLengthDictDecoder final : public BaseDictDecoder { res->insert_many_strings(&dict_values[0], dict_values.size()); return res; } - std::unordered_map _dict_value_to_code; // For dictionary encoding std::vector _dict_items; }; diff --git a/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.h b/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.h index 79ee3cd646306c..a00a4683725762 100644 --- a/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.h @@ -183,11 +183,6 @@ class ColumnChunkReader { ->read_dict_values_to_column(doris_column); } - Status get_dict_codes(const ColumnString* column_string, std::vector* dict_codes) { - return _decoders[static_cast(tparquet::Encoding::RLE_DICTIONARY)]->get_dict_codes( - column_string, dict_codes); - } - MutableColumnPtr convert_dict_column_to_string_column(const ColumnInt32* dict_column) { return _decoders[static_cast(tparquet::Encoding::RLE_DICTIONARY)] ->convert_dict_column_to_string_column(dict_column); diff --git a/be/src/vec/exec/format/parquet/vparquet_column_reader.cpp b/be/src/vec/exec/format/parquet/vparquet_column_reader.cpp index c31c63ee87c36c..9c368b6a7a6fa5 100644 --- a/be/src/vec/exec/format/parquet/vparquet_column_reader.cpp +++ b/be/src/vec/exec/format/parquet/vparquet_column_reader.cpp @@ -454,11 +454,6 @@ Status ScalarColumnReader::read_dict_values_to_column(MutableColumnPtr& doris_co return Status::OK(); } -Status ScalarColumnReader::get_dict_codes(const ColumnString* column_string, - std::vector* dict_codes) { - return _chunk_reader->get_dict_codes(column_string, dict_codes); -} - MutableColumnPtr ScalarColumnReader::convert_dict_column_to_string_column( const ColumnInt32* dict_column) { return _chunk_reader->convert_dict_column_to_string_column(dict_column); diff --git a/be/src/vec/exec/format/parquet/vparquet_column_reader.h b/be/src/vec/exec/format/parquet/vparquet_column_reader.h index f0eadb8bcd61c5..4c6e5b1eac9f60 100644 --- a/be/src/vec/exec/format/parquet/vparquet_column_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_column_reader.h @@ -128,11 +128,6 @@ class ParquetColumnReader { return Status::NotSupported("read_dict_values_to_column is not supported"); } - virtual Status get_dict_codes(const ColumnString* column_string, - std::vector* dict_codes) { - return Status::NotSupported("get_dict_codes is not supported"); - } - virtual MutableColumnPtr convert_dict_column_to_string_column(const ColumnInt32* dict_column) { LOG(FATAL) << "Method convert_dict_column_to_string_column is not supported"; __builtin_unreachable(); @@ -180,8 +175,6 @@ class ScalarColumnReader : public ParquetColumnReader { ColumnSelectVector& select_vector, size_t batch_size, size_t* read_rows, bool* eof, bool is_dict_filter) override; Status read_dict_values_to_column(MutableColumnPtr& doris_column, bool* has_dict) override; - Status get_dict_codes(const ColumnString* column_string, - std::vector* dict_codes) override; MutableColumnPtr convert_dict_column_to_string_column(const ColumnInt32* dict_column) override; const std::vector& get_rep_level() const override { return _rep_levels; } const std::vector& get_def_level() const override { return _def_levels; } diff --git a/be/src/vec/exec/format/parquet/vparquet_group_reader.cpp b/be/src/vec/exec/format/parquet/vparquet_group_reader.cpp index 576ed58a284f8f..b993f4cd31e313 100644 --- a/be/src/vec/exec/format/parquet/vparquet_group_reader.cpp +++ b/be/src/vec/exec/format/parquet/vparquet_group_reader.cpp @@ -841,7 +841,7 @@ Status RowGroupReader::_rewrite_dict_predicates() { ++index; } - // 2.2 Execute conjuncts and filter block. + // 2.2 Execute conjuncts. VExprContextSPtrs ctxs; auto iter = _slot_id_to_filter_conjuncts->find(slot_id); if (iter != _slot_id_to_filter_conjuncts->end()) { @@ -854,33 +854,39 @@ Status RowGroupReader::_rewrite_dict_predicates() { return Status::NotFound(msg.str()); } - std::vector columns_to_filter(1, dict_pos); - int column_to_keep = temp_block.columns(); if (dict_pos != 0) { // VExprContext.execute has an optimization, the filtering is executed when block->rows() > 0 // The following process may be tricky and time-consuming, but we have no other way. temp_block.get_by_position(0).column->assume_mutable()->resize(dict_value_column_size); } + IColumn::Filter result_filter(temp_block.rows(), 1); + bool can_filter_all; { SCOPED_RAW_TIMER(&_predicate_filter_time); - RETURN_IF_ERROR_OR_CATCH_EXCEPTION(VExprContext::execute_conjuncts_and_filter_block( - ctxs, &temp_block, columns_to_filter, column_to_keep)); + RETURN_IF_ERROR(VExprContext::execute_conjuncts(ctxs, nullptr, &temp_block, + &result_filter, &can_filter_all)); } if (dict_pos != 0) { // We have to clean the first column to insert right data. temp_block.get_by_position(0).column->assume_mutable()->clear(); } - // Check some conditions. - ColumnPtr& dict_column = temp_block.get_by_position(dict_pos).column; - // If dict_column->size() == 0, can filter this row group. - if (dict_column->size() == 0) { + // If can_filter_all = true, can filter this row group. + if (can_filter_all) { _is_row_group_filtered = true; return Status::OK(); } + // 3. Get dict codes. + std::vector dict_codes; + for (size_t i = 0; i < result_filter.size(); ++i) { + if (result_filter[i]) { + dict_codes.emplace_back(i); + } + } + // About Performance: if dict_column size is too large, it will generate a large IN filter. - if (dict_column->size() > MAX_DICT_CODE_PREDICATE_TO_REWRITE) { + if (dict_codes.size() > MAX_DICT_CODE_PREDICATE_TO_REWRITE) { it = _dict_filter_cols.erase(it); for (auto& ctx : ctxs) { _filter_conjuncts.push_back(ctx); @@ -888,22 +894,9 @@ Status RowGroupReader::_rewrite_dict_predicates() { continue; } - // 3. Get dict codes. - std::vector dict_codes; - if (dict_column->is_nullable()) { - const ColumnNullable* nullable_column = - static_cast(dict_column.get()); - const ColumnString* nested_column = static_cast( - nullable_column->get_nested_column_ptr().get()); - RETURN_IF_ERROR(_column_readers[dict_filter_col_name]->get_dict_codes( - assert_cast(nested_column), &dict_codes)); - } else { - RETURN_IF_ERROR(_column_readers[dict_filter_col_name]->get_dict_codes( - assert_cast(dict_column.get()), &dict_codes)); - } - // 4. Rewrite conjuncts. - RETURN_IF_ERROR(_rewrite_dict_conjuncts(dict_codes, slot_id, dict_column->is_nullable())); + RETURN_IF_ERROR(_rewrite_dict_conjuncts( + dict_codes, slot_id, temp_block.get_by_position(dict_pos).column->is_nullable())); ++it; } return Status::OK(); From 52536421cc7830ef8a26f6ae10e494aaed77a3b7 Mon Sep 17 00:00:00 2001 From: daidai <2017501503@qq.com> Date: Thu, 29 Aug 2024 10:52:12 +0800 Subject: [PATCH 04/60] [fix](ut)fix be enable_http_auth ut (#40071) ## Proposed changes before pr #39577 --- be/test/http/http_auth_test.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/be/test/http/http_auth_test.cpp b/be/test/http/http_auth_test.cpp index 6077f871b7076e..f39e61aa6e2cf7 100644 --- a/be/test/http/http_auth_test.cpp +++ b/be/test/http/http_auth_test.cpp @@ -73,8 +73,7 @@ TEST_F(HttpAuthTest, enable_all_http_auth) { { auto evhttp_req = evhttp_request_new(nullptr, nullptr); HttpRequest req2(evhttp_req); - auto auth = encode_basic_auth("doris", "passwd"); - req2._headers.emplace(HttpHeaders::AUTHORIZATION, auth); + req2._headers.emplace(HttpHeaders::AUTHORIZATION, "Basic cm9vdDo="); EXPECT_EQ(s_auth_handler.on_header(&req2), -1); } @@ -82,8 +81,7 @@ TEST_F(HttpAuthTest, enable_all_http_auth) { { auto evhttp_req = evhttp_request_new(nullptr, nullptr); HttpRequest req3(evhttp_req); - auto auth = encode_basic_auth("doris", "passwd"); - req3._headers.emplace(HttpHeaders::AUTHORIZATION, auth); + req3._headers.emplace(HttpHeaders::AUTHORIZATION, "Basic cm9vdDo="); req3._params.emplace("table", "T"); EXPECT_EQ(s_auth_handler.on_header(&req3), 0); evhttp_request_free(evhttp_req); From 83e6eacdef2c9ccd51acd824956a44c0e6a750eb Mon Sep 17 00:00:00 2001 From: zclllhhjj Date: Thu, 29 Aug 2024 10:55:18 +0800 Subject: [PATCH 05/60] [Fix](load) fix commit txn timeout when loading to table with many tablet (#40031) fix commit txn timeout when loading to table with many tablet --- .../apache/doris/load/BrokerFileGroup.java | 16 +++++--- .../transaction/DatabaseTransactionMgr.java | 40 +++++++++++++------ 2 files changed, 38 insertions(+), 18 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/BrokerFileGroup.java b/fe/fe-core/src/main/java/org/apache/doris/load/BrokerFileGroup.java index d182e1d08d4d21..366983d4139cde 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/BrokerFileGroup.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/BrokerFileGroup.java @@ -152,12 +152,16 @@ public void parse(Database db, DataDescription dataDescription) throws DdlExcept } } - boolean isPartitionRestoring = olapTable.getPartitions().stream().anyMatch( - partition -> partition.getState() == PartitionState.RESTORE - ); - // restore table - if (!isPartitionRestoring && olapTable.getState() == OlapTableState.RESTORE) { - throw new DdlException("Table [" + olapTable.getName() + "] is under restore"); + // only do check when here's restore on this table now + if (olapTable.getState() == OlapTableState.RESTORE) { + boolean hasPartitionRestoring = olapTable.getPartitions().stream() + .anyMatch(partition -> partition.getState() == PartitionState.RESTORE); + // tbl RESTORE && all partition NOT RESTORE -> whole table restore + // tbl RESTORE && some partition RESTORE -> just partitions restore, NOT WHOLE TABLE + // so check wether the whole table restore here + if (!hasPartitionRestoring) { + throw new DdlException("Table [" + olapTable.getName() + "] is under restore"); + } } if (olapTable.getKeysType() != KeysType.AGG_KEYS && dataDescription.isNegative()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java index 3680f081d038dd..5e9c22bede7e82 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java @@ -509,7 +509,9 @@ private void checkCommitStatus(List tableList, TransactionState transacti List tabletIds = tabletCommitInfos.stream() .map(TabletCommitInfo::getTabletId).collect(Collectors.toList()); List tabletMetaList = tabletInvertedIndex.getTabletMetaList(tabletIds); + HashMap tableIdtoRestoring = new HashMap<>(); for (int i = 0; i < tabletMetaList.size(); i++) { + // get partition and table of this tablet TabletMeta tabletMeta = tabletMetaList.get(i); if (tabletMeta == TabletInvertedIndex.NOT_EXIST_TABLET_META) { continue; @@ -518,29 +520,43 @@ private void checkCommitStatus(List
tableList, TransactionState transacti long tableId = tabletMeta.getTableId(); OlapTable tbl = (OlapTable) idToTable.get(tableId); if (tbl == null) { - // this can happen when tableId == -1 (tablet being dropping) - // or table really not exist. + // this can happen when tableId == -1 (tablet being dropping) or table really not exist. continue; } + // check relative partition restore here long partitionId = tabletMeta.getPartitionId(); if (tbl.getPartition(partitionId) == null) { - // this can happen when partitionId == -1 (tablet being dropping) - // or partition really not exist. + // this can happen when partitionId == -1 (tablet being dropping) or partition really not exist. continue; - } else if (tbl.getPartition(partitionId).getState() == PartitionState.RESTORE) { + } + if (tbl.getPartition(partitionId).getState() == PartitionState.RESTORE) { // partition in restore process which can not load data throw new LoadException("Table [" + tbl.getName() + "], Partition [" + tbl.getPartition(partitionId).getName() + "] is in restore process. Can not load into it"); } - boolean isPartitionRestoring = tbl.getPartitions().stream().anyMatch( - partition -> partition.getState() == PartitionState.RESTORE - ); - // restore table - if (!isPartitionRestoring && tbl.getState() == OlapTableState.RESTORE) { - throw new LoadException("Table " + tbl.getName() + " is in restore process. " - + "Can not load into it"); + // only do check when here's restore on this table now + if (tbl.getState() == OlapTableState.RESTORE) { + boolean hasPartitionRestoring = false; + if (tableIdtoRestoring.containsKey(tableId)) { + hasPartitionRestoring = tableIdtoRestoring.get(tableId); + } else { + for (Partition partition : tbl.getPartitions()) { + if (partition.getState() == PartitionState.RESTORE) { + hasPartitionRestoring = true; + break; + } + } + tableIdtoRestoring.put(tableId, hasPartitionRestoring); + } + // tbl RESTORE && all partition NOT RESTORE -> whole table restore + // tbl RESTORE && some partition RESTORE -> just partitions restore, NOT WHOLE TABLE + // so check wether the whole table restore here + if (!hasPartitionRestoring) { + throw new LoadException( + "Table " + tbl.getName() + " is in restore process. " + "Can not load into it"); + } } if (!tableToPartition.containsKey(tableId)) { From 9ce66718f075fb4b3b79fce36f0b7f01d580919d Mon Sep 17 00:00:00 2001 From: Lijia Liu Date: Thu, 29 Aug 2024 10:59:06 +0800 Subject: [PATCH 06/60] [improvement](nereids) Support * EXCEPT, * REPLACE and other expressions to appear in select statement at the same time (#40014) Follow #36777 Sometimes, users want to use * EXCEPT and other expressions in select clause. For example: SELECT * EXCEPT (value), sum(value) FROM tbl GROUP BY 1 Co-authored-by: liutang123 --- .../org/apache/doris/nereids/DorisParser.g4 | 14 +- .../doris/nereids/analyzer/UnboundStar.java | 91 ++++++++++-- .../nereids/parser/LogicalPlanBuilder.java | 134 +++++++----------- .../LogicalPlanBuilderForCreateView.java | 49 ------- .../rules/analysis/BindExpression.java | 57 +++++--- .../trees/plans/logical/LogicalAggregate.java | 4 + .../trees/plans/logical/LogicalProject.java | 52 +++---- .../trees/expressions/SelectExceptTest.java | 98 +++++++------ .../trees/expressions/SelectReplaceTest.java | 18 ++- .../nereids/trees/plans/PlanToStringTest.java | 2 +- .../data/correctness/test_select_except.out | 9 ++ .../correctness/test_select_except.groovy | 13 ++ 12 files changed, 288 insertions(+), 253 deletions(-) create mode 100644 regression-test/data/correctness/test_select_except.out 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 1c695f29c329d2..5480c2a3215b5d 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 @@ -739,11 +739,6 @@ selectClause selectColumnClause : namedExpressionSeq - | ASTERISK exceptOrReplace+ - ; - -exceptOrReplace - : (EXCEPT | REPLACE) LEFT_PAREN namedExpressionSeq RIGHT_PAREN ; whereClause @@ -1134,8 +1129,8 @@ primaryExpression | name=CAST LEFT_PAREN expression AS castDataType RIGHT_PAREN #cast | constant #constantDefault | interval #intervalLiteral - | ASTERISK #star - | qualifiedName DOT ASTERISK #star + | ASTERISK (exceptOrReplace)* #star + | qualifiedName DOT ASTERISK (exceptOrReplace)* #star | CHAR LEFT_PAREN arguments+=expression (COMMA arguments+=expression)* (USING charSet=identifierOrText)? @@ -1158,6 +1153,11 @@ primaryExpression | primaryExpression COLLATE (identifier | STRING_LITERAL | DEFAULT) #collate ; +exceptOrReplace + : EXCEPT LEFT_PAREN namedExpressionSeq RIGHT_PAREN #except + | REPLACE LEFT_PAREN namedExpressionSeq RIGHT_PAREN #replace + ; + castDataType : dataType |(SIGNED|UNSIGNED) (INT|INTEGER)? 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 2875036eb07493..6d8ed904ec109d 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 @@ -30,6 +30,7 @@ import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.stream.Collectors; /** * Star expression. @@ -39,21 +40,78 @@ public class UnboundStar extends NamedExpression implements LeafExpression, Unbo // the start and end position of the sql substring(e.g. "*", "table.*") private final Optional> indexInSqlString; + private final List exceptedSlots; + + private final List replacedAlias; + public UnboundStar(List qualifier) { super(ImmutableList.of()); this.qualifier = Objects.requireNonNull(ImmutableList.copyOf(qualifier), "qualifier can not be null"); this.indexInSqlString = Optional.empty(); + this.exceptedSlots = ImmutableList.of(); + this.replacedAlias = ImmutableList.of(); } public UnboundStar(List qualifier, Optional> indexInSqlString) { super(ImmutableList.of()); this.qualifier = Objects.requireNonNull(ImmutableList.copyOf(qualifier), "qualifier can not be null"); this.indexInSqlString = indexInSqlString; + this.exceptedSlots = ImmutableList.of(); + this.replacedAlias = ImmutableList.of(); + } + + /** + * The star expression is used in the select list, and the exceptedSlots is the column that is not selected. + * + * @param qualifier for example, "table1.*" + * @param exceptedSlots for example, * EXCEPT(a, b) + * @param replacedAlias for example, * REPLACE(a + 1 AS a) + */ + public UnboundStar(List qualifier, List exceptedSlots, + List replacedAlias) { + super(ImmutableList.of()); + this.qualifier = Objects.requireNonNull(ImmutableList.copyOf(qualifier), "qualifier can not be null"); + this.indexInSqlString = Optional.empty(); + this.exceptedSlots = Objects.requireNonNull(ImmutableList.copyOf(exceptedSlots), + "except columns can not be null"); + this.replacedAlias = Objects.requireNonNull(ImmutableList.copyOf(replacedAlias), + "replace columns can not be null"); + } + + /** + * The star expression is used in the select list, and the exceptedSlots is the column that is not selected. + * + * @param qualifier for example, "table1.*" + * @param exceptedSlots for example, * EXCEPT(a, b) + * @param replacedAlias for example, * REPLACE(a + 1 AS a) + * @param indexInSqlString see {@link UnboundStar#indexInSqlString} + */ + public UnboundStar(List qualifier, List exceptedSlots, List replacedAlias, + Optional> indexInSqlString) { + super(ImmutableList.of()); + this.qualifier = Objects.requireNonNull(ImmutableList.copyOf(qualifier), "qualifier can not be null"); + this.indexInSqlString = indexInSqlString; + this.exceptedSlots = Objects.requireNonNull(ImmutableList.copyOf(exceptedSlots), + "except columns can not be null"); + this.replacedAlias = Objects.requireNonNull(ImmutableList.copyOf(replacedAlias), + "replace columns can not be null"); } @Override public String toSql() { - return Utils.qualifiedName(qualifier, "*"); + StringBuilder builder = new StringBuilder(); + builder.append(Utils.qualifiedName(qualifier, "*")); + if (!exceptedSlots.isEmpty()) { + String exceptStr = exceptedSlots.stream().map(NamedExpression::toSql) + .collect(Collectors.joining(", ", " EXCEPT(", ")")); + builder.append(exceptStr); + } + if (!replacedAlias.isEmpty()) { + String replaceStr = replacedAlias.stream().map(NamedExpression::toSql) + .collect(Collectors.joining(", ", " REPLACE(", ")")); + builder.append(replaceStr); + } + return builder.toString(); } @Override @@ -66,6 +124,15 @@ public String toString() { return toSql(); } + public Optional> getIndexInSqlString() { + return indexInSqlString; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitUnboundStar(this, context); + } + @Override public boolean equals(Object o) { if (this == o) { @@ -78,24 +145,24 @@ public boolean equals(Object o) { return false; } UnboundStar that = (UnboundStar) o; - return qualifier.equals(that.qualifier); - } - - public Optional> getIndexInSqlString() { - return indexInSqlString; + return qualifier.equals(that.qualifier) && exceptedSlots.equals(that.exceptedSlots) && replacedAlias.equals( + that.replacedAlias); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), qualifier); + return Objects.hash(super.hashCode(), qualifier, exceptedSlots, replacedAlias); } - @Override - public R accept(ExpressionVisitor visitor, C context) { - return visitor.visitUnboundStar(this, context); + public UnboundStar withIndexInSql(Pair index) { + return new UnboundStar(qualifier, exceptedSlots, replacedAlias, Optional.ofNullable(index)); } - public UnboundStar withIndexInSql(Pair index) { - return new UnboundStar(qualifier, Optional.ofNullable(index)); + public List getReplacedAlias() { + return replacedAlias; + } + + public List getExceptedSlots() { + return exceptedSlots; } } 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 d82fd817370c08..af7d6cd63affd2 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 @@ -96,6 +96,8 @@ import org.apache.doris.nereids.DorisParser.DropMTMVContext; import org.apache.doris.nereids.DorisParser.DropProcedureContext; import org.apache.doris.nereids.DorisParser.ElementAtContext; +import org.apache.doris.nereids.DorisParser.ExceptContext; +import org.apache.doris.nereids.DorisParser.ExceptOrReplaceContext; import org.apache.doris.nereids.DorisParser.ExistContext; import org.apache.doris.nereids.DorisParser.ExplainContext; import org.apache.doris.nereids.DorisParser.ExportContext; @@ -160,6 +162,7 @@ import org.apache.doris.nereids.DorisParser.RefreshTriggerContext; import org.apache.doris.nereids.DorisParser.RegularQuerySpecificationContext; import org.apache.doris.nereids.DorisParser.RelationContext; +import org.apache.doris.nereids.DorisParser.ReplaceContext; import org.apache.doris.nereids.DorisParser.ResumeMTMVContext; import org.apache.doris.nereids.DorisParser.RollupDefContext; import org.apache.doris.nereids.DorisParser.RollupDefsContext; @@ -488,7 +491,6 @@ import org.antlr.v4.runtime.tree.ParseTree; import org.antlr.v4.runtime.tree.RuleNode; import org.antlr.v4.runtime.tree.TerminalNode; -import org.springframework.util.CollectionUtils; import java.math.BigDecimal; import java.math.BigInteger; @@ -1355,10 +1357,6 @@ public LogicalPlan visitRegularQuerySpecification(RegularQuerySpecificationConte LogicalPlan selectPlan; LogicalPlan relation; if (ctx.fromClause() == null) { - SelectColumnClauseContext columnCtx = selectCtx.selectColumnClause(); - if (!CollectionUtils.isEmpty(columnCtx.exceptOrReplace())) { - throw new ParseException("select with modifiers cannot be used in one row relation", selectCtx); - } relation = new UnboundOneRowRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of(new UnboundAlias(Literal.of(0)))); } else { @@ -1521,7 +1519,46 @@ public Expression visitStar(StarContext ctx) { } else { target = ImmutableList.of(); } - return new UnboundStar(target); + List exceptOrReplaceList = ctx.exceptOrReplace(); + if (exceptOrReplaceList != null && !exceptOrReplaceList.isEmpty()) { + List finalExpectSlots = ImmutableList.of(); + List finalReplacedAlias = ImmutableList.of(); + for (ExceptOrReplaceContext exceptOrReplace : exceptOrReplaceList) { + if (exceptOrReplace instanceof ExceptContext) { + if (!finalExpectSlots.isEmpty()) { + throw new ParseException("only one except clause is supported", ctx); + } + ExceptContext exceptContext = (ExceptContext) exceptOrReplace; + List expectSlots = getNamedExpressions(exceptContext.namedExpressionSeq()); + boolean allSlots = expectSlots.stream().allMatch(UnboundSlot.class::isInstance); + if (expectSlots.isEmpty() || !allSlots) { + throw new ParseException( + "only column name is supported in except clause", ctx); + } + finalExpectSlots = expectSlots; + } else if (exceptOrReplace instanceof ReplaceContext) { + if (!finalReplacedAlias.isEmpty()) { + throw new ParseException("only one replace clause is supported", ctx); + } + ReplaceContext replaceContext = (ReplaceContext) exceptOrReplace; + List expectAlias = getNamedExpressions(replaceContext.namedExpressionSeq()); + boolean allAlias = expectAlias.stream() + .allMatch(e -> e instanceof UnboundAlias + && ((UnboundAlias) e).getAlias().isPresent()); + if (expectAlias.isEmpty() || !allAlias) { + throw new ParseException( + "only alias is supported in select-replace clause", ctx); + } + finalReplacedAlias = expectAlias; + } else { + throw new ParseException("Unsupported except or replace clause: " + exceptOrReplace.getText(), + ctx); + } + } + return new UnboundStar(target, finalExpectSlots, finalReplacedAlias); + } else { + return new UnboundStar(target); + } }); } @@ -3032,46 +3069,8 @@ protected LogicalPlan withSelectQuerySpecification( if (!(aggregate instanceof Aggregate) && havingClause.isPresent()) { // create a project node for pattern match of ProjectToGlobalAggregate rule // then ProjectToGlobalAggregate rule can insert agg node as LogicalHaving node's child - LogicalPlan project; - if (!CollectionUtils.isEmpty(selectColumnCtx.exceptOrReplace())) { - List except = ImmutableList.of(); - List replace = ImmutableList.of(); - boolean hasExcept = false; - boolean hasReplace = false; - for (DorisParser.ExceptOrReplaceContext er : selectColumnCtx.exceptOrReplace()) { - if ((hasReplace && er.REPLACE() != null) || (hasExcept && er.EXCEPT() != null)) { - throw new ParseException("only one except or replace is supported", selectColumnCtx); - } - if (er.EXCEPT() != null) { - hasExcept = true; - except = getNamedExpressions(er.namedExpressionSeq()); - if (!except.stream().allMatch(UnboundSlot.class::isInstance)) { - throw new ParseException( - "only column name is supported in except clause", selectColumnCtx); - } - } else if (er.REPLACE() != null) { - hasReplace = true; - replace = getNamedExpressions(er.namedExpressionSeq()); - boolean isEmpty = replace.isEmpty(); - boolean allAlias = replace.stream() - .allMatch(e -> e instanceof UnboundAlias - && ((UnboundAlias) e).getAlias().isPresent()); - if (isEmpty || !allAlias) { - throw new ParseException( - "only alias is supported in select-replace clause", selectColumnCtx); - } - } else { - throw new ParseException("only except or replace is supported", selectColumnCtx); - } - } - UnboundStar star = new UnboundStar(ImmutableList.of()); - project = new LogicalProject<>( - ImmutableList.of(star), except, replace, isDistinct, aggregate); - } else { - List projects = getNamedExpressions(selectColumnCtx.namedExpressionSeq()); - project = new LogicalProject<>( - projects, ImmutableList.of(), ImmutableList.of(), isDistinct, aggregate); - } + List projects = getNamedExpressions(selectColumnCtx.namedExpressionSeq()); + LogicalPlan project = new LogicalProject<>(projects, isDistinct, aggregate); return new LogicalHaving<>(ExpressionUtils.extractConjunctionToSet( getExpression((havingClause.get().booleanExpression()))), project); } else { @@ -3268,49 +3267,18 @@ protected LogicalPlan withProjection(LogicalPlan input, SelectColumnClauseContex if (aggCtx.isPresent()) { if (isDistinct) { return new LogicalProject<>(ImmutableList.of(new UnboundStar(ImmutableList.of())), - Collections.emptyList(), Collections.emptyList(), isDistinct, input); + isDistinct, input); } else { return input; } } else { - if (!CollectionUtils.isEmpty(selectCtx.exceptOrReplace())) { - List except = ImmutableList.of(); - List replace = ImmutableList.of(); - boolean hasExcept = false; - boolean hasReplace = false; - for (DorisParser.ExceptOrReplaceContext er : selectCtx.exceptOrReplace()) { - if ((hasReplace && er.REPLACE() != null) || (hasExcept && er.EXCEPT() != null)) { - throw new ParseException("only one except or replace is supported", selectCtx); - } - if (er.EXCEPT() != null) { - hasExcept = true; - except = getNamedExpressions(er.namedExpressionSeq()); - if (!except.stream().allMatch(UnboundSlot.class::isInstance)) { - throw new ParseException( - "only column name is supported in except clause", selectCtx); - } - } else if (er.REPLACE() != null) { - hasReplace = true; - replace = getNamedExpressions(er.namedExpressionSeq()); - boolean isEmpty = replace.isEmpty(); - boolean allAlias = replace.stream() - .allMatch(e -> e instanceof UnboundAlias - && ((UnboundAlias) e).getAlias().isPresent()); - if (isEmpty || !allAlias) { - throw new ParseException( - "only alias is supported in select-replace clause", selectCtx); - } - } else { - throw new ParseException("only except or replace is supported", selectCtx); - } + List projects = getNamedExpressions(selectCtx.namedExpressionSeq()); + if (input instanceof UnboundOneRowRelation) { + if (projects.stream().anyMatch(project -> project instanceof UnboundStar)) { + throw new ParseException("SELECT * must have a FROM clause"); } - UnboundStar star = new UnboundStar(ImmutableList.of()); - return new LogicalProject<>(ImmutableList.of(star), except, replace, isDistinct, input); - } else { - List projects = getNamedExpressions(selectCtx.namedExpressionSeq()); - return new LogicalProject<>( - projects, Collections.emptyList(), Collections.emptyList(), isDistinct, input); } + return new LogicalProject<>(projects, isDistinct, input); } }); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilderForCreateView.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilderForCreateView.java index 2109ea5969d0bd..cb5005d48835f0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilderForCreateView.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilderForCreateView.java @@ -24,17 +24,13 @@ import org.apache.doris.nereids.DorisParser.ColumnReferenceContext; import org.apache.doris.nereids.DorisParser.DereferenceContext; import org.apache.doris.nereids.DorisParser.GroupingElementContext; -import org.apache.doris.nereids.DorisParser.HavingClauseContext; import org.apache.doris.nereids.DorisParser.IdentifierContext; import org.apache.doris.nereids.DorisParser.LateralViewContext; import org.apache.doris.nereids.DorisParser.MultipartIdentifierContext; import org.apache.doris.nereids.DorisParser.NamedExpressionContext; -import org.apache.doris.nereids.DorisParser.SelectClauseContext; -import org.apache.doris.nereids.DorisParser.SelectColumnClauseContext; import org.apache.doris.nereids.DorisParser.StarContext; import org.apache.doris.nereids.DorisParser.TableAliasContext; import org.apache.doris.nereids.DorisParser.TableNameContext; -import org.apache.doris.nereids.DorisParser.WhereClauseContext; import org.apache.doris.nereids.analyzer.UnboundFunction; import org.apache.doris.nereids.analyzer.UnboundRelation; import org.apache.doris.nereids.analyzer.UnboundSlot; @@ -42,19 +38,15 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.logical.LogicalHaving; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; -import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; import org.apache.doris.nereids.util.Utils; import org.apache.doris.qe.ConnectContext; import com.google.common.collect.ImmutableList; import org.antlr.v4.runtime.ParserRuleContext; -import org.springframework.util.CollectionUtils; import java.util.Map; -import java.util.Optional; /**LogicalPlanBuilderForCreateView*/ public class LogicalPlanBuilderForCreateView extends LogicalPlanBuilder { @@ -94,47 +86,6 @@ public LogicalSubQueryAlias visitAliasQuery(AliasQueryContext ctx) { return super.visitAliasQuery(ctx); } - @Override - protected LogicalPlan withSelectQuerySpecification( - ParserRuleContext ctx, - LogicalPlan inputRelation, - SelectClauseContext selectClause, - Optional whereClause, - Optional aggClause, - Optional havingClause) { - LogicalPlan plan = super.withSelectQuerySpecification(ctx, inputRelation, selectClause, whereClause, - aggClause, havingClause); - SelectColumnClauseContext selectColumnCtx = selectClause.selectColumnClause(); - if ((!aggClause.isPresent() || isRepeat(aggClause.get())) && havingClause.isPresent() - && !CollectionUtils.isEmpty(selectColumnCtx.exceptOrReplace()) - && plan instanceof LogicalHaving && plan.child(0) instanceof LogicalProject) { - LogicalHaving> having = (LogicalHaving) plan; - LogicalProject project = having.child(); - UnboundStar star = (UnboundStar) project.getProjects().get(0); - star = star.withIndexInSql(Pair.of(selectColumnCtx.start.getStartIndex(), - selectColumnCtx.stop.getStopIndex())); - project = project.withProjects(ImmutableList.of(star)); - return (LogicalPlan) plan.withChildren(project); - } else { - return plan; - } - } - - @Override - protected LogicalPlan withProjection(LogicalPlan input, SelectColumnClauseContext selectCtx, - Optional aggCtx, boolean isDistinct) { - LogicalPlan plan = super.withProjection(input, selectCtx, aggCtx, isDistinct); - if (!aggCtx.isPresent() && !CollectionUtils.isEmpty(selectCtx.exceptOrReplace()) - && plan instanceof LogicalProject) { - LogicalProject project = (LogicalProject) plan; - UnboundStar star = (UnboundStar) project.getProjects().get(0); - star = star.withIndexInSql(Pair.of(selectCtx.start.getStartIndex(), selectCtx.stop.getStopIndex())); - return project.withProjects(ImmutableList.of(star)); - } else { - return plan; - } - } - @Override protected LogicalPlan withTableAlias(LogicalPlan plan, TableAliasContext ctx) { if (ctx.strictIdentifier() == null) { 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 cb36128d6c3338..ea1d923f83a3f9 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 @@ -620,29 +620,26 @@ private Plan bindProject(MatchingContext> ctx) { SimpleExprAnalyzer analyzer = buildSimpleExprAnalyzer( project, cascadesContext, project.children(), true, true); - List excepts = project.getExcepts(); - Supplier> boundExcepts = Suppliers.memoize( - () -> analyzer.analyzeToSet(project.getExcepts())); - - List replaces = project.getReplaces(); - Supplier> boundReplaces = Suppliers.memoize( - () -> analyzer.analyzeToList(project.getReplaces())); - - Builder boundProjections = ImmutableList.builderWithExpectedSize(project.arity()); + Builder boundProjections = ImmutableList.builderWithExpectedSize(project.getProjects().size()); StatementContext statementContext = ctx.statementContext; for (Expression expression : project.getProjects()) { Expression expr = analyzer.analyze(expression); if (!(expr instanceof BoundStar)) { boundProjections.add((NamedExpression) expr); } else { + UnboundStar unboundStar = (UnboundStar) expression; + List excepts = unboundStar.getExceptedSlots(); + Set boundExcepts = Suppliers.memoize(() -> analyzer.analyzeToSet(excepts)).get(); BoundStar boundStar = (BoundStar) expr; - List slots = boundStar.getSlots(); - if (!excepts.isEmpty()) { - slots = Utils.filterImmutableList(slots, slot -> !boundExcepts.get().contains((Slot) slot)); - } + + List slots = exceptStarSlots(boundExcepts, boundStar); + + List replaces = unboundStar.getReplacedAlias(); if (!replaces.isEmpty()) { final Map replaceMap = new HashMap<>(); final Set replaced = new HashSet<>(); + Supplier> boundReplaces = Suppliers.memoize( + () -> analyzer.analyzeToList(replaces)); for (NamedExpression replace : boundReplaces.get()) { Preconditions.checkArgument(replace instanceof Alias); Alias alias = (Alias) replace; @@ -654,7 +651,7 @@ private Plan bindProject(MatchingContext> ctx) { replaceMap.put(slot, alias); } - Collection c = CollectionUtils.intersection(boundExcepts.get(), replaceMap.keySet()); + Collection c = CollectionUtils.intersection(boundExcepts, replaceMap.keySet()); if (!c.isEmpty()) { throw new AnalysisException("Replace column name: " + c + " is in excepts"); } @@ -676,7 +673,6 @@ private Plan bindProject(MatchingContext> ctx) { // for create view stmt expand star List slotsForLambda = slots; - UnboundStar unboundStar = (UnboundStar) expression; unboundStar.getIndexInSqlString().ifPresent(pair -> { statementContext.addIndexInSqlToString(pair, toSqlWithBackquote(slotsForLambda)); }); @@ -701,6 +697,17 @@ private Plan bindFilter(MatchingContext> ctx) { return new LogicalFilter<>(boundConjuncts.build(), filter.child()); } + private List exceptStarSlots(Set boundExcepts, BoundStar boundStar) { + List slots = boundStar.getSlots(); + if (!boundExcepts.isEmpty()) { + slots = Utils.filterImmutableList(slots, slot -> !boundExcepts.contains(slot)); + if (slots.isEmpty()) { + throw new AnalysisException("All slots in * EXCEPT clause are excepted"); + } + } + return slots; + } + private Plan bindAggregate(MatchingContext> ctx) { LogicalAggregate agg = ctx.root; CascadesContext cascadesContext = ctx.cascadesContext; @@ -709,13 +716,29 @@ private Plan bindAggregate(MatchingContext> ctx) { agg, cascadesContext, agg.children(), true, true); List boundAggOutput = aggOutputAnalyzer.analyzeToList(agg.getOutputExpressions()); List boundProjections = new ArrayList<>(boundAggOutput.size()); - for (NamedExpression output : boundAggOutput) { + for (int i = 0; i < boundAggOutput.size(); i++) { + NamedExpression output = boundAggOutput.get(i); if (output instanceof BoundStar) { - boundProjections.addAll(((BoundStar) output).getSlots()); + UnboundStar unboundStar = (UnboundStar) agg.getOutputExpression(i); + if (!unboundStar.getReplacedAlias().isEmpty()) { + // If user use * replace in agg, witch slot the group by clause should reference is ambiguous. + // For example, select * replace (col % 2 as col) from t group by col + // The sql should be rewritten like follows: + // select * except (col), col % 2 as col1 from t group by ..., col1 + // or + // select * except (col), col % 2 as col1 from t group by ..., col -- group by origin col in t + throw new AnalysisException("* REPLACE in agg clause is not supported, use * EXCEPT instead"); + } + List excepts = unboundStar.getExceptedSlots(); + Set boundExcepts = Suppliers.memoize( + () -> aggOutputAnalyzer.analyzeToSet(excepts)).get(); + List slots = exceptStarSlots(boundExcepts, (BoundStar) output); + boundProjections.addAll(slots); } else { boundProjections.add(output); } } + Supplier aggOutputScopeWithoutAggFun = buildAggOutputScopeWithoutAggFun(boundProjections, cascadesContext); List boundGroupBy = bindGroupBy( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java index 9f65988da95358..2798b1aef0102b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java @@ -157,6 +157,10 @@ public List getOutputExpressions() { return outputExpressions; } + public NamedExpression getOutputExpression(int index) { + return outputExpressions.get(index); + } + public String getOutputExprsSql() { return outputExpressions.stream().map(ExpressionTrait::toSql).collect(Collectors.joining(", ")); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java index f6e076d87b8962..1484030e9c25a9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalProject.java @@ -58,28 +58,24 @@ public class LogicalProject extends LogicalUnary projects; private final Supplier> projectsSet; - private final List excepts; - private final List replaces; private final boolean isDistinct; public LogicalProject(List projects, CHILD_TYPE child) { - this(projects, ImmutableList.of(), ImmutableList.of(), false, ImmutableList.of(child)); + this(projects, false, ImmutableList.of(child)); } - public LogicalProject(List projects, List excepts, List replaces, - boolean isDistinct, List child) { - this(projects, excepts, replaces, isDistinct, Optional.empty(), Optional.empty(), child); + public LogicalProject(List projects, boolean isDistinct, List child) { + this(projects, isDistinct, Optional.empty(), Optional.empty(), child); } - public LogicalProject(List projects, List excepts, List replaces, - boolean isDistinct, Plan child) { - this(projects, excepts, replaces, isDistinct, + public LogicalProject(List projects, boolean isDistinct, Plan child) { + this(projects, isDistinct, Optional.empty(), Optional.empty(), ImmutableList.of(child)); } - private LogicalProject(List projects, List excepts, - List replaces, boolean isDistinct, Optional groupExpression, - Optional logicalProperties, List child) { + private LogicalProject(List projects, boolean isDistinct, + Optional groupExpression, Optional logicalProperties, + List child) { super(PlanType.LOGICAL_PROJECT, groupExpression, logicalProperties, child); Preconditions.checkArgument(projects != null, "projects can not be null"); // only ColumnPrune rule may produce empty projects, this happens in rewrite phase @@ -90,8 +86,6 @@ private LogicalProject(List projects, List exc ? ImmutableList.of(ExpressionUtils.selectMinimumColumn(child.get(0).getOutput())) : projects; this.projectsSet = Suppliers.memoize(() -> ImmutableSet.copyOf(this.projects)); - this.excepts = Utils.fastToImmutableList(excepts); - this.replaces = Utils.fastToImmutableList(replaces); this.isDistinct = isDistinct; } @@ -105,14 +99,6 @@ public List getProjects() { return projects; } - public List getExcepts() { - return excepts; - } - - public List getReplaces() { - return replaces; - } - @Override public List computeOutput() { Builder slots = ImmutableList.builderWithExpectedSize(projects.size()); @@ -126,9 +112,7 @@ public List computeOutput() { public String toString() { return Utils.toSqlString("LogicalProject[" + id.asInt() + "]", "distinct", isDistinct, - "projects", projects, - "excepts", excepts, - "replaces", replaces + "projects", projects ); } @@ -152,8 +136,6 @@ public boolean equals(Object o) { } LogicalProject that = (LogicalProject) o; boolean equal = projectsSet.get().equals(that.projectsSet.get()) - && excepts.equals(that.excepts) - && replaces.equals(that.replaces) && isDistinct == that.isDistinct; // TODO: should add exprId for UnBoundStar and BoundStar for equality comparison if (!projects.isEmpty() && (projects.get(0) instanceof UnboundStar || projects.get(0) instanceof BoundStar)) { @@ -164,18 +146,18 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(projectsSet.get(), excepts, replaces, isDistinct); + return Objects.hash(projectsSet.get(), isDistinct); } @Override public LogicalProject withChildren(List children) { Preconditions.checkArgument(children.size() == 1); - return new LogicalProject<>(projects, excepts, replaces, isDistinct, Utils.fastToImmutableList(children)); + return new LogicalProject<>(projects, isDistinct, Utils.fastToImmutableList(children)); } @Override public LogicalProject withGroupExpression(Optional groupExpression) { - return new LogicalProject<>(projects, excepts, replaces, isDistinct, + return new LogicalProject<>(projects, isDistinct, groupExpression, Optional.of(getLogicalProperties()), children); } @@ -183,20 +165,20 @@ public LogicalProject withGroupExpression(Optional groupE public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { Preconditions.checkArgument(children.size() == 1); - return new LogicalProject<>(projects, excepts, replaces, isDistinct, + return new LogicalProject<>(projects, isDistinct, groupExpression, logicalProperties, children); } public LogicalProject withProjects(List projects) { - return new LogicalProject<>(projects, excepts, replaces, isDistinct, children); + return new LogicalProject<>(projects, isDistinct, children); } public LogicalProject withProjectsAndChild(List projects, Plan child) { - return new LogicalProject<>(projects, excepts, replaces, isDistinct, ImmutableList.of(child)); + return new LogicalProject<>(projects, isDistinct, ImmutableList.of(child)); } public LogicalProject withDistinct(boolean isDistinct) { - return new LogicalProject<>(projects, excepts, replaces, isDistinct, children); + return new LogicalProject<>(projects, isDistinct, children); } public boolean isDistinct() { @@ -218,8 +200,6 @@ public JSONObject toJson() { JSONObject logicalProject = super.toJson(); JSONObject properties = new JSONObject(); properties.put("Projects", projects.toString()); - properties.put("Excepts", excepts.toString()); - properties.put("Replaces", replaces.toString()); properties.put("IsDistinct", isDistinct); logicalProject.put("Properties", properties); return logicalProject; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/SelectExceptTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/SelectExceptTest.java index b0f29714478115..d4dce7e80648a1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/SelectExceptTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/SelectExceptTest.java @@ -19,7 +19,10 @@ import org.apache.doris.nereids.analyzer.UnboundSlot; import org.apache.doris.nereids.analyzer.UnboundStar; +import org.apache.doris.nereids.datasets.tpch.AnalyzeCheckTestBase; +import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.exceptions.ParseException; +import org.apache.doris.nereids.pattern.PatternDescriptor; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.util.MemoPatternMatchSupported; @@ -31,22 +34,35 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -class SelectExceptTest implements MemoPatternMatchSupported { +class SelectExceptTest extends AnalyzeCheckTestBase implements MemoPatternMatchSupported { + + @Override + protected void runBeforeAll() throws Exception { + createDatabase("test"); + connectContext.setDatabase("test"); + String t = "create table t1(" + + "id int, \n" + + "value int)\n" + + "distributed by hash(id) buckets 1\n" + + "properties('replication_num' = '1');"; + createTables(t); + } + @Test void testExcept() { LogicalOlapScan olapScan = PlanConstructor.newLogicalOlapScan(0, "t1", 1); LogicalProject project = new LogicalProject<>( - ImmutableList.of(new UnboundStar(ImmutableList.of("db", "t1"))), - ImmutableList.of(new UnboundSlot("db", "t1", "id")), - ImmutableList.of(), - false, - olapScan); + ImmutableList.of( + new UnboundStar(ImmutableList.of("db", "t1"), + ImmutableList.of(new UnboundSlot("db", "t1", "id")), + ImmutableList.of() + )), false, olapScan); PlanChecker.from(MemoTestUtils.createConnectContext()) .analyze(project) .matches( logicalProject( logicalOlapScan() - ).when(proj -> proj.getExcepts().size() == 1 && proj.getProjects().size() == 1) + ).when(proj -> proj.getProjects().size() == 1 && proj.getProjects().get(0).getName().equals("name")) ); } @@ -59,52 +75,52 @@ void testParse() { logicalCheckPolicy( unboundRelation() ) - ).when(project -> project.getExcepts().size() == 2 - && project.getProjects().get(0) instanceof UnboundStar) + ).when(project -> project.getProjects().size() == 1 + && project.getProjects().get(0) instanceof UnboundStar + && ((UnboundStar) project.getProjects().get(0)).getExceptedSlots().size() == 2) )); String sql2 = "select k1, k2, v1, v2 except(v1, v2) from t1"; Assertions.assertThrows(ParseException.class, () -> PlanChecker.from(MemoTestUtils.createConnectContext()) - .checkParse(sql2, (checker) -> checker.matches( - logicalProject( - logicalCheckPolicy( - unboundRelation() - ) - ).when(project -> project.getExcepts().size() == 2 - && project.getProjects().get(0) instanceof UnboundStar) - ))); + .parse(sql2)); String sql3 = "select * except(v1, v2)"; Assertions.assertThrows(ParseException.class, () -> PlanChecker.from(MemoTestUtils.createConnectContext()) - .checkParse(sql3, (checker) -> checker.matches( - logicalProject( - logicalCheckPolicy( - unboundRelation() - ) - ).when(project -> project.getExcepts().size() == 2 - && project.getProjects().get(0) instanceof UnboundStar) - ))); + .parse(sql3)); String sql4 = "select * except() from t1"; Assertions.assertThrows(ParseException.class, () -> PlanChecker.from(MemoTestUtils.createConnectContext()) - .checkParse(sql4, (checker) -> checker.matches( - logicalProject( - logicalCheckPolicy( - unboundRelation() - ) - ).when(project -> project.getExcepts().size() == 2 - && project.getProjects().get(0) instanceof UnboundStar) - ))); + .parse(sql4)); String sql5 = "select * except(v1 + v2, v3 as k3) from t1"; Assertions.assertThrows(ParseException.class, () -> PlanChecker.from(MemoTestUtils.createConnectContext()) - .checkParse(sql5, (checker) -> checker.matches( - logicalProject( - logicalCheckPolicy( - unboundRelation() - ) - ).when(project -> project.getExcepts().size() == 2 - && project.getProjects().get(0) instanceof UnboundStar) - ))); + .parse(sql5)); + + String sql6 = "select * except(id name) from t1"; + Assertions.assertThrows(ParseException.class, () -> PlanChecker.from(MemoTestUtils.createConnectContext()) + .parse(sql6)); + } + + @Test + public void testExceptAnalyze() { + PatternDescriptor expected = logicalAggregate( + logicalProject( + logicalOlapScan() + ).when(proj -> proj.getProjects().size() == 2) + ).when(agg -> agg.getOutputExpressions().size() == 2 + && agg.getOutputExpression(0).getName().equals("id") + && agg.getOutputExpression(1).getName().equals("value") + && agg.getGroupByExpressions().size() == 1 + ); + String sql1 = "select * except(value), sum(value) as value from t1 group by id"; + PlanChecker.from(connectContext) + .parse(sql1).analyze().matches(expected); + + String sql2 = "select * except(value), sum(value) as value from t1 group by 1"; + PlanChecker.from(connectContext) + .parse(sql2).analyze().matches(expected); + + String sql3 = "select * except(id, value) from t1"; // All slots in * EXCEPT clause are excepted + Assertions.assertThrows(AnalysisException.class, () -> PlanChecker.from(connectContext).parse(sql3).analyze()); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/SelectReplaceTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/SelectReplaceTest.java index 47c079d45548c0..a002fe1a9c04a9 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/SelectReplaceTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/SelectReplaceTest.java @@ -88,24 +88,28 @@ void testReplace() { String sql6 = "select * replace (1 as fake) from t1"; Assertions.assertThrows(NereidsException.class, () -> PlanChecker.from(connectContext).checkPlannerResult(sql6)); + + // agg not support replace + String sql7 = "select * replace (v2 + 1 as v2) from t2 group by id, k1, v2"; + Assertions.assertThrows(NereidsException.class, + () -> PlanChecker.from(connectContext).checkPlannerResult(sql7)); } @Test public void testReplace2() { LogicalOlapScan olapScan = PlanConstructor.newLogicalOlapScan(0, "t1", 1); LogicalProject project = new LogicalProject<>( - ImmutableList.of(new UnboundStar(ImmutableList.of("db", "t1"))), - ImmutableList.of(), - ImmutableList.of(new UnboundAlias(new UnboundSlot("name"), "id")), - false, - olapScan); + ImmutableList.of( + new UnboundStar(ImmutableList.of("db", "t1"), + ImmutableList.of(), + ImmutableList.of(new UnboundAlias(new UnboundSlot("name"), "id")) + )), false, olapScan); PlanChecker.from(MemoTestUtils.createConnectContext()) .analyze(project) .matches( logicalProject( logicalOlapScan() - ).when(proj -> proj.getReplaces().size() == 1 - && proj.getProjects().get(0).getName().equals("id")) + ).when(proj -> proj.getProjects().get(0).getName().equals("id")) ); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanToStringTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanToStringTest.java index 38525eaf0a3f6f..312afd22b9269d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanToStringTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanToStringTest.java @@ -91,7 +91,7 @@ public void testLogicalOlapScan() { public void testLogicalProject(@Mocked Plan child) { LogicalProject plan = new LogicalProject<>(ImmutableList.of( new SlotReference(new ExprId(0), "a", BigIntType.INSTANCE, true, Lists.newArrayList())), child); - Assertions.assertTrue(plan.toString().matches("LogicalProject\\[\\d+\\] \\( distinct=false, projects=\\[a#\\d+], excepts=\\[], replaces=\\[] \\)")); + Assertions.assertTrue(plan.toString().matches("LogicalProject\\[\\d+\\] \\( distinct=false, projects=\\[a#\\d+] \\)")); } @Test diff --git a/regression-test/data/correctness/test_select_except.out b/regression-test/data/correctness/test_select_except.out new file mode 100644 index 00000000000000..cdec4aa6410d41 --- /dev/null +++ b/regression-test/data/correctness/test_select_except.out @@ -0,0 +1,9 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !except_agg -- +1 5 +2 4 + +-- !except_agg_ordinal -- +1 5 +2 4 + diff --git a/regression-test/suites/correctness/test_select_except.groovy b/regression-test/suites/correctness/test_select_except.groovy index 68eb3066312f06..de36dd1d048f07 100644 --- a/regression-test/suites/correctness/test_select_except.groovy +++ b/regression-test/suites/correctness/test_select_except.groovy @@ -63,6 +63,19 @@ suite("test_select_except") { """ exception "errCode" } + test { + sql """ + select * except (siteid, citycode, username, pv) from tbl_select_except""" + exception "errCode" + } + qt_except_agg """ + select * except (siteid, username, pv), sum(pv) + from tbl_select_except + group by citycode order by citycode""" + qt_except_agg_ordinal """ + select * except (siteid, username, pv), sum(pv) + from tbl_select_except + group by 1 order by citycode""" } finally { sql "drop table if exists tbl_select_except" } From 0a638e472ca59efc836a81cc8f6e7d691a3c93a2 Mon Sep 17 00:00:00 2001 From: zhangdong <493738387@qq.com> Date: Thu, 29 Aug 2024 11:06:05 +0800 Subject: [PATCH 07/60] [case](mtmv) fix test_base_mtmv failed (#39794) errCode = 2, detailMessage = Table[t_test_base_mtmv_user]'s state(SCHEMA_CHANGE) is not NORMAL. Do not allow doing ALTER ops --- regression-test/suites/mtmv_p0/test_base_mtmv.groovy | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/regression-test/suites/mtmv_p0/test_base_mtmv.groovy b/regression-test/suites/mtmv_p0/test_base_mtmv.groovy index 0015a0fa13061d..fce6ae523b57c9 100644 --- a/regression-test/suites/mtmv_p0/test_base_mtmv.groovy +++ b/regression-test/suites/mtmv_p0/test_base_mtmv.groovy @@ -71,12 +71,14 @@ suite("test_base_mtmv","mtmv") { sql """ alter table ${tableName} add COLUMN new_col INT AFTER username; """ + assertEquals("FINISHED", getAlterColumnFinalState("${tableName}")) order_qt_add_column "select Name,State,RefreshState from mv_infos('database'='${dbName}') where Name='${mvName}'" // rename column sql """ alter table ${tableName} rename COLUMN new_col new_col_1; """ + assertEquals("FINISHED", getAlterColumnFinalState("${tableName}")) order_qt_rename_column "select Name,State,RefreshState from mv_infos('database'='${dbName}') where Name='${mvName}'" sql """ REFRESH MATERIALIZED VIEW ${mvName} AUTO @@ -88,6 +90,7 @@ suite("test_base_mtmv","mtmv") { sql """ alter table ${tableName} modify COLUMN new_col_1 BIGINT; """ + assertEquals("FINISHED", getAlterColumnFinalState("${tableName}")) order_qt_modify_column "select Name,State,RefreshState from mv_infos('database'='${dbName}') where Name='${mvName}'" sql """ REFRESH MATERIALIZED VIEW ${mvName} AUTO @@ -99,6 +102,7 @@ suite("test_base_mtmv","mtmv") { sql """ alter table ${tableName} drop COLUMN new_col_1; """ + assertEquals("FINISHED", getAlterColumnFinalState("${tableName}")) order_qt_drop_column "select Name,State,RefreshState from mv_infos('database'='${dbName}') where Name='${mvName}'" sql """ REFRESH MATERIALIZED VIEW ${mvName} AUTO From 4e6f5b0e33c994d5e80e988720989c4f02d77eb6 Mon Sep 17 00:00:00 2001 From: zhangdong <493738387@qq.com> Date: Thu, 29 Aug 2024 11:06:50 +0800 Subject: [PATCH 08/60] [case](mtmv) MTMV upgrade downgrade case (#39789) --- .../test_upgrade_downgrade_mtmv.out | 7 +++ .../suites/mtmv_up_down_p0/load.groovy | 47 +++++++++++++++++++ .../test_upgrade_downgrade_mtmv.groovy | 38 +++++++++++++++ 3 files changed, 92 insertions(+) create mode 100644 regression-test/data/mtmv_up_down_p0/test_upgrade_downgrade_mtmv.out create mode 100644 regression-test/suites/mtmv_up_down_p0/load.groovy create mode 100644 regression-test/suites/mtmv_up_down_p0/test_upgrade_downgrade_mtmv.groovy diff --git a/regression-test/data/mtmv_up_down_p0/test_upgrade_downgrade_mtmv.out b/regression-test/data/mtmv_up_down_p0/test_upgrade_downgrade_mtmv.out new file mode 100644 index 00000000000000..3c49cb862ed670 --- /dev/null +++ b/regression-test/data/mtmv_up_down_p0/test_upgrade_downgrade_mtmv.out @@ -0,0 +1,7 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !refresh_init -- +1 1 1 name1 + +-- !mtmv_sync -- +true + diff --git a/regression-test/suites/mtmv_up_down_p0/load.groovy b/regression-test/suites/mtmv_up_down_p0/load.groovy new file mode 100644 index 00000000000000..45a309ae0a772d --- /dev/null +++ b/regression-test/suites/mtmv_up_down_p0/load.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. + +suite("test_upgrade_downgrade_prepare_mtmv","p0,mtmv,restart_fe,external,hive,external_docker,external_docker_hive") { + String enabled = context.config.otherConfigs.get("enableHiveTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("diable Hive test.") + return; + } + String suiteName = "mtmv_up_down" + String hivePrefix = "hive2"; + String hms_port = context.config.otherConfigs.get(hivePrefix + "HmsPort") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String catalog_name = "${hivePrefix}_${suiteName}_catalog" + String mvName = "${hivePrefix}_${suiteName}_mtmv" + + sql """drop catalog if exists ${catalog_name}""" + sql """create catalog if not exists ${catalog_name} properties ( + "type"="hms", + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}' + );""" + sql """drop materialized view if exists ${mvName};""" + + sql """ + create MATERIALIZED VIEW ${mvName} + REFRESH auto ON MANUAL + partition by(part1) + DISTRIBUTED BY RANDOM BUCKETS AUTO + PROPERTIES ('replication_num' = '1') + as select * from ${catalog_name}.multi_catalog.one_partition a inner join ${catalog_name}.multi_catalog.region b on a.id=b.r_regionkey; + """ + waitingMTMVTaskFinishedByMvName(mvName) +} diff --git a/regression-test/suites/mtmv_up_down_p0/test_upgrade_downgrade_mtmv.groovy b/regression-test/suites/mtmv_up_down_p0/test_upgrade_downgrade_mtmv.groovy new file mode 100644 index 00000000000000..a49bda7120217a --- /dev/null +++ b/regression-test/suites/mtmv_up_down_p0/test_upgrade_downgrade_mtmv.groovy @@ -0,0 +1,38 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_upgrade_downgrade_mtmv","p0,mtmv,restart_fe,external,hive,external_docker,external_docker_hive") { + String enabled = context.config.otherConfigs.get("enableHiveTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("diable Hive test.") + return; + } + String suiteName = "mtmv_up_down" + String dbName = context.config.getDbNameByFile(context.file) + String hivePrefix = "hive2"; + String catalog_name = "${hivePrefix}_${suiteName}_catalog" + String mvName = "${hivePrefix}_${suiteName}_mtmv" + // test data is normal + order_qt_refresh_init "SELECT * FROM ${mvName}" + // test is sync + order_qt_mtmv_sync "select SyncWithBaseTables from mv_infos('database'='${dbName}') where Name='${mvName}'" + sql """ + REFRESH MATERIALIZED VIEW ${mvName} complete + """ + // test can refresh success + waitingMTMVTaskFinishedByMvName(mvName) +} From 964c33d30eb52f738d8629011f90e750feddf820 Mon Sep 17 00:00:00 2001 From: lihangyu <15605149486@163.com> Date: Thu, 29 Aug 2024 11:10:48 +0800 Subject: [PATCH 09/60] [Fix](Variant) variant fallthrough with inverted index (#40069) When reading from segment, the schema type is variant, if we check type valid in `get_inverted_index`, the result should always return nullptr(since variant type it self does not support inverted index), but the actual storage could be `string` or etc.So we should ignore the type check and return the correct inverted index iterators introduced by #36163 --- be/src/olap/rowset/segment_v2/segment.cpp | 6 ++++-- be/src/olap/rowset/segment_v2/segment_iterator.cpp | 7 ++++++- regression-test/data/variant_github_events_p2/load.out | 10 ++++++++++ .../suites/variant_github_events_p2/load.groovy | 9 +++++++-- .../suites/variant_p0/with_index/load.groovy | 1 + .../suites/variant_p0/with_index/var_index.groovy | 2 ++ 6 files changed, 30 insertions(+), 5 deletions(-) diff --git a/be/src/olap/rowset/segment_v2/segment.cpp b/be/src/olap/rowset/segment_v2/segment.cpp index ac54c9252c95cb..64f58e546c22ba 100644 --- a/be/src/olap/rowset/segment_v2/segment.cpp +++ b/be/src/olap/rowset/segment_v2/segment.cpp @@ -203,7 +203,8 @@ Status Segment::new_iterator(SchemaSPtr schema, const StorageReadOptions& read_o ColumnReader* reader = nullptr; if (col.is_extracted_column()) { auto relative_path = col.path_info_ptr()->copy_pop_front(); - const auto* node = _sub_column_tree[col.unique_id()].find_exact(relative_path); + int32_t unique_id = col.unique_id() > 0 ? col.unique_id() : col.parent_unique_id(); + const auto* node = _sub_column_tree[unique_id].find_exact(relative_path); reader = node != nullptr ? node->data.reader.get() : nullptr; } else { reader = _column_readers.contains(col.unique_id()) @@ -775,8 +776,9 @@ ColumnReader* Segment::_get_column_reader(const TabletColumn& col) { // init column iterator by path info if (col.has_path_info() || col.is_variant_type()) { auto relative_path = col.path_info_ptr()->copy_pop_front(); + int32_t unique_id = col.unique_id() > 0 ? col.unique_id() : col.parent_unique_id(); const auto* node = col.has_path_info() - ? _sub_column_tree[col.unique_id()].find_exact(relative_path) + ? _sub_column_tree[unique_id].find_exact(relative_path) : nullptr; if (node != nullptr) { return node->data.reader.get(); diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp index d4ae09b1bb739a..c79621647414c7 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -1488,10 +1488,15 @@ Status SegmentIterator::_init_inverted_index_iterators() { } for (auto cid : _schema->column_ids()) { if (_inverted_index_iterators[cid] == nullptr) { + // Not check type valid, since we need to get inverted index for related variant type when reading the segment. + // If check type valid, we can not get inverted index for variant type, and result nullptr.The result for calling + // get_inverted_index with variant suffix should return corresponding inverted index meta. + bool check_inverted_index_by_type = false; // Use segment’s own index_meta, for compatibility with future indexing needs to default to lowercase. RETURN_IF_ERROR(_segment->new_inverted_index_iterator( _opts.tablet_schema->column(cid), - _segment->_tablet_schema->get_inverted_index(_opts.tablet_schema->column(cid)), + _segment->_tablet_schema->get_inverted_index(_opts.tablet_schema->column(cid), + check_inverted_index_by_type), _opts, &_inverted_index_iterators[cid])); } } diff --git a/regression-test/data/variant_github_events_p2/load.out b/regression-test/data/variant_github_events_p2/load.out index 8d5e3327e3c12e..4bee99a71a99ea 100644 --- a/regression-test/data/variant_github_events_p2/load.out +++ b/regression-test/data/variant_github_events_p2/load.out @@ -11,3 +11,13 @@ 5451 {"actor":{"avatar_url":"https://avatars.githubusercontent.com/u/3437916?","gravatar_id":"","id":3437916,"login":"misol","url":"https://api.github.com/users/misol"},"created_at":"2015-01-01T02:48:28Z","id":"2489433218","org":{"avatar_url":"https://avatars.githubusercontent.com/u/1429259?","gravatar_id":"","id":1429259,"login":"xpressengine","url":"https://api.github.com/orgs/xpressengine"},"payload":{"action":"created","comment":{"body":"Html5 도 같이 지원하는 업로더였으면 좋겠어요! 구글링 해보면 꽤 나와요 :)","created_at":"2015-01-01T02:48:27Z","html_url":"https://github.com/xpressengine/xe-core/issues/1086#issuecomment-68479093","id":68479093,"issue_url":"https://api.github.com/repos/xpressengine/xe-core/issues/1086","updated_at":"2015-01-01T02:48:27Z","url":"https://api.github.com/repos/xpressengine/xe-core/issues/comments/68479093","user":{"avatar_url":"https://avatars.githubusercontent.com/u/3437916?v=3","events_url":"https://api.github.com/users/misol/events{/privacy}","followers_url":"https://api.github.com/users/misol/followers","following_url":"https://api.github.com/users/misol/following{/other_user}","gists_url":"https://api.github.com/users/misol/gists{/gist_id}","gravatar_id":"","html_url":"https://github.com/misol","id":3437916,"login":"misol","organizations_url":"https://api.github.com/users/misol/orgs","received_events_url":"https://api.github.com/users/misol/received_events","repos_url":"https://api.github.com/users/misol/repos","site_admin":0,"starred_url":"https://api.github.com/users/misol/starred{/owner}{/repo}","subscriptions_url":"https://api.github.com/users/misol/subscriptions","type":"User","url":"https://api.github.com/users/misol"}},"issue":{"comments":4,"comments_url":"https://api.github.com/repos/xpressengine/xe-core/issues/1086/comments","created_at":"2014-12-12T11:48:03Z","events_url":"https://api.github.com/repos/xpressengine/xe-core/issues/1086/events","html_url":"https://github.com/xpressengine/xe-core/issues/1086","id":51797879,"labels_url":"https://api.github.com/repos/xpressengine/xe-core/issues/1086/labels{/name}","locked":0,"number":1086,"state":"open","title":"파일 업로더 교체","updated_at":"2015-01-01T02:48:27Z","url":"https://api.github.com/repos/xpressengine/xe-core/issues/1086","user":{"avatar_url":"https://avatars.githubusercontent.com/u/53764?v=3","events_url":"https://api.github.com/users/bnu/events{/privacy}","followers_url":"https://api.github.com/users/bnu/followers","following_url":"https://api.github.com/users/bnu/following{/other_user}","gists_url":"https://api.github.com/users/bnu/gists{/gist_id}","gravatar_id":"","html_url":"https://github.com/bnu","id":53764,"login":"bnu","organizations_url":"https://api.github.com/users/bnu/orgs","received_events_url":"https://api.github.com/users/bnu/received_events","repos_url":"https://api.github.com/users/bnu/repos","site_admin":0,"starred_url":"https://api.github.com/users/bnu/starred{/owner}{/repo}","subscriptions_url":"https://api.github.com/users/bnu/subscriptions","type":"User","url":"https://api.github.com/users/bnu"}}},"public":1,"repo":{"id":7953576,"name":"xpressengine/xe-core","url":"https://api.github.com/repos/xpressengine/xe-core"},"type":"IssueCommentEvent"} 5995 {"actor":{"avatar_url":"https://avatars.githubusercontent.com/u/3437916?","gravatar_id":"","id":3437916,"login":"misol","url":"https://api.github.com/users/misol"},"created_at":"2015-01-01T01:47:44Z","id":"2489414108","org":{"avatar_url":"https://avatars.githubusercontent.com/u/1429259?","gravatar_id":"","id":1429259,"login":"xpressengine","url":"https://api.github.com/orgs/xpressengine"},"payload":{"action":"opened","number":1120,"pull_request":{"_links":{"comments":{"href":"https://api.github.com/repos/xpressengine/xe-core/issues/1120/comments"},"commits":{"href":"https://api.github.com/repos/xpressengine/xe-core/pulls/1120/commits"},"html":{"href":"https://github.com/xpressengine/xe-core/pull/1120"},"issue":{"href":"https://api.github.com/repos/xpressengine/xe-core/issues/1120"},"review_comment":{"href":"https://api.github.com/repos/xpressengine/xe-core/pulls/comments/{number}"},"review_comments":{"href":"https://api.github.com/repos/xpressengine/xe-core/pulls/1120/comments"},"self":{"href":"https://api.github.com/repos/xpressengine/xe-core/pulls/1120"},"statuses":{"href":"https://api.github.com/repos/xpressengine/xe-core/statuses/d2b05732abfd85020335ce272abd37c0ad1c6654"}},"additions":4748,"base":{"label":"xpressengine:develop","ref":"develop","repo":{"archive_url":"https://api.github.com/repos/xpressengine/xe-core/{archive_format}{/ref}","assignees_url":"https://api.github.com/repos/xpressengine/xe-core/assignees{/user}","blobs_url":"https://api.github.com/repos/xpressengine/xe-core/git/blobs{/sha}","branches_url":"https://api.github.com/repos/xpressengine/xe-core/branches{/branch}","clone_url":"https://github.com/xpressengine/xe-core.git","collaborators_url":"https://api.github.com/repos/xpressengine/xe-core/collaborators{/collaborator}","comments_url":"https://api.github.com/repos/xpressengine/xe-core/comments{/number}","commits_url":"https://api.github.com/repos/xpressengine/xe-core/commits{/sha}","compare_url":"https://api.github.com/repos/xpressengine/xe-core/compare/{base}...{head}","contents_url":"https://api.github.com/repos/xpressengine/xe-core/contents/{+path}","contributors_url":"https://api.github.com/repos/xpressengine/xe-core/contributors","created_at":"2013-02-01T07:16:05Z","default_branch":"master","description":"PHP Open Source CMS","downloads_url":"https://api.github.com/repos/xpressengine/xe-core/downloads","events_url":"https://api.github.com/repos/xpressengine/xe-core/events","fork":0,"forks":143,"forks_count":143,"forks_url":"https://api.github.com/repos/xpressengine/xe-core/forks","full_name":"xpressengine/xe-core","git_commits_url":"https://api.github.com/repos/xpressengine/xe-core/git/commits{/sha}","git_refs_url":"https://api.github.com/repos/xpressengine/xe-core/git/refs{/sha}","git_tags_url":"https://api.github.com/repos/xpressengine/xe-core/git/tags{/sha}","git_url":"git://github.com/xpressengine/xe-core.git","has_downloads":1,"has_issues":1,"has_pages":0,"has_wiki":1,"homepage":"http://www.xpressengine.com","hooks_url":"https://api.github.com/repos/xpressengine/xe-core/hooks","html_url":"https://github.com/xpressengine/xe-core","id":7953576,"issue_comment_url":"https://api.github.com/repos/xpressengine/xe-core/issues/comments/{number}","issue_events_url":"https://api.github.com/repos/xpressengine/xe-core/issues/events{/number}","issues_url":"https://api.github.com/repos/xpressengine/xe-core/issues{/number}","keys_url":"https://api.github.com/repos/xpressengine/xe-core/keys{/key_id}","labels_url":"https://api.github.com/repos/xpressengine/xe-core/labels{/name}","language":"PHP","languages_url":"https://api.github.com/repos/xpressengine/xe-core/languages","merges_url":"https://api.github.com/repos/xpressengine/xe-core/merges","milestones_url":"https://api.github.com/repos/xpressengine/xe-core/milestones{/number}","name":"xe-core","notifications_url":"https://api.github.com/repos/xpressengine/xe-core/notifications{?since,all,participating}","open_issues":156,"open_issues_count":156,"owner":{"avatar_url":"https://avatars.githubusercontent.com/u/1429259?v=3","events_url":"https://api.github.com/users/xpressengine/events{/privacy}","followers_url":"https://api.github.com/users/xpressengine/followers","following_url":"https://api.github.com/users/xpressengine/following{/other_user}","gists_url":"https://api.github.com/users/xpressengine/gists{/gist_id}","gravatar_id":"","html_url":"https://github.com/xpressengine","id":1429259,"login":"xpressengine","organizations_url":"https://api.github.com/users/xpressengine/orgs","received_events_url":"https://api.github.com/users/xpressengine/received_events","repos_url":"https://api.github.com/users/xpressengine/repos","site_admin":0,"starred_url":"https://api.github.com/users/xpressengine/starred{/owner}{/repo}","subscriptions_url":"https://api.github.com/users/xpressengine/subscriptions","type":"Organization","url":"https://api.github.com/users/xpressengine"},"private":0,"pulls_url":"https://api.github.com/repos/xpressengine/xe-core/pulls{/number}","pushed_at":"2014-12-31T08:52:46Z","releases_url":"https://api.github.com/repos/xpressengine/xe-core/releases{/id}","size":90250,"ssh_url":"git@github.com:xpressengine/xe-core.git","stargazers_count":149,"stargazers_url":"https://api.github.com/repos/xpressengine/xe-core/stargazers","statuses_url":"https://api.github.com/repos/xpressengine/xe-core/statuses/{sha}","subscribers_url":"https://api.github.com/repos/xpressengine/xe-core/subscribers","subscription_url":"https://api.github.com/repos/xpressengine/xe-core/subscription","svn_url":"https://github.com/xpressengine/xe-core","tags_url":"https://api.github.com/repos/xpressengine/xe-core/tags","teams_url":"https://api.github.com/repos/xpressengine/xe-core/teams","trees_url":"https://api.github.com/repos/xpressengine/xe-core/git/trees{/sha}","updated_at":"2014-12-30T00:05:52Z","url":"https://api.github.com/repos/xpressengine/xe-core","watchers":149,"watchers_count":149},"sha":"c3430d1c724f42154ca5dd648637c4df796d1708","user":{"avatar_url":"https://avatars.githubusercontent.com/u/1429259?v=3","events_url":"https://api.github.com/users/xpressengine/events{/privacy}","followers_url":"https://api.github.com/users/xpressengine/followers","following_url":"https://api.github.com/users/xpressengine/following{/other_user}","gists_url":"https://api.github.com/users/xpressengine/gists{/gist_id}","gravatar_id":"","html_url":"https://github.com/xpressengine","id":1429259,"login":"xpressengine","organizations_url":"https://api.github.com/users/xpressengine/orgs","received_events_url":"https://api.github.com/users/xpressengine/received_events","repos_url":"https://api.github.com/users/xpressengine/repos","site_admin":0,"starred_url":"https://api.github.com/users/xpressengine/starred{/owner}{/repo}","subscriptions_url":"https://api.github.com/users/xpressengine/subscriptions","type":"Organization","url":"https://api.github.com/users/xpressengine"}},"body":"https://github.com/xpressengine/xe-core/issues/634\\r\\n\\r\\n# Internet Explorer send punycode URL(ASCII) URL and non-alphabet\\r\\nUnicode URL URL as a referer. 인터넷 익스플로러는 리퍼러 주소로 퓨니코드 주소와 유니코드 URL을 섞어\\r\\n쓰고 있습니다. AJAX 통신에는 리퍼러로 Unicode를 사용하고 요청 호스트로는 퓨니코드 URL을 사용(이건 다국어 주소\\r\\n형식으로 접속하려면 이렇게 했어야 할 것)합니다.\\r\\n- XE strictly compare referer and server host for denying CSRF, but\\r\\npunycode URL and Unicode URL should be dealt as a same one. 그런데 XE는 리퍼러의\\r\\n호스트와 서버 호스트를 비교합니다. punycode로 쓰인 주소와 Unicode로 쓰인 주소는 같은 주소를 지시하더라도 문자열이\\r\\n다릅니다. 같은 주소를 지칭하는 다른 문자열을 punycode로 변환해서 같은 주소라고 인식할 수 있게 수정했습니다.\\r\\n- Fix checkCSRF function to deal both form as a same one.\\r\\n- Convert Unicode URL input to punycode URL on the Admin Default URL\\r\\nSettings. 관리자가 유니코드 형식으로 기본 주소를 입력하더라도, 퓨니코드로 변환해 저장하도록 했습니다. 퓨니코드로 저장하는\\r\\n것이 여러모로 유용하기 때문입니다.\\r\\n- For converting punycode URL, include IDNA coverting class. 퓨니코드와 유니코드\\r\\n간 변환을 위해서 IDNA 변환 클래스(LGPL사용권)를 포함시켰습니다.\\r\\n\\r\\n**이 수정을 하면 *한글 도메인에서 글 작성이 가능*합니다. 하지만, *파일 업로드의 경우는 SWF Uploader 이슈로 파일 업로드가 불가능*합니다. 이 문제는, HTML5를 지원하는 파일 업로더를 이용하면 해결됩니다. (워드 프레스 등의 해결법) HTML5를 지원하는 파일 업로더는 AXISJ 업로더 등을 포함해서 XE 공식 홈페이지 자료실에서 다운받아 사용할 수 있습니다.(에디터 변경)**","changed_files":8,"comments":0,"comments_url":"https://api.github.com/repos/xpressengine/xe-core/issues/1120/comments","commits":1,"commits_url":"https://api.github.com/repos/xpressengine/xe-core/pulls/1120/commits","created_at":"2015-01-01T01:47:43Z","deletions":1,"diff_url":"https://github.com/xpressengine/xe-core/pull/1120.diff","head":{"label":"misol:support_punycode_domain","ref":"support_punycode_domain","repo":{"archive_url":"https://api.github.com/repos/misol/xe-core/{archive_format}{/ref}","assignees_url":"https://api.github.com/repos/misol/xe-core/assignees{/user}","blobs_url":"https://api.github.com/repos/misol/xe-core/git/blobs{/sha}","branches_url":"https://api.github.com/repos/misol/xe-core/branches{/branch}","clone_url":"https://github.com/misol/xe-core.git","collaborators_url":"https://api.github.com/repos/misol/xe-core/collaborators{/collaborator}","comments_url":"https://api.github.com/repos/misol/xe-core/comments{/number}","commits_url":"https://api.github.com/repos/misol/xe-core/commits{/sha}","compare_url":"https://api.github.com/repos/misol/xe-core/compare/{base}...{head}","contents_url":"https://api.github.com/repos/misol/xe-core/contents/{+path}","contributors_url":"https://api.github.com/repos/misol/xe-core/contributors","created_at":"2014-12-31T14:41:05Z","default_branch":"master","description":"PHP Open Source CMS","downloads_url":"https://api.github.com/repos/misol/xe-core/downloads","events_url":"https://api.github.com/repos/misol/xe-core/events","fork":1,"forks":0,"forks_count":0,"forks_url":"https://api.github.com/repos/misol/xe-core/forks","full_name":"misol/xe-core","git_commits_url":"https://api.github.com/repos/misol/xe-core/git/commits{/sha}","git_refs_url":"https://api.github.com/repos/misol/xe-core/git/refs{/sha}","git_tags_url":"https://api.github.com/repos/misol/xe-core/git/tags{/sha}","git_url":"git://github.com/misol/xe-core.git","has_downloads":1,"has_issues":0,"has_pages":0,"has_wiki":1,"homepage":"http://www.xpressengine.com","hooks_url":"https://api.github.com/repos/misol/xe-core/hooks","html_url":"https://github.com/misol/xe-core","id":28667946,"issue_comment_url":"https://api.github.com/repos/misol/xe-core/issues/comments/{number}","issue_events_url":"https://api.github.com/repos/misol/xe-core/issues/events{/number}","issues_url":"https://api.github.com/repos/misol/xe-core/issues{/number}","keys_url":"https://api.github.com/repos/misol/xe-core/keys{/key_id}","labels_url":"https://api.github.com/repos/misol/xe-core/labels{/name}","language":"PHP","languages_url":"https://api.github.com/repos/misol/xe-core/languages","merges_url":"https://api.github.com/repos/misol/xe-core/merges","milestones_url":"https://api.github.com/repos/misol/xe-core/milestones{/number}","name":"xe-core","notifications_url":"https://api.github.com/repos/misol/xe-core/notifications{?since,all,participating}","open_issues":0,"open_issues_count":0,"owner":{"avatar_url":"https://avatars.githubusercontent.com/u/3437916?v=3","events_url":"https://api.github.com/users/misol/events{/privacy}","followers_url":"https://api.github.com/users/misol/followers","following_url":"https://api.github.com/users/misol/following{/other_user}","gists_url":"https://api.github.com/users/misol/gists{/gist_id}","gravatar_id":"","html_url":"https://github.com/misol","id":3437916,"login":"misol","organizations_url":"https://api.github.com/users/misol/orgs","received_events_url":"https://api.github.com/users/misol/received_events","repos_url":"https://api.github.com/users/misol/repos","site_admin":0,"starred_url":"https://api.github.com/users/misol/starred{/owner}{/repo}","subscriptions_url":"https://api.github.com/users/misol/subscriptions","type":"User","url":"https://api.github.com/users/misol"},"private":0,"pulls_url":"https://api.github.com/repos/misol/xe-core/pulls{/number}","pushed_at":"2015-01-01T01:36:28Z","releases_url":"https://api.github.com/repos/misol/xe-core/releases{/id}","size":90250,"ssh_url":"git@github.com:misol/xe-core.git","stargazers_count":0,"stargazers_url":"https://api.github.com/repos/misol/xe-core/stargazers","statuses_url":"https://api.github.com/repos/misol/xe-core/statuses/{sha}","subscribers_url":"https://api.github.com/repos/misol/xe-core/subscribers","subscription_url":"https://api.github.com/repos/misol/xe-core/subscription","svn_url":"https://github.com/misol/xe-core","tags_url":"https://api.github.com/repos/misol/xe-core/tags","teams_url":"https://api.github.com/repos/misol/xe-core/teams","trees_url":"https://api.github.com/repos/misol/xe-core/git/trees{/sha}","updated_at":"2014-12-31T14:41:10Z","url":"https://api.github.com/repos/misol/xe-core","watchers":0,"watchers_count":0},"sha":"d2b05732abfd85020335ce272abd37c0ad1c6654","user":{"avatar_url":"https://avatars.githubusercontent.com/u/3437916?v=3","events_url":"https://api.github.com/users/misol/events{/privacy}","followers_url":"https://api.github.com/users/misol/followers","following_url":"https://api.github.com/users/misol/following{/other_user}","gists_url":"https://api.github.com/users/misol/gists{/gist_id}","gravatar_id":"","html_url":"https://github.com/misol","id":3437916,"login":"misol","organizations_url":"https://api.github.com/users/misol/orgs","received_events_url":"https://api.github.com/users/misol/received_events","repos_url":"https://api.github.com/users/misol/repos","site_admin":0,"starred_url":"https://api.github.com/users/misol/starred{/owner}{/repo}","subscriptions_url":"https://api.github.com/users/misol/subscriptions","type":"User","url":"https://api.github.com/users/misol"}},"html_url":"https://github.com/xpressengine/xe-core/pull/1120","id":26739793,"issue_url":"https://api.github.com/repos/xpressengine/xe-core/issues/1120","locked":0,"mergeable_state":"unknown","merged":0,"number":1120,"patch_url":"https://github.com/xpressengine/xe-core/pull/1120.patch","review_comment_url":"https://api.github.com/repos/xpressengine/xe-core/pulls/comments/{number}","review_comments":0,"review_comments_url":"https://api.github.com/repos/xpressengine/xe-core/pulls/1120/comments","state":"open","statuses_url":"https://api.github.com/repos/xpressengine/xe-core/statuses/d2b05732abfd85020335ce272abd37c0ad1c6654","title":"fix for Not-Alphabet URL document writing (#634)","updated_at":"2015-01-01T01:47:43Z","url":"https://api.github.com/repos/xpressengine/xe-core/pulls/1120","user":{"avatar_url":"https://avatars.githubusercontent.com/u/3437916?v=3","events_url":"https://api.github.com/users/misol/events{/privacy}","followers_url":"https://api.github.com/users/misol/followers","following_url":"https://api.github.com/users/misol/following{/other_user}","gists_url":"https://api.github.com/users/misol/gists{/gist_id}","gravatar_id":"","html_url":"https://github.com/misol","id":3437916,"login":"misol","organizations_url":"https://api.github.com/users/misol/orgs","received_events_url":"https://api.github.com/users/misol/received_events","repos_url":"https://api.github.com/users/misol/repos","site_admin":0,"starred_url":"https://api.github.com/users/misol/starred{/owner}{/repo}","subscriptions_url":"https://api.github.com/users/misol/subscriptions","type":"User","url":"https://api.github.com/users/misol"}}},"public":1,"repo":{"id":7953576,"name":"xpressengine/xe-core","url":"https://api.github.com/repos/xpressengine/xe-core"},"type":"PullRequestEvent"} +-- !sql -- +\N +\N +\N +\N +4748 + +-- !sql -- +135 + diff --git a/regression-test/suites/variant_github_events_p2/load.groovy b/regression-test/suites/variant_github_events_p2/load.groovy index 4e81dc2237c380..e1742231afc9ab 100644 --- a/regression-test/suites/variant_github_events_p2/load.groovy +++ b/regression-test/suites/variant_github_events_p2/load.groovy @@ -169,7 +169,7 @@ suite("regression_test_variant_github_events_p2", "nonConcurrent,p2"){ // // build inverted index at middle of loading the data // ADD INDEX - sql """ ALTER TABLE github_events ADD INDEX idx_var (`v`) USING INVERTED PROPERTIES("parser" = "chinese", "parser_mode" = "fine_grained", "support_phrase" = "true") """ + sql """ ALTER TABLE github_events ADD INDEX idx_var (`v`) USING INVERTED PROPERTIES("parser" = "english", "support_phrase" = "true") """ wait_for_latest_op_on_table_finish("github_events", timeout) // 2022 @@ -214,7 +214,8 @@ suite("regression_test_variant_github_events_p2", "nonConcurrent,p2"){ } while (running) } - + sql """set enable_match_without_inverted_index = false""" + // filter by bloom filter 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""" sql """select * from github_events order by k limit 10""" @@ -230,4 +231,8 @@ suite("regression_test_variant_github_events_p2", "nonConcurrent,p2"){ sql """insert into github_events2 select * from github_events order by k""" sql """select v['payload']['commits'] from github_events order by k ;""" sql """select v['payload']['commits'] from github_events2 order by k ;""" + + // query with inverted index + qt_sql """select cast(v["payload"]["pull_request"]["additions"] as int) from github_events where v["repo"]["name"] match 'xpressengine' order by 1;""" + qt_sql """select count() from github_events where v["repo"]["name"] match 'apache' order by 1;""" } \ No newline at end of file diff --git a/regression-test/suites/variant_p0/with_index/load.groovy b/regression-test/suites/variant_p0/with_index/load.groovy index 2ff781a2008ab0..93737e8a5b9dc8 100644 --- a/regression-test/suites/variant_p0/with_index/load.groovy +++ b/regression-test/suites/variant_p0/with_index/load.groovy @@ -61,6 +61,7 @@ suite("regression_test_variant_with_index", "nonConcurrent"){ properties("replication_num" = "1", "disable_auto_compaction" = "true"); """ sql """insert into var_with_index values(1, '{"a" : 0, "b": 3}', 'hello world'), (2, '{"a" : 123}', 'world'),(3, '{"a" : 123}', 'hello world')""" + sql """set enable_match_without_inverted_index = false""" qt_sql_inv_1 """select v["a"] from var_with_index where inv match 'hello' order by k""" qt_sql_inv_2 """select v["a"] from var_with_index where inv match 'hello' and cast(v['a'] as int) > 0 order by k""" qt_sql_inv_3 """select * from var_with_index where inv match 'hello' and cast(v["a"] as int) > 0 order by k""" diff --git a/regression-test/suites/variant_p0/with_index/var_index.groovy b/regression-test/suites/variant_p0/with_index/var_index.groovy index 5f061f3208bab8..a21d242954504a 100644 --- a/regression-test/suites/variant_p0/with_index/var_index.groovy +++ b/regression-test/suites/variant_p0/with_index/var_index.groovy @@ -33,7 +33,9 @@ suite("regression_test_variant_var_index", "p0"){ sql """insert into var_index values(2, '{"a" : 18811, "b" : "hello world", "c" : 1181111}')""" sql """insert into var_index values(3, '{"a" : 18811, "b" : "hello wworld", "c" : 11111}')""" sql """insert into var_index values(4, '{"a" : 1234, "b" : "hello xxx world", "c" : 8181111}')""" + sql """set enable_match_without_inverted_index = false""" qt_sql """select * from var_index where cast(v["a"] as smallint) > 123 and cast(v["b"] as string) match 'hello' and cast(v["c"] as int) > 1024 order by k""" + sql """set enable_match_without_inverted_index = true""" sql """insert into var_index values(5, '{"a" : 123456789, "b" : 123456, "c" : 8181111}')""" qt_sql """select * from var_index where cast(v["a"] as int) > 123 and cast(v["b"] as string) match 'hello' and cast(v["c"] as int) > 11111 order by k""" // insert double/float/array/json From df079db57e495dc5f9b0c1ecd8f21818579d8358 Mon Sep 17 00:00:00 2001 From: Dongyang Li Date: Thu, 29 Aug 2024 11:15:02 +0800 Subject: [PATCH 10/60] [chore](ci) remove ci pipeline cloud_p1 (#40087) It is enough to run p1 daily. --- .asf.yaml | 3 --- 1 file changed, 3 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index 821947fa1a0451..e71e55de23fc83 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -52,9 +52,7 @@ github: - Clang Formatter - CheckStyle - P0 Regression (Doris Regression) - - P1 Regression (Doris Regression) - External Regression (Doris External Regression) - - cloud_p1 (Doris Cloud Regression) - cloud_p0 (Doris Cloud Regression) - FE UT (Doris FE UT) - BE UT (Doris BE UT) @@ -114,7 +112,6 @@ github: - Clang Formatter - CheckStyle - P0 Regression (Doris Regression) - - P1 Regression (Doris Regression) - External Regression (Doris External Regression) - FE UT (Doris FE UT) - BE UT (Doris BE UT) From 9dbd4d951839d53c0663a1028d776246c2f7ab2a Mon Sep 17 00:00:00 2001 From: xzj7019 <131111794+xzj7019@users.noreply.github.com> Date: Thu, 29 Aug 2024 11:26:51 +0800 Subject: [PATCH 11/60] [fix](coordinator) fix union all instance number (#39999) In following plan pattern, fragment 11 has 3 child fragments aligning with 3 exchange nodes and the union plan node only have 2 plan children, i.e, the aggr node and the exchange node(4). In current union node's parallelism computing, it used the plan children's size to do iteration to find the max parallel but ignore the last fragment's. Actually, it should use fragment's children(exclude unpartitioned ones) instead of plan node's children to do the iteration. 11:VUNION(703) | |----10:VAGGREGATE (update finalize)(697) | | group by: brand_name[#23], skc[#24] | | sortByGroupKey:false | | cardinality=1 | | final projections: '1', brand_name[#25], skc[#26] | | final project output tuple id: 11 | | distribute expr lists: skc[#24] | | | 9:VHASH JOIN(691) | | join op: INNER JOIN(PARTITIONED)[] | | equal join conjunct: (skc[#19] = skc_code[#12]) | | cardinality=1 | | vec output tuple id: 9 | | output tuple id: 9 | | vIntermediate tuple ids: 8 | | hash output slot ids: 18 19 | | final projections: brand_name[#20], skc[#21] | | final project output tuple id: 9 | | distribute expr lists: skc[#19] | | distribute expr lists: skc_code[#12] | | | |----6:VEXCHANGE | | offset: 0 | | distribute expr lists: skc_code[#12] | | | 8:VEXCHANGE | offset: 0 | distribute expr lists: | 4:VEXCHANGE offset: 0 distribute expr lists: --- .../java/org/apache/doris/qe/Coordinator.java | 43 ++++----- .../query_p0/union/test_union_instance.groovy | 88 +++++++++++++++++++ 2 files changed, 111 insertions(+), 20 deletions(-) create mode 100644 regression-test/suites/query_p0/union/test_union_instance.groovy 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 018602fafd3948..0cbe3eebbab9f1 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 @@ -1788,7 +1788,6 @@ protected void computeFragmentHosts() throws Exception { } Pair pairNodes = findLeftmostNode(fragment.getPlanRoot()); - PlanNode fatherNode = pairNodes.first; PlanNode leftMostNode = pairNodes.second; /* @@ -1803,25 +1802,8 @@ protected void computeFragmentHosts() throws Exception { // (Case B) // there is no leftmost scan; we assign the same hosts as those of our // input fragment which has a higher instance_number - - int inputFragmentIndex = 0; - int maxParallelism = 0; - // If the fragment has three children, then the first child and the second child are - // the children(both exchange node) of shuffle HashJoinNode, - // and the third child is the right child(ExchangeNode) of broadcast HashJoinNode. - // We only need to pay attention to the maximum parallelism among - // the two ExchangeNodes of shuffle HashJoinNode. - int childrenCount = (fatherNode != null) ? fatherNode.getChildren().size() : 1; - for (int j = 0; j < childrenCount; j++) { - int currentChildFragmentParallelism - = fragmentExecParamsMap.get(fragment.getChild(j).getFragmentId()).instanceExecParams.size(); - if (currentChildFragmentParallelism > maxParallelism) { - maxParallelism = currentChildFragmentParallelism; - inputFragmentIndex = j; - } - } - - PlanFragmentId inputFragmentId = fragment.getChild(inputFragmentIndex).getFragmentId(); + int maxParallelFragmentIndex = findMaxParallelFragmentIndex(fragment); + PlanFragmentId inputFragmentId = fragment.getChild(maxParallelFragmentIndex).getFragmentId(); // AddAll() soft copy() int exchangeInstances = -1; if (ConnectContext.get() != null && ConnectContext.get().getSessionVariable() != null) { @@ -1978,6 +1960,27 @@ protected void computeFragmentHosts() throws Exception { } } + private int findMaxParallelFragmentIndex(PlanFragment fragment) { + Preconditions.checkState(!fragment.getChildren().isEmpty(), "fragment has no children"); + + // exclude broadcast join right side's child fragments + List childFragmentCandidates = fragment.getChildren().stream() + .filter(e -> e.getOutputPartition() != DataPartition.UNPARTITIONED) + .collect(Collectors.toList()); + + int maxParallelism = 0; + int maxParaIndex = 0; + for (int i = 0; i < childFragmentCandidates.size(); i++) { + PlanFragmentId childFragmentId = childFragmentCandidates.get(i).getFragmentId(); + int currentChildFragmentParallelism = fragmentExecParamsMap.get(childFragmentId).instanceExecParams.size(); + if (currentChildFragmentParallelism > maxParallelism) { + maxParallelism = currentChildFragmentParallelism; + maxParaIndex = i; + } + } + return maxParaIndex; + } + private TNetworkAddress getGroupCommitBackend(Map addressToBackendID) { // Used for Nereids planner Group commit insert BE select. TNetworkAddress execHostport = new TNetworkAddress(groupCommitBackend.getHost(), diff --git a/regression-test/suites/query_p0/union/test_union_instance.groovy b/regression-test/suites/query_p0/union/test_union_instance.groovy new file mode 100644 index 00000000000000..17a0d3ef1dd614 --- /dev/null +++ b/regression-test/suites/query_p0/union/test_union_instance.groovy @@ -0,0 +1,88 @@ +// 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_union_instance") { + multi_sql """ + drop table if exists t1; + drop table if exists t2; + drop table if exists t3; + drop table if exists t4; + CREATE TABLE `t1` ( + `c1` int NULL, + `c2` int NULL, + `c3` int NULL + ) ENGINE=OLAP + DUPLICATE KEY(`c1`, `c2`, `c3`) + DISTRIBUTED BY HASH(`c1`) BUCKETS 3 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1"); + + insert into t1 values (1,1,1); + insert into t1 values (2,2,2); + insert into t1 values (3,3,3); + + CREATE TABLE `t2` ( + `year_week` varchar(45) NULL + ) ENGINE=OLAP + DUPLICATE KEY(`year_week`) + DISTRIBUTED BY HASH(`year_week`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1"); + + CREATE TABLE `t3` ( + `unique_key` varchar(2999) NULL, + `brand_name` varchar(96) NULL, + `skc` varchar(150) NULL + ) ENGINE=OLAP + UNIQUE KEY(`unique_key`) + DISTRIBUTED BY HASH(`unique_key`) BUCKETS 20 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1"); + + CREATE TABLE `t4` ( + `skc_code` varchar(150) NULL + ) ENGINE=OLAP + UNIQUE KEY(`skc_code`) + DISTRIBUTED BY HASH(`skc_code`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1"); + + set parallel_pipeline_task_num=1; + set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + """ + explain { + sql """ + SELECT `t`.`year_week` AS `year_week` + ,'' AS `brand_name` + , '' AS `skc_code` + FROM `t1` a + CROSS JOIN `t2` t + union all + SELECT '1' AS `year_week` + ,`a`.`brand_name` AS `brand_name` + ,`a`.`skc` AS `skc_code` + FROM `t3` a + INNER JOIN[shuffle] `t4` b ON `a`.`skc` = `b`.`skc_code` + GROUP BY 1, 2, 3; + """ + contains "3:VNESTED LOOP JOIN" + contains "4:VEXCHANGE" + contains "8:VEXCHANGE" + contains "6:VEXCHANGE" + contains "11:VUNION" + } +} From 664ca8ad780164963ea870e160dda6396c63464a Mon Sep 17 00:00:00 2001 From: Sun Chenyang Date: Thu, 29 Aug 2024 11:38:54 +0800 Subject: [PATCH 12/60] [fix](Nereids) retrieve the table using SlotReference in the Match (#39652) ## Proposed changes Sometimes, there is no table in SlotRef, so we need to use SlotReference to get the table. --- .../glue/translator/ExpressionTranslator.java | 41 +++-- .../inverted_index_p0/test_match_and_join.out | 4 + .../test_match_and_join.groovy | 146 ++++++++++++++++++ 3 files changed, 170 insertions(+), 21 deletions(-) create mode 100644 regression-test/data/inverted_index_p0/test_match_and_join.out create mode 100644 regression-test/suites/inverted_index_p0/test_match_and_join.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java index 47d11d56c7c9ec..e876b30280b4a2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java @@ -37,11 +37,10 @@ import org.apache.doris.analysis.LambdaFunctionExpr; import org.apache.doris.analysis.MatchPredicate; import org.apache.doris.analysis.OrderByElement; -import org.apache.doris.analysis.SlotDescriptor; import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.TimestampArithmeticExpr; -import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.catalog.ArrayType; +import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Function; import org.apache.doris.catalog.Function.NullableMode; import org.apache.doris.catalog.Index; @@ -189,19 +188,11 @@ public Expr visitLessThanEqual(LessThanEqual lessThanEqual, PlanTranslatorContex return le; } - private OlapTable getOlapTableFromSlotDesc(SlotDescriptor slotDesc) { - if (slotDesc != null && slotDesc.isScanSlot()) { - TupleDescriptor slotParent = slotDesc.getParent(); - return (OlapTable) slotParent.getTable(); - } - return null; - } - - private OlapTable getOlapTableDirectly(SlotRef left) { - if (left.getTableDirect() instanceof OlapTable) { - return (OlapTable) left.getTableDirect(); - } - return null; + private OlapTable getOlapTableDirectly(SlotReference slot) { + return slot.getTable() + .filter(OlapTable.class::isInstance) + .map(OlapTable.class::cast) + .orElse(null); } @Override @@ -213,12 +204,20 @@ public Expr visitElementAt(ElementAt elementAt, PlanTranslatorContext context) { public Expr visitMatch(Match match, PlanTranslatorContext context) { Index invertedIndex = null; // Get the first slot from match's left expr - SlotRef left = (SlotRef) match.left().getInputSlots().stream().findFirst().get().accept(this, context); - OlapTable olapTbl = Optional.ofNullable(getOlapTableFromSlotDesc(left.getDesc())) - .orElse(getOlapTableDirectly(left)); - + SlotReference slot = match.getInputSlots().stream() + .findFirst() + .filter(SlotReference.class::isInstance) + .map(SlotReference.class::cast) + .orElseThrow(() -> new AnalysisException( + "No SlotReference found in Match, SQL is " + match.toSql())); + + Column column = slot.getColumn() + .orElseThrow(() -> new AnalysisException( + "SlotReference in Match failed to get Column, SQL is " + match.toSql())); + + OlapTable olapTbl = getOlapTableDirectly(slot); if (olapTbl == null) { - throw new AnalysisException("slotRef in matchExpression failed to get OlapTable"); + throw new AnalysisException("SlotReference in Match failed to get OlapTable, SQL is " + match.toSql()); } List indexes = olapTbl.getIndexes(); @@ -226,7 +225,7 @@ public Expr visitMatch(Match match, PlanTranslatorContext context) { for (Index index : indexes) { if (index.getIndexType() == IndexDef.IndexType.INVERTED) { List columns = index.getColumns(); - if (columns != null && !columns.isEmpty() && left.getColumnName().equals(columns.get(0))) { + if (columns != null && !columns.isEmpty() && column.getName().equals(columns.get(0))) { invertedIndex = index; break; } diff --git a/regression-test/data/inverted_index_p0/test_match_and_join.out b/regression-test/data/inverted_index_p0/test_match_and_join.out new file mode 100644 index 00000000000000..d05128e455a86c --- /dev/null +++ b/regression-test/data/inverted_index_p0/test_match_and_join.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +810 + diff --git a/regression-test/suites/inverted_index_p0/test_match_and_join.groovy b/regression-test/suites/inverted_index_p0/test_match_and_join.groovy new file mode 100644 index 00000000000000..f4d0bcf38877cf --- /dev/null +++ b/regression-test/suites/inverted_index_p0/test_match_and_join.groovy @@ -0,0 +1,146 @@ +// 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_match_and_join") { + + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + + sql """ drop table if exists table_30_undef_partitions2_keys3_properties4_distributed_by55 """ + sql """ + create table table_30_undef_partitions2_keys3_properties4_distributed_by55 ( + pk int, + col_int_undef_signed_index_inverted int null , + col_int_undef_signed int null , + col_int_undef_signed_not_null int not null , + col_int_undef_signed_not_null_index_inverted int not null , + col_date_undef_signed date null , + col_date_undef_signed_index_inverted date null , + col_date_undef_signed_not_null date not null , + col_date_undef_signed_not_null_index_inverted date not null , + col_varchar_1024__undef_signed varchar(1024) null , + col_varchar_1024__undef_signed_index_inverted varchar(1024) null , + col_varchar_1024__undef_signed_index_inverted_parser_english_support_phrase_true varchar(1024) null , + col_varchar_1024__undef_signed_index_inverted_parser_english_support_phrase_false varchar(1024) null , + col_varchar_1024__undef_signed_index_inverted_parser_unicode_support_phrase_true varchar(1024) null , + col_varchar_1024__undef_signed_index_inverted_parser_unicode_support_phrase_false varchar(1024) null , + col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_true_parser_mode_coarse_grained varchar(1024) null , + col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_true_parser_mode_fine_grained varchar(1024) null , + col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_false varchar(1024) null , + col_varchar_1024__undef_signed_not_null varchar(1024) not null , + col_varchar_1024__undef_signed_not_null_index_inverted varchar(1024) not null , + col_varchar_1024__undef_signed_not_null_index_inverted_support_phrase_true_parser_english varchar(1024) not null , + col_varchar_1024__undef_signed_not_null_index_inverted_parser_english_support_phrase_false varchar(1024) not null , + col_varchar_1024__undef_signed_not_null_index_inverted_parser_unicode_support_phrase_true varchar(1024) not null , + col_varchar_1024__undef_signed_not_null_index_inverted_parser_unicode_support_phrase_false varchar(1024) not null , + col_varchar_1024__undef_signed_not_null_index_inverted_parser_chinese_support_phrase_true_parser_mode_coarse_grained varchar(1024) not null , + col_varchar_1024__undef_signed_not_null_index_inverted_parser_chinese_support_phrase_true_parser_mode_fine_grained varchar(1024) not null , + col_varchar_1024__undef_signed_not_null_index_inverted_support_phrase_false_parser_chinese varchar(1024) not null , + INDEX col_int_undef_signed_index_inverted_idx (`col_int_undef_signed_index_inverted`) USING INVERTED, + INDEX col_int_undef_signed_not_null_index_inverted_idx (`col_int_undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_date_undef_signed_index_inverted_idx (`col_date_undef_signed_index_inverted`) USING INVERTED, + INDEX col_date_undef_signed_not_null_index_inverted_idx (`col_date_undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_varchar_1024__undef_signed_index_inverted_idx (`col_varchar_1024__undef_signed_index_inverted`) USING INVERTED, + INDEX col_varchar_1024__undef_signed_index_inv_0_idx (`col_varchar_1024__undef_signed_index_inverted_parser_english_support_phrase_true`) USING INVERTED PROPERTIES("parser" = "english", "support_phrase" = "true"), + INDEX col_varchar_1024__undef_signed_index_inv_1_idx (`col_varchar_1024__undef_signed_index_inverted_parser_english_support_phrase_false`) USING INVERTED PROPERTIES("parser" = "english", "support_phrase" = "false"), + INDEX col_varchar_1024__undef_signed_index_inv_2_idx (`col_varchar_1024__undef_signed_index_inverted_parser_unicode_support_phrase_true`) USING INVERTED PROPERTIES("parser" = "unicode", "support_phrase" = "true"), + INDEX col_varchar_1024__undef_signed_index_inv_3_idx (`col_varchar_1024__undef_signed_index_inverted_parser_unicode_support_phrase_false`) USING INVERTED PROPERTIES("parser" = "unicode", "support_phrase" = "false"), + INDEX col_varchar_1024__undef_signed_index_inv_4_idx (`col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_true_parser_mode_coarse_grained`) USING INVERTED PROPERTIES("parser" = "chinese", "support_phrase" = "true", "parser_mode" = "coarse_grained"), + INDEX col_varchar_1024__undef_signed_index_inv_5_idx (`col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_true_parser_mode_fine_grained`) USING INVERTED PROPERTIES("parser" = "chinese", "support_phrase" = "true", "parser_mode" = "fine_grained"), + INDEX col_varchar_1024__undef_signed_index_inv_6_idx (`col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_false`) USING INVERTED PROPERTIES("parser" = "chinese", "support_phrase" = "false"), + INDEX col_varchar_1024__undef_signed_not_null_index_inverted_idx (`col_varchar_1024__undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_varchar_1024__undef_signed_not_null__7_idx (`col_varchar_1024__undef_signed_not_null_index_inverted_support_phrase_true_parser_english`) USING INVERTED PROPERTIES("parser" = "english", "support_phrase" = "true"), + INDEX col_varchar_1024__undef_signed_not_null__8_idx (`col_varchar_1024__undef_signed_not_null_index_inverted_parser_english_support_phrase_false`) USING INVERTED PROPERTIES("parser" = "english", "support_phrase" = "false"), + INDEX col_varchar_1024__undef_signed_not_null__9_idx (`col_varchar_1024__undef_signed_not_null_index_inverted_parser_unicode_support_phrase_true`) USING INVERTED PROPERTIES("parser" = "unicode", "support_phrase" = "true"), + INDEX col_varchar_1024__undef_signed_not_null__10_idx (`col_varchar_1024__undef_signed_not_null_index_inverted_parser_unicode_support_phrase_false`) USING INVERTED PROPERTIES("parser" = "unicode", "support_phrase" = "false"), + INDEX col_varchar_1024__undef_signed_not_null__11_idx (`col_varchar_1024__undef_signed_not_null_index_inverted_parser_chinese_support_phrase_true_parser_mode_coarse_grained`) USING INVERTED PROPERTIES("parser" = "chinese", "support_phrase" = "true", "parser_mode" = "coarse_grained"), + INDEX col_varchar_1024__undef_signed_not_null__12_idx (`col_varchar_1024__undef_signed_not_null_index_inverted_parser_chinese_support_phrase_true_parser_mode_fine_grained`) USING INVERTED PROPERTIES("parser" = "chinese", "support_phrase" = "true", "parser_mode" = "fine_grained"), + INDEX col_varchar_1024__undef_signed_not_null__13_idx (`col_varchar_1024__undef_signed_not_null_index_inverted_support_phrase_false_parser_chinese`) USING INVERTED PROPERTIES("parser" = "chinese", "support_phrase" = "false") + ) engine=olap + DUPLICATE KEY(pk, col_int_undef_signed_index_inverted) + distributed by hash(pk) buckets 10 + properties("replication_num" = "1"); + """ + + sql """ + insert into table_30_undef_partitions2_keys3_properties4_distributed_by55(pk,col_int_undef_signed,col_int_undef_signed_index_inverted,col_int_undef_signed_not_null,col_int_undef_signed_not_null_index_inverted,col_date_undef_signed,col_date_undef_signed_index_inverted,col_date_undef_signed_not_null,col_date_undef_signed_not_null_index_inverted,col_varchar_1024__undef_signed,col_varchar_1024__undef_signed_index_inverted,col_varchar_1024__undef_signed_index_inverted_parser_english_support_phrase_true,col_varchar_1024__undef_signed_index_inverted_parser_english_support_phrase_false,col_varchar_1024__undef_signed_index_inverted_parser_unicode_support_phrase_true,col_varchar_1024__undef_signed_index_inverted_parser_unicode_support_phrase_false,col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_true_parser_mode_coarse_grained,col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_true_parser_mode_fine_grained,col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_false,col_varchar_1024__undef_signed_not_null,col_varchar_1024__undef_signed_not_null_index_inverted,col_varchar_1024__undef_signed_not_null_index_inverted_support_phrase_true_parser_english,col_varchar_1024__undef_signed_not_null_index_inverted_parser_english_support_phrase_false,col_varchar_1024__undef_signed_not_null_index_inverted_parser_unicode_support_phrase_true,col_varchar_1024__undef_signed_not_null_index_inverted_parser_unicode_support_phrase_false,col_varchar_1024__undef_signed_not_null_index_inverted_parser_chinese_support_phrase_true_parser_mode_coarse_grained,col_varchar_1024__undef_signed_not_null_index_inverted_parser_chinese_support_phrase_true_parser_mode_fine_grained,col_varchar_1024__undef_signed_not_null_index_inverted_support_phrase_false_parser_chinese) values (0,0,-1603201726,2,1,'2023-12-14','2024-02-18','2023-12-17','2011-04-03','and I''ll didn''t would her him back see','放心','','hey','see','--','注册','?','谈好','they','型号呢里交流你是谁柏霖焦作市中旬郑娜出团','?','g','肥姐也冲击随其自然每次鹏飞叫做亲戚华阳','we because who look about you''re as','h','金海','your so up at right got because who'),(1,2,null,1073463547,-1,'2023-12-11',null,'2024-02-18','2023-12-19','未税',null,'?','have don''t get now but','核对过','will','as','could','say I''m','?','?','签订','节点快乐今天儿童经销总代理出面','v','how','get he''s can yeah no her yes oh he''s would','青云不知道阿奎那干嘛维泽','r'),(2,1,1777219286,-2,2011685283,null,'2026-01-18','2025-06-18','2023-12-18','t','','使人','been',null,'?','','分公司平志伟长光','all oh are they his','调价东风路','c','先锋','宝龙','','彭巧语稳定','that''s out it about','good','want okay now he go do you''re something'),(3,1,6,7,0,'2025-06-18','2023-12-13','2024-02-18','2025-02-18','','?','分享合创人参金鳞','直管问起','','','走在网络失误','ok',null,'yeah back how yeah come he i hey were do','序列号','不懂','?','go','丰富','are don''t','then','泥石谁'),(4,-2,3,-1564396755,1197391189,'2026-02-18','2024-02-18','2023-12-16','2023-12-09','now','k','come you''re had I''ll in are look he''s on to','go',null,'ok you was','雅虎相关天天自私休息许昌有种效果','南天',null,'有些时候','?','could','his','--','of','we he can on and all','--','had'),(5,null,8,-2,0,null,'2023-12-14','2023-12-13','2024-02-18','','say','胡华威','','g','out yeah her didn''t for what like can''t not','about','r','--','?','something','?','--','l','再来全程若虚','一颗','','oh know was how when just not'),(6,-1,null,-2,-2,'2025-06-18','2023-12-15','2024-02-18','2024-01-17','x','','have as are know of a','b','time','at','her here know you your were as mean','a','--','know','in','家里应该山东能不鞥','我刚回来呀','--','--','?','原装双千兆莱克特上面妹妹飞机多久笔记本治疗那捕鲸船','?'),(7,null,null,-1723016302,-2,'2025-02-18','2024-01-19','2023-12-16','2025-02-17','--','晓得','on he''s with','--','电厂','小浪底接触','look','有点','','?','f','怎么样份款','want there did yes','老公你这个满意老刘勇气四天离谱有数广大不敢','back been at like','say not really','of all are could the know would','are'),(8,38379557,null,-1,-1,'2024-01-31','2027-01-09','2025-02-18','2024-01-09','--','分钟','will','惭愧李静加入我刚来呀颜色','look i got we really ok','谈谈','who','something',null,'','didn''t if','','j','证明金信会刊单挑刘金军港','--','--','going it',''),(9,968361924,0,-2,-2,null,'2023-12-16','2023-12-09','2023-12-19','got who tell in didn''t','been I''ll i if don''t it don''t then had','--','?','听着逍遥预计和兴涨价暴风影音就在埋在拼搏','原厂海信大品牌腾龙瑞丽盛源几天就走许愿树详谈','didn''t','don''t look me I''ll','','优惠最起码热卖不已本公司嘉通机会底下','mean','t','not not of come don''t go i it','up what come','going can say the time look of when','so','为什么',''),(10,-1,-1,752321536,-819779964,'2025-06-18','2024-01-31','2023-12-20','2023-12-12','--','?','挑战散落的反应都是准备都还多久大公司','','金蝶','',null,'you''re','热线在那普惠质保金烟火城市','?','one','','get it','b','负责','with','he i have about hey or you','中科部分一鸣国标'),(11,-1,3,-2,-1,'2023-12-13',null,'2025-02-17','2023-12-14','讨论组','?','they','鼠标','think he your want know','I''ll','it see in','--','see','图美创世纪','还跟他快运小曹别人','样机','有机会算错','go have had as all do I''ll','f','got all can didn''t something back had no','so','time'),(12,null,-1,5,0,'2023-12-15','2023-12-13','2023-12-20','2023-12-19',null,'?','询问无论汉威叁万领导地方亮度','注明','at just look from been at time','been','know your you''re','with now when in know been as I''ll','my he time with come','着呢很多','--','大学老师照片快运','?','could it if at tell was oh and yeah','--','all','不会被班车学校交叉照片治疗严格分数加载浪漫','?'),(13,-1,null,210811703,9,'2026-02-18','2002-12-15','2025-06-18','2026-02-18','不愿意','','know what','甘心是不是办事处数字见你难说发吧发吧晚上给把太原','but','贸易通','it really mean all how could when with all',null,'know something','now were all can''t can my tell oh they','didn''t up','真理','that''s know yeah i','d','as he they would been something his I''ll','张总曙光主机油漆正常','','was'),(14,-1,1291248968,1,-1,'2024-01-31','2024-01-19','2023-12-17','2024-02-18',null,'','when not not because back there so',null,'as ok but about up at see could come',null,'美女及其胡提高性能女孩健民没沈晓海不含','?','can','--','what say not she about is','齐村据对中病毒你问火球绿城只能唯一的','','便宜','--','','--','--'),(15,-1,-1,-1,-186244435,'2024-02-18','2024-02-18','2023-12-11','2027-01-09','so i come','right','because','做好','who','调价付费生活炒货谁家有货看着要不然心事报价格大话',null,'市场','','about do go know all','弄个','汇宝单单自己国栋底下刻录新款一周心情','停留批单','重新发一天第三方管理一千多绝对','me this she you''re were on','--','I''m','we to is he was not she'),(16,6,-2,-623692175,1438177278,'2025-06-18','2025-02-17','2024-01-09','2024-01-09','?','x','周星驰四川力天','','','老婆明白泰杰斯操作合计难说地市做方案','o','mean','--','外面不完利万','?','?','x','k','吃过饭仪器','as is like','--','--'),(17,1,-2,-2,-2,null,'2023-12-10','2023-12-16','2024-01-17',null,'g','did I''m her would could','回忆订票周星驰礼拜天代表王佩丽进一步房地产','think','can','look or want and no yeah','','','长信请你','组哟','同方','--','g','?','加密','外运','提示到来派人郑州还有要不然高亮屏你呢岂能被人'),(18,6,6,236306652,5,'2024-01-17','2023-12-15','2026-01-18','2023-12-11','安全正好标底圆圆跨区分钟不多','你用手机吧','it''s him back or it''s just','?','只有南阳路回家诊断他说你还没有回来啊北京报价格','了吗舍得良子停机都想共同三润','a','about','hey','he''s out to because ok mean','say','','his','命名层次还不错附上作风','光电清理着那个家不止它是热销中','think all and','核对轩明令你','about'),(19,-1,null,-1,-1,null,'2023-12-11','2023-12-11','2023-12-17','?',null,'what going from','莱克特','--','惊喜礼物刘海开头所谓开封吃肉我刚回来呀','子卡','y','who I''m now all to like got why','春江寸草心合作过半天班安小瑞达证明一次','--','out','有人你好呀重要机架短暂天地','yeah but on be did or there when','she','get here it''s see could like what would','','yeah her your'),(20,1256894923,-1,-1772431512,-1,'2023-12-12',null,'2027-01-09','2023-12-13','还得东风系列则会有谈好主演少秋经营第二','一夜开除做完疲劳单挑','or they','?','','no',null,'导入液晶移动关键','?','主办','心情常用生意说说提示','go one been yeah as','','会员卡','not get','伤心双飞燕安装','it''s','售后'),(21,-2,2,-1,-1,'2023-12-14','2023-12-11','2012-01-08','2025-02-17',null,'but','--','','would','教练史丽威达机柜唯一对了五万条快乐在乎志彬','a','电子卡将','a','','going','','拟在建爱找到严重商人大量晓燕写出来重要性营销聪明','独立','背景','l','d','back that were because will some going it did on'),(22,132295595,null,-1,-1,'2023-12-12','2025-06-18','2026-01-18','2026-02-18','相约总代也要想起来接收','?','?','?',null,'方式','?','认可','工业大学','?','one but','?','have','什么弯弯的月亮一套节省动物上有盘点始终','盛源分辨率另外图站','我给你打电话中病毒珍贵苦楚','小曹文宁正式版刀片柯艳琦','--'),(23,null,-1,1444541305,2,'2023-12-14','2023-12-14','2024-01-17','2023-12-17','would here from just going','out','about now back know it''s','丁丁圣荣松伟我还港湾金海','well it well of here is really','--','已经创世纪','?','','your','拿住网页委托收款海星报名说不出电厂通过短句','弟子快点','ok on this the see about at you you','no something don''t because see will why at she yeah','兴致订货治好出团预算赵总冲动','out why he''s her is that''s well','mean','?'),(24,-1482687319,5,-1,-1,null,'2023-12-10','2023-12-16','2023-12-20','','多少钱一群立刻你问若虚顺风晚上',null,'业茂','--','?','','使用','d','will in well','拿住','are','做好','say not it will for come no but so','--','位置综合学员能不鞥很好用文宁','no','not'),(25,null,4,8,5,'2023-12-17','2024-02-18','2024-01-31','2023-12-19','?','one they okay have when you''re','共同应该股份有限公司处理一下','--','离开','动作','would','客运量特此证明地区真理初次肆万小楼听雨北京','成交这块工商局','--','but','think','地区','because','--','?','网易有空吗晚上给把一颗真正以以上的授狗那里指点','图像'),(26,3,-792244912,-1,-2,'2023-12-20','2027-01-16','2027-01-09','2023-12-12','惊喜总代理商部队款且','?','x','王总签不签都浪潮年前分享应用独立雪松路黄委会冲动','yeah','简单嘉运达那么饲料核对过蠢材最新版本处理一下百川售后','at','','科贸被授予谈下来冀海潮交流群','here been yes because','先打款不忙','','then right it all not','?','--','why come going on out','k','资格'),(27,-1,-1,-2,1216258389,'2023-12-09','2024-01-19','2024-01-09','2023-12-20','--','?','喝酒','几下','?','know','?','洗手间','?','小键盘','will go did','身份证造成怎么着你吃法三业','o','--','k','--','号码学员星星珊瑚版','传真'),(28,7,-1844462362,6,-2,null,'2024-01-09','2023-12-14','2023-12-20','不然佩利飞机','对吧','come get why how to is what','he','do','发票浪漫没人','你问','--','?','说句','?','u','are','y','I''ll','oh in he''s i yeah no','what it''s','大学路商联'),(29,845502225,1862705239,-2,-1,'2023-12-10','2024-01-31','2024-01-17','2023-12-18','',null,null,'about ok about come for here','?','','--','口语铭岳刚出门热备不够','I''ll','没有直销免费想要周鹏留个鑫海源签字高亮屏漫长','办好未税图站','--','联恒你们已经上班了柏霖方向楼下群管理员保证良子能用吗神龙','just','','--','?','some'); + """ + + sql """ drop table if exists table_300_undef_partitions2_keys3_properties4_distributed_by5 """ + sql """ + create table table_300_undef_partitions2_keys3_properties4_distributed_by5 ( + col_int_undef_signed int/*agg_type_placeholder*/ null , + col_int_undef_signed_index_inverted int/*agg_type_placeholder*/ null , + col_int_undef_signed_not_null int/*agg_type_placeholder*/ not null , + col_int_undef_signed_not_null_index_inverted int/*agg_type_placeholder*/ not null , + col_date_undef_signed date/*agg_type_placeholder*/ null , + col_date_undef_signed_index_inverted date/*agg_type_placeholder*/ null , + col_date_undef_signed_not_null date/*agg_type_placeholder*/ not null , + col_date_undef_signed_not_null_index_inverted date/*agg_type_placeholder*/ not null , + col_varchar_1024__undef_signed varchar(1024)/*agg_type_placeholder*/ null , + col_varchar_1024__undef_signed_index_inverted varchar(1024)/*agg_type_placeholder*/ null , + col_varchar_1024__undef_signed_index_inverted_parser_english_support_phrase_true varchar(1024)/*agg_type_placeholder*/ null , + col_varchar_1024__undef_signed_index_inverted_parser_english_support_phrase_false varchar(1024)/*agg_type_placeholder*/ null , + col_varchar_1024__undef_signed_index_inverted_parser_unicode_support_phrase_true varchar(1024)/*agg_type_placeholder*/ null , + col_varchar_1024__undef_signed_index_inverted_parser_unicode_support_phrase_false varchar(1024)/*agg_type_placeholder*/ null , + col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_true_parser_mode_coarse_grained varchar(1024)/*agg_type_placeholder*/ null , + col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_true_parser_mode_fine_grained varchar(1024)/*agg_type_placeholder*/ null , + col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_false varchar(1024)/*agg_type_placeholder*/ null , + col_varchar_1024__undef_signed_not_null varchar(1024)/*agg_type_placeholder*/ not null , + col_varchar_1024__undef_signed_not_null_index_inverted varchar(1024)/*agg_type_placeholder*/ not null , + col_varchar_1024__undef_signed_not_null_index_inverted_support_phrase_true_parser_english varchar(1024)/*agg_type_placeholder*/ not null , + col_varchar_1024__undef_signed_not_null_index_inverted_parser_english_support_phrase_false varchar(1024)/*agg_type_placeholder*/ not null , + col_varchar_1024__undef_signed_not_null_index_inverted_parser_unicode_support_phrase_true varchar(1024)/*agg_type_placeholder*/ not null , + col_varchar_1024__undef_signed_not_null_index_inverted_parser_unicode_support_phrase_false varchar(1024)/*agg_type_placeholder*/ not null , + col_varchar_1024__undef_signed_not_null_index_inverted_parser_chinese_support_phrase_true_parser_mode_coarse_grained varchar(1024)/*agg_type_placeholder*/ not null , + col_varchar_1024__undef_signed_not_null_index_inverted_parser_chinese_support_phrase_true_parser_mode_fine_grained varchar(1024)/*agg_type_placeholder*/ not null , + col_varchar_1024__undef_signed_not_null_index_inverted_support_phrase_false_parser_chinese varchar(1024)/*agg_type_placeholder*/ not null , + pk int/*agg_type_placeholder*/, + INDEX col_int_undef_signed_index_inverted_idx (`col_int_undef_signed_index_inverted`) USING INVERTED, + INDEX col_int_undef_signed_not_null_index_inverted_idx (`col_int_undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_date_undef_signed_index_inverted_idx (`col_date_undef_signed_index_inverted`) USING INVERTED, + INDEX col_date_undef_signed_not_null_index_inverted_idx (`col_date_undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_varchar_1024__undef_signed_index_inverted_idx (`col_varchar_1024__undef_signed_index_inverted`) USING INVERTED, + INDEX col_varchar_1024__undef_signed_index_inv_0_idx (`col_varchar_1024__undef_signed_index_inverted_parser_english_support_phrase_true`) USING INVERTED PROPERTIES("parser" = "english", "support_phrase" = "true"), + INDEX col_varchar_1024__undef_signed_index_inv_1_idx (`col_varchar_1024__undef_signed_index_inverted_parser_english_support_phrase_false`) USING INVERTED PROPERTIES("parser" = "english", "support_phrase" = "false"), + INDEX col_varchar_1024__undef_signed_index_inv_2_idx (`col_varchar_1024__undef_signed_index_inverted_parser_unicode_support_phrase_true`) USING INVERTED PROPERTIES("parser" = "unicode", "support_phrase" = "true"), + INDEX col_varchar_1024__undef_signed_index_inv_3_idx (`col_varchar_1024__undef_signed_index_inverted_parser_unicode_support_phrase_false`) USING INVERTED PROPERTIES("parser" = "unicode", "support_phrase" = "false"), + INDEX col_varchar_1024__undef_signed_index_inv_4_idx (`col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_true_parser_mode_coarse_grained`) USING INVERTED PROPERTIES("parser" = "chinese", "support_phrase" = "true", "parser_mode" = "coarse_grained"), + INDEX col_varchar_1024__undef_signed_index_inv_5_idx (`col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_true_parser_mode_fine_grained`) USING INVERTED PROPERTIES("parser" = "chinese", "support_phrase" = "true", "parser_mode" = "fine_grained"), + INDEX col_varchar_1024__undef_signed_index_inv_6_idx (`col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_false`) USING INVERTED PROPERTIES("parser" = "chinese", "support_phrase" = "false"), + INDEX col_varchar_1024__undef_signed_not_null_index_inverted_idx (`col_varchar_1024__undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_varchar_1024__undef_signed_not_null__7_idx (`col_varchar_1024__undef_signed_not_null_index_inverted_support_phrase_true_parser_english`) USING INVERTED PROPERTIES("parser" = "english", "support_phrase" = "true"), + INDEX col_varchar_1024__undef_signed_not_null__8_idx (`col_varchar_1024__undef_signed_not_null_index_inverted_parser_english_support_phrase_false`) USING INVERTED PROPERTIES("parser" = "english", "support_phrase" = "false"), + INDEX col_varchar_1024__undef_signed_not_null__9_idx (`col_varchar_1024__undef_signed_not_null_index_inverted_parser_unicode_support_phrase_true`) USING INVERTED PROPERTIES("parser" = "unicode", "support_phrase" = "true"), + INDEX col_varchar_1024__undef_signed_not_null__10_idx (`col_varchar_1024__undef_signed_not_null_index_inverted_parser_unicode_support_phrase_false`) USING INVERTED PROPERTIES("parser" = "unicode", "support_phrase" = "false"), + INDEX col_varchar_1024__undef_signed_not_null__11_idx (`col_varchar_1024__undef_signed_not_null_index_inverted_parser_chinese_support_phrase_true_parser_mode_coarse_grained`) USING INVERTED PROPERTIES("parser" = "chinese", "support_phrase" = "true", "parser_mode" = "coarse_grained"), + INDEX col_varchar_1024__undef_signed_not_null__12_idx (`col_varchar_1024__undef_signed_not_null_index_inverted_parser_chinese_support_phrase_true_parser_mode_fine_grained`) USING INVERTED PROPERTIES("parser" = "chinese", "support_phrase" = "true", "parser_mode" = "fine_grained"), + INDEX col_varchar_1024__undef_signed_not_null__13_idx (`col_varchar_1024__undef_signed_not_null_index_inverted_support_phrase_false_parser_chinese`) USING INVERTED PROPERTIES("parser" = "chinese", "support_phrase" = "false") + ) engine=olap + distributed by hash(pk) buckets 10 + properties("replication_num" = "1"); + """ + + sql """ + insert into table_300_undef_partitions2_keys3_properties4_distributed_by5(pk,col_int_undef_signed,col_int_undef_signed_index_inverted,col_int_undef_signed_not_null,col_int_undef_signed_not_null_index_inverted,col_date_undef_signed,col_date_undef_signed_index_inverted,col_date_undef_signed_not_null,col_date_undef_signed_not_null_index_inverted,col_varchar_1024__undef_signed,col_varchar_1024__undef_signed_index_inverted,col_varchar_1024__undef_signed_index_inverted_parser_english_support_phrase_true,col_varchar_1024__undef_signed_index_inverted_parser_english_support_phrase_false,col_varchar_1024__undef_signed_index_inverted_parser_unicode_support_phrase_true,col_varchar_1024__undef_signed_index_inverted_parser_unicode_support_phrase_false,col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_true_parser_mode_coarse_grained,col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_true_parser_mode_fine_grained,col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_false,col_varchar_1024__undef_signed_not_null,col_varchar_1024__undef_signed_not_null_index_inverted,col_varchar_1024__undef_signed_not_null_index_inverted_support_phrase_true_parser_english,col_varchar_1024__undef_signed_not_null_index_inverted_parser_english_support_phrase_false,col_varchar_1024__undef_signed_not_null_index_inverted_parser_unicode_support_phrase_true,col_varchar_1024__undef_signed_not_null_index_inverted_parser_unicode_support_phrase_false,col_varchar_1024__undef_signed_not_null_index_inverted_parser_chinese_support_phrase_true_parser_mode_coarse_grained,col_varchar_1024__undef_signed_not_null_index_inverted_parser_chinese_support_phrase_true_parser_mode_fine_grained,col_varchar_1024__undef_signed_not_null_index_inverted_support_phrase_false_parser_chinese) values (0,-2,-1901730183,5,-1,'2023-12-14',null,'2023-12-16','2023-12-20','肯定','华康','--','from but when okay would when','','弹出','from my','下月很难说奇偶上班不止单位接受','加密狗铁路董事长重命名策略一点河南压力为你','提示硕博伟泽聊聊冰河心理学对吧','中小金汇通','then can','','will there don''t had did want been some','f','过年','her','--'),(1,1382192944,-1,-234802337,-1,'2025-02-18','2025-02-17','2023-12-12','2027-01-16','国务卿','','look back i in you me say look go','孙瑞霞很想成本还是记录那位','well do he say can it''s oh don''t','--','--','got','think','there','合计回公司遮盖很大置业','','成绩','from','what','原因的','','是的交单记得拆机有点不好漫长司军几关系'),(2,-1,-2,2,-2,'2004-04-02','2023-12-19','2023-12-15','2023-12-12','up','--','汇众孤独办款名家生后打算武汉瀚海答案','--','f','could',null,'差价图形工作站中卢有伴随熟悉大量中创热备软件产生休息','','是你呀有车吗','不过图腾水哦这话大家冀海潮','good say like','my he some yes come so','--','or','?','a come a','see'),(3,-2,-2,-2017523107,-2,'2023-12-17','2027-01-16','2024-01-31','2023-12-17','你家报表表情','不走除非飞信许昌后勤教练新买这一生不下令你','你杀时间回来',null,'time there he well it with me','排列','表情','招工','right','请他','I''m mean be from','','?','完美','here my','b','关于','c'),(4,1065841778,-2,-1,-1,'2024-02-18',null,'2025-06-18','2023-12-20','左右','','?','面子故事离开邮箱合适','ok why here time in that','等级','i','--',null,'be look out of had we to what be don''t','no can hey to get','or as you''re about','go all her some well','只有塔式海花见你名称也冲击幸福跟进中核对过只有','see could i','?','?','海星动态刻意海星刘畅上不了学习'),(5,1458245006,null,-1,-2,'2026-01-18','2023-12-13','2024-01-31','2024-01-08','?','v','when to could','--','发布神秘一小时试过备用金','g','用管','v','这是','is','g','','then','--','','','王哥','p'),(6,3,-1,-239832171,-1,'2024-01-08','2024-01-17','2023-12-12','2026-02-18','is well if all time have you''re','n','?','have',null,'键盘鼠标',null,'第二上门十分新远方表达商丘','the going were at mean','there like been on and yes what or','--','--','go','it''s','花月夜','什么时间','行业屏保长得','我在政治哦同志们参加'),(7,-2,-2,-1,1754315155,'2024-02-18','2023-12-19','2025-02-18','2023-12-14','','--','','显卡宇瑞鑫海源关系','or don''t no ok is',null,'科技大厦经常实现航务新增而是阿奎那纸业自动','x','寝室','then','用户','i how how','一两智博在不在数据公寓电粉','礼拜天','well you here so been what okay see something','--','河南总经销','?'),(8,112676751,-127780972,-1937697475,5,'2024-01-17','2023-12-11','2023-12-16','2024-01-09','三阳','you''re','who','残品作风折旧到底怎么办正规承认又给','','have','r','r','j','if is in','空间冗余','his','','浇水过节费先付新增综合杨学说不','你不认识华阳跟着预计长光理光许昌想象','e','','千金屏保九九马马虎虎怡海'),(9,3,-957042490,3,-1,'2023-12-09','2023-12-20','2023-12-16','2023-12-09',null,'每一小时',null,'was',null,'','广州市','me or you''re look right got when you''re about','it''s okay this up how are for','t','was she don''t at i know can when because','go','how here ok okay','got her he it''s could can''t back had about have','she','?','he''s him they well I''m yeah',''),(10,-200104483,-1,4,-2,'2023-12-16','2025-06-18','2024-02-18','2024-02-18','see','j','今晚','how','态度','he','will','him','不多','had about did','眼睁睁','--','?','打电话','到来','利盟','w','歘俩'),(11,-1,null,-2,-2,null,'2023-12-10','2025-02-18','2024-01-08',null,'生物','你吃法','my','这几太难','状态第二次郑娜备份文件三个业务下班','零捌','time I''ll about on they because don''t good why will','就算格飞五洲一块小妹特性吃过饭叁仟肯定写上去','--','大雪服务站光纤跃民独显从你那里石龙汇宝真是','','on get you''re will','him','why','分开信阳','填写','him of yeah my get they can'),(12,7,8,1567572056,-2,'2023-12-13','2025-02-18','2025-06-18','2023-12-11','命名操心你怎么去的保密中档说吧淀粉英联生命中这几天','some','则会有','充满',null,'领出','i','would oh me are','was','字库文件','?','老公好早一定','见面蓝牙','?','不如孙海洋一辈子蓝牙晓燕','--','it didn''t or hey some think do like here','自私空间'),(13,null,-2079628735,-2,1,'2023-12-11','2023-12-15','2023-12-18','2024-01-09','','--','?','停产','?','农村一起',null,'','一种下了期待刘总圣诞节金汇通千万','见到李东贸易通','?','?','have get did the i to then','for','--','?','线上输入不再梦想朱晓明','who with don''t for that ok him can one'),(14,7,-1104140591,-2,-1,'2023-12-16','2024-01-08','2025-02-17','2023-12-10',null,null,'oh ok','a','?','也许','?','could you','','对你以上有人多家','请他信海维修总之张海燕孤独过去列表奖金','say','or','it''s','o','so that''s she she yeah the right if','?','?'),(15,-2,-1,-2,-1,'2023-12-09','2025-02-17','2024-01-17','2023-12-13','今晚','up ok can but i do one','','will','s',null,'v','b','王枫','come','广发','for I''m now can hey when because time come','?','were','go some ok something of','富通你好删过刚刚','when yeah think how I''ll now','--'),(16,1,null,1025070939,-1065499727,'2024-02-18','2025-02-17','2023-12-19','2024-01-31','','富通怡海赶快瑞星左右河南经销','not','姓江','was','何时','v','me can''t now he ok is been from hey we','o','--','--','we','s','b','that for here at you''re can''t right hey yes','come hey on here yes mean going it''s about or','j','中铁'),(17,1406750472,null,-2,5,'2023-12-16','2024-01-09','2024-01-08','2024-02-18','?',null,'the been is but then can''t yes him this really','really okay say tell how','但愿',null,'--','we right that was I''m','?','','','something','五万多条','--','','','在家里主页绝对北环备份文件','几个工业大学带你正道直接费用科峰不需要鑫海源接口'),(18,6,-1,1073335855,6,'2023-12-18','2023-12-19','2025-02-17','2023-12-10','hey','ok','have was my like','万一证明雅虎','his at didn''t it','分销商中层经理网通等于','维泽特配机器备份文件招工河南经销新普怡海','不可能奖励空间冗余加密狗分钟玉清','?','can''t not','c','空间冗余','--','u','--','have at now was yes have out','would then now is','y'),(19,null,-2,-2,-1,'2024-02-18','2025-06-18','2027-01-16','2024-01-31','could good','up i I''ll','mean','--','--','记得一岁无聊也得空间冗余打印机','英特','客户',null,'','got','明月这个扩展名城市科贸','世安误区独显安装愿意联盟比爱白贰台','?','简介','come a that''s or','?','go'),(20,-2,null,-2,634169930,'2023-12-16','2024-01-17','2024-01-17','2023-12-16','be','?','','--','had','天孝','','then ok to that think this all you like will','细节新闻有钱老公张小莉这点','河南总经销个人自己','有些时候在呀自己的刷卡','j','','I''ll all or','that no why your then okay some her I''ll i','','have','帐户宠物关机也是'),(21,-1,null,-1,0,'2025-02-17','2024-01-31','2024-01-08','2024-01-31','等于合作单位看待解难答案三种','','折扣不行立博我那个叫东西上半年沃尔普会员卡','那款武警总队今晚华北区输得起不曾很想熊猫','?',null,'but out ok how','out',null,'it''s','','三阳开泰','on and he''s and one know know yes good or','帮帮','伟泽这么双子安阳咨询服务时间磁盘柜高密度','速度钢琴曲金牌总代理顺河路彩霞金立下一部分广告下了黄河科学','p','why'),(22,-1,1,-2,-2,'2027-01-16','2024-01-09','2023-12-15','2023-12-15','工资','are really I''m','when of was in well a get think well','are','--','know','?','well say about been going why would yeah out go','?','r','did he''s be something out can''t some him be','','痛苦私人水货绿城打击不起网站关键','不说需要帮忙做到经销商历来三润十来年记得','your in no have got','or look','磁盘你们已经上班了丹尼斯李金才泡茶等等赵洪应一共','进入了'),(23,-1,-2,-2,-2,'2023-12-09','2024-02-18','2023-12-16','2023-12-16','本人','屠龙记','at','瑞昌','?','r','he had are going he''s because from would',null,'we','p','你不中小','all he good go would go my up a what','中天','didn''t','why','w','联盟新闻重新化验好处职务其中欧颜色',''),(24,-2,null,-2117558545,720042588,'2027-01-09','2014-06-24','2023-12-19','2025-02-18','?','l','been you i','广大两千量大优惠索尼','c','come go know when','新买','忘不了大部分有误国有企业','this','加为好友','I''m','your were just as of look good a your or','侠诺灵生不敢电视海川回你在家里海花本科市场上','for didn''t','最重要哪呢个五万多条难过姑娘','for about the I''m yeah go hey','我刚回来呀','--'),(25,9,-2,-1,-1562783499,'2024-01-09','2025-06-18','2025-02-17','2024-01-08','珍惜不再','--','下个','?','电话','some see go from don''t that''s be really','杀毒你来找我吧日常也有过政府','鼠标','you''re here back about she we some don''t they','--','中的','--','on say is on what','卖友公路说吧答案','这款','结果','基地连续公司近台南不愿意','q'),(26,null,null,566690289,-1,'2024-01-17','2023-12-19','2023-12-13','2024-02-18','about','隆康王子','就有','n','中层经理','购买','虚妄主奴婢最近忙什么呢找你见到新区外面','be is','','北环私人郑大沧田第一次请他肛门锐成文件夹浪潮','下来','?','z','生活屏保派送','?','--','万方雷林授权发生过焕然去不了生意还以为直供','he one go'),(27,8,-1329519821,5,-1,'2023-12-19','2023-12-14','2023-12-20','2025-06-18','some they want I''ll they yeah going','花园','原因','?','中午随风很好用珊瑚版比人很多输得起售后王燕纯度',null,'i','because i look what','--','his','why','--','that''s','the it for some are don''t here','oh back say they are from','新密韩鸽飞空分电脑一部一万有机会开发强暴找你','你休息吸纳中脱机正式版视讯重要性客户群以上','it''s'),(28,4,7,738661869,1,'2024-02-18','2024-01-19','2025-02-18','2023-12-15','?',null,'都想','王星银河外运保留令你年后使人瑞星','?','hey up','we say','科技市场','刘雪','s','--','something','tell','办事处进去祝福','get with hey that''s back','玉冰','老刘批发价忙碌小型机','三石'),(29,0,143593368,-2,335460636,'2026-02-18','2025-02-18','2025-06-18','2026-01-18',null,'暂时控制卡','?','知道','b','占到热线我给你打吧企业方法首创那你只能员工','','安信情歌内存王晶威达他娘的语文说好','离开逝去自己小苏网卡新一代武汉方法色戒','找不到','time','小楼听雨宝贵不但是忙碌飞扬农科阿奎那一套虚妄','废话王总签不签都各位收集','既然','宫颈糜烂','惊喜金牌','this',''); + """ + + qt_sql """ + select count() from table_30_undef_partitions2_keys3_properties4_distributed_by55 t1, table_300_undef_partitions2_keys3_properties4_distributed_by5 t2 where t1. col_date_undef_signed_index_inverted is not null or (t1. col_varchar_1024__undef_signed_index_inverted_parser_chinese_support_phrase_true_parser_mode_coarse_grained MATCH "儿童") and not (t2. col_int_undef_signed is null) and not (t1. col_int_undef_signed_not_null_index_inverted is null or t1. col_int_undef_signed_not_null_index_inverted is not null); + """ + +} From d7eb3431095d529a42fa12429651ebb1608a360b Mon Sep 17 00:00:00 2001 From: Sun Chenyang Date: Thu, 29 Aug 2024 12:01:38 +0800 Subject: [PATCH 13/60] [fix] (UT) fix be ut InvertedIndexArrayTest (#40090) ## Proposed changes fix be ut InvertedIndexArrayTest --- .../segment_v2/inverted_index_array_test.cpp | 26 ++++++++++++------- 1 file changed, 16 insertions(+), 10 deletions(-) diff --git a/be/test/olap/rowset/segment_v2/inverted_index_array_test.cpp b/be/test/olap/rowset/segment_v2/inverted_index_array_test.cpp index 469f5243c79f5e..8621b81518af80 100644 --- a/be/test/olap/rowset/segment_v2/inverted_index_array_test.cpp +++ b/be/test/olap/rowset/segment_v2/inverted_index_array_test.cpp @@ -58,13 +58,21 @@ class InvertedIndexArrayTest : public testing::Test { public: const std::string kTestDir = "./ut_dir/inverted_index_array_test"; - void check_terms_stats(string dir_str, string file_str) { - CLuceneError err; - CL_NS(store)::IndexInput* index_input = nullptr; - DorisFSDirectory::FSIndexInput::open(io::global_local_filesystem(), file_str.c_str(), - index_input, err, 4096); - std::unique_ptr reader = - std::make_unique(index_input, 4096); + void check_terms_stats(string file_str) { + std::unique_ptr reader; + try { + CLuceneError err; + CL_NS(store)::IndexInput* index_input = nullptr; + auto ok = DorisFSDirectory::FSIndexInput::open( + io::global_local_filesystem(), file_str.c_str(), index_input, err, 4096); + if (!ok) { + throw err; + } + reader = std::make_unique(index_input, 4096); + } catch (...) { + EXPECT_TRUE(false); + } + std::cout << "Term statistics for " << file_str << std::endl; std::cout << "==================================" << std::endl; lucene::store::Directory* dir = reader.get(); @@ -121,8 +129,6 @@ class InvertedIndexArrayTest : public testing::Test { InvertedIndexDescriptor::get_index_file_path_v1(index_path_prefix, index_id, ""); auto fs = io::global_local_filesystem(); - io::FileWriterPtr file_writer; - EXPECT_TRUE(fs->create_file(index_path, &file_writer).ok()); auto index_meta_pb = std::make_unique(); index_meta_pb->set_index_type(IndexType::INVERTED); index_meta_pb->set_index_id(index_id); @@ -204,7 +210,7 @@ class InvertedIndexArrayTest : public testing::Test { EXPECT_EQ(_inverted_index_builder->finish(), Status::OK()); EXPECT_EQ(index_file_writer->close(), Status::OK()); - check_terms_stats(file_writer->path().parent_path(), file_writer->path().filename()); + check_terms_stats(index_path); } }; From e656cb7980ddb440ff614966c0f80fb809dd449b Mon Sep 17 00:00:00 2001 From: qiye Date: Thu, 29 Aug 2024 12:05:52 +0800 Subject: [PATCH 14/60] [fix](build index)Only get file size for inverted index (#39965) close #39963 Master will not have the warning logs due to `file_system.h` removed the log line. But the bug still exists. --- be/src/olap/rowset/beta_rowset.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/be/src/olap/rowset/beta_rowset.cpp b/be/src/olap/rowset/beta_rowset.cpp index b269051e43f455..209aca7fb03b4c 100644 --- a/be/src/olap/rowset/beta_rowset.cpp +++ b/be/src/olap/rowset/beta_rowset.cpp @@ -83,6 +83,10 @@ Status BetaRowset::get_inverted_index_size(size_t* index_size) { if (_schema->get_inverted_index_storage_format() == InvertedIndexStorageFormatPB::V1) { auto indices = _schema->indexes(); for (auto& index : indices) { + // only get file_size for inverted index + if (index.index_type() != IndexType::INVERTED) { + continue; + } for (int seg_id = 0; seg_id < num_segments(); ++seg_id) { auto seg_path = DORIS_TRY(segment_path(seg_id)); int64_t file_size = 0; From ea4c77f9622f218ff76d4743e9a9d82cfc3452c4 Mon Sep 17 00:00:00 2001 From: hui lai <1353307710@qq.com> Date: Thu, 29 Aug 2024 14:12:02 +0800 Subject: [PATCH 15/60] [chore](routine load) make error msg clear if routine load name illegal (#40037) Make error msg clear if routine load name illegal(such as job name is too long). --- .../doris/analysis/CreateRoutineLoadStmt.java | 9 +++- .../org/apache/doris/common/FeNameFormat.java | 2 + .../test_routine_load_name.groovy | 44 +++++++++++++++++++ 3 files changed, 54 insertions(+), 1 deletion(-) create mode 100644 regression-test/suites/load_p0/routine_load/test_routine_load_name.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java index 13654509821d12..1648d943ae4b89 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java @@ -347,7 +347,14 @@ public void analyze(Analyzer analyzer) throws UserException { // check dbName and tableName checkDBTable(analyzer); // check name - FeNameFormat.checkCommonName(NAME_TYPE, name); + try { + FeNameFormat.checkCommonName(NAME_TYPE, name); + } catch (AnalysisException e) { + // 64 is the length of regular expression matching + // (FeNameFormat.COMMON_NAME_REGEX/UNDERSCORE_COMMON_NAME_REGEX) + throw new AnalysisException(e.getMessage() + + " Maybe routine load job name is longer than 64 or contains illegal characters"); + } // check load properties include column separator etc. checkLoadProperties(); // check routine load job properties include desired concurrent number etc. diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/FeNameFormat.java b/fe/fe-core/src/main/java/org/apache/doris/common/FeNameFormat.java index 287d23686607e7..18dfce2fc081a6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/FeNameFormat.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/FeNameFormat.java @@ -30,6 +30,8 @@ public class FeNameFormat { private static final String LABEL_REGEX = "^[-_A-Za-z0-9:]{1," + Config.label_regex_length + "}$"; + // if modify the matching length of a regular expression, + // please modify error msg when FeNameFormat.checkCommonName throw exception in CreateRoutineLoadStmt private static final String COMMON_NAME_REGEX = "^[a-zA-Z][a-zA-Z0-9-_]{0,63}$"; private static final String UNDERSCORE_COMMON_NAME_REGEX = "^[_a-zA-Z][a-zA-Z0-9-_]{0,63}$"; private static final String TABLE_NAME_REGEX = "^[a-zA-Z][a-zA-Z0-9-_]*$"; diff --git a/regression-test/suites/load_p0/routine_load/test_routine_load_name.groovy b/regression-test/suites/load_p0/routine_load/test_routine_load_name.groovy new file mode 100644 index 00000000000000..53606fbdf6ef33 --- /dev/null +++ b/regression-test/suites/load_p0/routine_load/test_routine_load_name.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("test_routine_load_name","p0") { + String kafka_port = context.config.otherConfigs.get("kafka_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + try { + sql """ + CREATE ROUTINE LOAD test_routine_load_name_too_much_filler_filler_filler_filler_filler_filler_filler + COLUMNS TERMINATED BY "|" + PROPERTIES + ( + "max_batch_interval" = "5", + "max_batch_rows" = "300000", + "max_batch_size" = "209715200" + ) + FROM KAFKA + ( + "kafka_broker_list" = "${externalEnvIp}:${kafka_port}", + "kafka_topic" = "multi_table_load_invalid_table", + "property.kafka_default_offsets" = "OFFSET_BEGINNING" + ); + """ + } catch (Exception e) { + log.info("exception: ${e.toString()}".toString()) + assertEquals(e.toString().contains("Incorrect ROUTINE LOAD NAME name"), true) + assertEquals(e.toString().contains("required format is"), true) + assertEquals(e.toString().contains("Maybe routine load job name is longer than 64 or contains illegal characters"), true) + } +} \ No newline at end of file From e24bc7f650ab7dbc2e974e68e42cdd5be38ac776 Mon Sep 17 00:00:00 2001 From: zhangdong <493738387@qq.com> Date: Thu, 29 Aug 2024 14:47:30 +0800 Subject: [PATCH 16/60] [chore](mtmv)add plan log for UpdateMvByPartitionCommand (#40072) fe.conf set `sys_log_verbose_modules = org.apache.doris.nereids.trees.plans.commands.UpdateMvByPartitionCommand` --- .../trees/plans/commands/UpdateMvByPartitionCommand.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java index ba1a054752b55a..de284bd837748f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java @@ -65,6 +65,8 @@ import com.google.common.collect.Lists; import com.google.common.collect.Range; import com.google.common.collect.Sets; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.ArrayList; import java.util.HashSet; @@ -79,6 +81,8 @@ * Update mv by partition */ public class UpdateMvByPartitionCommand extends InsertOverwriteTableCommand { + private static final Logger LOG = LogManager.getLogger(UpdateMvByPartitionCommand.class); + private UpdateMvByPartitionCommand(LogicalPlan logicalQuery) { super(logicalQuery, Optional.empty(), Optional.empty()); } @@ -104,6 +108,10 @@ public static UpdateMvByPartitionCommand from(MTMV mv, Set partitionName } LogicalSink sink = UnboundTableSinkCreator.createUnboundTableSink(mv.getFullQualifiers(), ImmutableList.of(), ImmutableList.of(), parts, plan); + if (LOG.isDebugEnabled()) { + LOG.debug("MTMVTask plan for mvName: {}, partitionNames: {}, plan: {}", mv.getName(), partitionNames, + sink.treeString()); + } return new UpdateMvByPartitionCommand(sink); } From 9cb7d83ed858c44dbcc1d845437c818caca78578 Mon Sep 17 00:00:00 2001 From: zhangdong <493738387@qq.com> Date: Thu, 29 Aug 2024 15:49:50 +0800 Subject: [PATCH 17/60] [fix](auth) ordinary users can see the processes of other users (#39747) Fix when set show_all_fe_connection="true"; At this time, ordinary users can see the processes of other users --- .../main/java/org/apache/doris/qe/ConnectScheduler.java | 9 ++++++++- .../src/main/java/org/apache/doris/qe/ShowExecutor.java | 1 + .../org/apache/doris/service/FrontendServiceImpl.java | 6 +++++- gensrc/thrift/FrontendService.thrift | 1 + 4 files changed, 15 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectScheduler.java index dfd6f0ebdae4fb..cafe9edd3a18f0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectScheduler.java @@ -17,6 +17,7 @@ package org.apache.doris.qe; +import org.apache.doris.analysis.UserIdentity; import org.apache.doris.catalog.Env; import org.apache.doris.common.ThreadPoolManager; import org.apache.doris.common.util.DebugUtil; @@ -173,10 +174,16 @@ public List listConnection(String user, boolean isFul } // used for thrift - public List> listConnectionWithoutAuth(boolean isShowFullSql) { + public List> listConnectionForRpc(UserIdentity userIdentity, boolean isShowFullSql) { List> list = new ArrayList<>(); long nowMs = System.currentTimeMillis(); for (ConnectContext ctx : connectionMap.values()) { + // Check auth + if (!ctx.getCurrentUserIdentity().equals(userIdentity) && !Env.getCurrentEnv() + .getAccessManager() + .checkGlobalPriv(userIdentity, PrivPredicate.GRANT)) { + continue; + } list.add(ctx.toThreadInfo(isShowFullSql).toRow(-1, nowMs)); } return list; 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 9c807a4b1bf9fe..1cbfcf8ba37fa9 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 @@ -533,6 +533,7 @@ private void handleShowProcesslist() { try { TShowProcessListRequest request = new TShowProcessListRequest(); request.setShowFullSql(isShowFullSql); + request.setCurrentUserIdent(ConnectContext.get().getCurrentUserIdentity().toThrift()); List> frontends = FrontendsProcNode.getFrontendWithRpcPort(Env.getCurrentEnv(), false); FrontendService.Client client = null; 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 f64f55a47fe62c..157ad280708485 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 @@ -3963,8 +3963,12 @@ public TShowProcessListResult showProcessList(TShowProcessListRequest request) { if (request.isSetShowFullSql()) { isShowFullSql = request.isShowFullSql(); } + UserIdentity userIdentity = UserIdentity.ROOT; + if (request.isSetCurrentUserIdent()) { + userIdentity = UserIdentity.fromThrift(request.getCurrentUserIdent()); + } List> processList = ExecuteEnv.getInstance().getScheduler() - .listConnectionWithoutAuth(isShowFullSql); + .listConnectionForRpc(userIdentity, isShowFullSql); TShowProcessListResult result = new TShowProcessListResult(); result.setProcessList(processList); return result; diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index b6e4aacf65614a..2dcdf9b718036f 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -1520,6 +1520,7 @@ struct TGetColumnInfoResult { struct TShowProcessListRequest { 1: optional bool show_full_sql + 2: optional Types.TUserIdentity current_user_ident } struct TShowProcessListResult { From fb509eedd233065166b8fbe62dc152f5ed9d1b41 Mon Sep 17 00:00:00 2001 From: zhangdong <493738387@qq.com> Date: Thu, 29 Aug 2024 15:50:36 +0800 Subject: [PATCH 18/60] [fix](mtmv) Mtmv support set both immediate and starttime (#39573) extends: #36805 Previously, if the user set immediate execution, the current time would replace the user's set start time --- .../org/apache/doris/mtmv/MTMVJobManager.java | 8 +- .../data/mtmv_p0/test_start_time_mtmv.out | 17 +++++ .../mtmv_p0/test_start_time_mtmv.groovy | 76 +++++++++++++++++++ 3 files changed, 97 insertions(+), 4 deletions(-) create mode 100644 regression-test/data/mtmv_p0/test_start_time_mtmv.out create mode 100644 regression-test/suites/mtmv_p0/test_start_time_mtmv.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobManager.java index 11089899b309a8..1ace738f1d0d9f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobManager.java @@ -105,14 +105,14 @@ private void setScheduleJobConfig(JobExecutionConfiguration jobExecutionConfigur .setInterval(mtmv.getRefreshInfo().getRefreshTriggerInfo().getIntervalTrigger().getInterval()); timerDefinition .setIntervalUnit(mtmv.getRefreshInfo().getRefreshTriggerInfo().getIntervalTrigger().getTimeUnit()); - if (mtmv.getRefreshInfo().getBuildMode().equals(BuildMode.IMMEDIATE)) { - jobExecutionConfiguration.setImmediate(true); - } else if (mtmv.getRefreshInfo().getBuildMode().equals(BuildMode.DEFERRED) && !StringUtils + if (!StringUtils .isEmpty(mtmv.getRefreshInfo().getRefreshTriggerInfo().getIntervalTrigger().getStartTime())) { timerDefinition.setStartTimeMs(TimeUtils.timeStringToLong( mtmv.getRefreshInfo().getRefreshTriggerInfo().getIntervalTrigger().getStartTime())); } - + if (mtmv.getRefreshInfo().getBuildMode().equals(BuildMode.IMMEDIATE)) { + jobExecutionConfiguration.setImmediate(true); + } jobExecutionConfiguration.setTimerDefinition(timerDefinition); } diff --git a/regression-test/data/mtmv_p0/test_start_time_mtmv.out b/regression-test/data/mtmv_p0/test_start_time_mtmv.out new file mode 100644 index 00000000000000..8e17dcacec9139 --- /dev/null +++ b/regression-test/data/mtmv_p0/test_start_time_mtmv.out @@ -0,0 +1,17 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !init -- +1 1 +2 2 +3 3 + +-- !create -- +EVERY 2 HOUR STARTS 9999-12-13 21:07:09 + +-- !alter -- +EVERY 2 HOUR STARTS 9998-12-13 21:07:09 + +-- !refresh -- +1 1 +2 2 +3 3 + diff --git a/regression-test/suites/mtmv_p0/test_start_time_mtmv.groovy b/regression-test/suites/mtmv_p0/test_start_time_mtmv.groovy new file mode 100644 index 00000000000000..89872adc253b7b --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_start_time_mtmv.groovy @@ -0,0 +1,76 @@ +// 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_start_time_mtmv","mtmv") { + String suiteName = "test_start_time_mtmv" + String tableName = "${suiteName}_table" + String mvName = "${suiteName}_mv" + + sql """drop table if exists `${tableName}`""" + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE TABLE ${tableName} + ( + k2 INT, + k3 varchar(32) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + insert into ${tableName} values(1,1),(2,2),(3,3); + """ + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD immediate REFRESH AUTO ON SCHEDULE EVERY 2 HOUR STARTS "9999-12-13 21:07:09" + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1' + ) + AS + SELECT * from ${tableName}; + """ + + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_init "SELECT * FROM ${mvName}" + + order_qt_create "select RecurringStrategy from jobs('type'='mv') where MvName='${mvName}'" + + sql """ + alter MATERIALIZED VIEW ${mvName} REFRESH auto ON SCHEDULE EVERY 2 HOUR STARTS "9998-12-13 21:07:09"; + """ + + order_qt_alter "select RecurringStrategy from jobs('type'='mv') where MvName='${mvName}'" + + // refresh mv + sql """ + REFRESH MATERIALIZED VIEW ${mvName} complete + """ + + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh "SELECT * FROM ${mvName}" + + sql """drop table if exists `${tableName}`""" + sql """drop materialized view if exists ${mvName};""" +} From 63b8949d5c5452fcce147295d920a98f6c07d103 Mon Sep 17 00:00:00 2001 From: zhangdong <493738387@qq.com> Date: Thu, 29 Aug 2024 15:55:06 +0800 Subject: [PATCH 19/60] [enhance](mtmv) Prohibit creating materialized views in external catalogs (#39557) --- .../trees/plans/commands/info/CreateMTMVInfo.java | 4 ++++ regression-test/suites/mtmv_p0/test_hive_mtmv.groovy | 10 ++++++++++ 2 files changed, 14 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index 67aaae7fb87057..6e1a4b8d6ec8f0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -38,6 +38,7 @@ import org.apache.doris.common.FeNameFormat; import org.apache.doris.common.util.DynamicPartitionUtil; import org.apache.doris.common.util.PropertyAnalyzer; +import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.mtmv.MTMVPartitionInfo; import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; import org.apache.doris.mtmv.MTMVPartitionUtil; @@ -151,6 +152,9 @@ public CreateMTMVInfo(boolean ifNotExists, TableNameInfo mvName, public void analyze(ConnectContext ctx) throws Exception { // analyze table name mvName.analyze(ctx); + if (!InternalCatalog.INTERNAL_CATALOG_NAME.equals(mvName.getCtl())) { + throw new AnalysisException("Only support creating asynchronous materialized views in internal catalog"); + } try { FeNameFormat.checkTableName(mvName.getTbl()); } catch (org.apache.doris.common.AnalysisException e) { diff --git a/regression-test/suites/mtmv_p0/test_hive_mtmv.groovy b/regression-test/suites/mtmv_p0/test_hive_mtmv.groovy index 4ac5ad9e890463..7b3abc2300f383 100644 --- a/regression-test/suites/mtmv_p0/test_hive_mtmv.groovy +++ b/regression-test/suites/mtmv_p0/test_hive_mtmv.groovy @@ -38,6 +38,16 @@ suite("test_hive_mtmv", "p0,external,hive,external_docker,external_docker_hive") def dbName = "regression_test_mtmv_p0" sql """drop materialized view if exists ${mvName};""" + test { + sql """CREATE MATERIALIZED VIEW ${catalog_name}.`default`.${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${catalog_name}.`default`.mtmv_base1;""" + exception "internal" + } + sql """ CREATE MATERIALIZED VIEW ${mvName} BUILD DEFERRED REFRESH AUTO ON MANUAL From 4514d0f52b220303cedad0bcdc3b3fef7961f705 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Thu, 29 Aug 2024 13:40:07 +0530 Subject: [PATCH 20/60] [Enhancement]use awaitility.await() add in variant and schema testcase (#38836) close #37817 use awaitility in variant schema_change testcases. --- .../test_agg_keys_schema_change_datev2.groovy | 99 ++++------- ...est_schema_change_varchar_to_datev2.groovy | 26 ++- ...st_agg_keys_schema_change_decimalv2.groovy | 156 ++++++------------ ...st_agg_keys_schema_change_decimalv3.groovy | 83 ++++------ .../test_varchar_schema_change.groovy | 51 ++---- .../compaction/test_compaction.groovy | 9 +- .../compaction_sparse_column.groovy | 9 +- 7 files changed, 152 insertions(+), 281 deletions(-) diff --git a/regression-test/suites/schema_change_p0/datev2/test_agg_keys_schema_change_datev2.groovy b/regression-test/suites/schema_change_p0/datev2/test_agg_keys_schema_change_datev2.groovy index 956df9371cbf10..3a92de35b8f967 100644 --- a/regression-test/suites/schema_change_p0/datev2/test_agg_keys_schema_change_datev2.groovy +++ b/regression-test/suites/schema_change_p0/datev2/test_agg_keys_schema_change_datev2.groovy @@ -16,6 +16,8 @@ // under the License. import org.codehaus.groovy.runtime.IOGroovyMethods +import java.util.concurrent.TimeUnit +import org.awaitility.Awaitility suite("test_agg_keys_schema_change_datev2") { def tbName = "test_agg_keys_schema_change_datev2" @@ -49,9 +51,7 @@ suite("test_agg_keys_schema_change_datev2") { // wait for all compactions done for (String[] tablet in tablets) { - boolean running = true - do { - Thread.sleep(100) + Awaitility.await().untilAsserted(() -> { String tablet_id = tablet[0] backend_id = tablet[2] (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) @@ -59,8 +59,8 @@ suite("test_agg_keys_schema_change_datev2") { assertEquals(code, 0) def compactionStatus = parseJson(out.trim()) assertEquals("success", compactionStatus.status.toLowerCase()) - running = compactionStatus.run_status - } while (running) + return compactionStatus.run_status; + }); } } @@ -93,19 +93,15 @@ suite("test_agg_keys_schema_change_datev2") { qt_sql """select * from ${tbName} ORDER BY `datek1`;""" sql """ alter table ${tbName} add column `datev3` datev2 DEFAULT '2022-01-01' """ - int max_try_time = 1000 - while (max_try_time--){ + int max_try_secs = 300 + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(100) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); + sql """sync""" qt_sql """select * from ${tbName} ORDER BY `datek1`;""" do_compact(tbName) @@ -115,19 +111,13 @@ suite("test_agg_keys_schema_change_datev2") { sql """sync""" qt_sql """select * from ${tbName} ORDER BY `datek1`;""" sql """ alter table ${tbName} drop column `datev3` """ - max_try_time = 1000 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(100) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); // datetimev2(0) sql """ insert into ${tbName} values('2022-01-02', '2022-01-02 11:11:11', '2022-01-02', '2022-01-02 11:11:11');""" @@ -144,19 +134,13 @@ suite("test_agg_keys_schema_change_datev2") { sql """sync""" qt_sql """select * from ${tbName} ORDER BY `datek1`;""" sql """ alter table ${tbName} add column `datev3` datetimev2 DEFAULT '2022-01-01 11:11:11' """ - max_try_time = 1000 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(100) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); sql """sync""" qt_sql """select * from ${tbName} ORDER BY `datek1`;""" do_compact(tbName) @@ -166,19 +150,13 @@ suite("test_agg_keys_schema_change_datev2") { sql """sync""" qt_sql """select * from ${tbName} ORDER BY `datek1`;""" sql """ alter table ${tbName} drop column `datev3` """ - max_try_time = 1000 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(100) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); // datetimev2(3) sql """ insert into ${tbName} values('2022-01-02', '2022-01-02 11:11:11', '2022-01-02', '2022-01-02 11:11:11');""" @@ -195,19 +173,15 @@ suite("test_agg_keys_schema_change_datev2") { sql """sync""" qt_sql """select * from ${tbName} ORDER BY `datek1`;""" sql """ alter table ${tbName} add column `datev3` datetimev2(3) DEFAULT '2022-01-01 11:11:11.111' """ - max_try_time = 1000 - while (max_try_time--){ + + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(100) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); + sql """sync""" qt_sql """select * from ${tbName} ORDER BY `datek1`;""" do_compact(tbName) @@ -225,19 +199,14 @@ suite("test_agg_keys_schema_change_datev2") { sql """sync""" qt_sql """select * from ${tbName} ORDER BY `datek1`;""" sql """ alter table ${tbName} drop column `datev3` """ - max_try_time = 1000 - while (max_try_time--){ + + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(100) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); sql """ DROP TABLE ${tbName} force""" } diff --git a/regression-test/suites/schema_change_p0/datev2/test_schema_change_varchar_to_datev2.groovy b/regression-test/suites/schema_change_p0/datev2/test_schema_change_varchar_to_datev2.groovy index 89874a8679d4ad..d01d8cd54f3747 100644 --- a/regression-test/suites/schema_change_p0/datev2/test_schema_change_varchar_to_datev2.groovy +++ b/regression-test/suites/schema_change_p0/datev2/test_schema_change_varchar_to_datev2.groovy @@ -16,6 +16,8 @@ // under the License. import org.codehaus.groovy.runtime.IOGroovyMethods +import java.util.concurrent.TimeUnit +import org.awaitility.Awaitility suite("test_schema_change_varchar_to_datev2") { def tbName = "test_schema_change_varchar_to_datev2" @@ -48,9 +50,7 @@ suite("test_schema_change_varchar_to_datev2") { // wait for all compactions done for (String[] tablet in tablets) { - boolean running = true - do { - Thread.sleep(100) + Awaitility.await().untilAsserted(() -> { String tablet_id = tablet[0] backend_id = tablet[2] (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) @@ -58,8 +58,8 @@ suite("test_schema_change_varchar_to_datev2") { assertEquals(code, 0) def compactionStatus = parseJson(out.trim()) assertEquals("success", compactionStatus.status.toLowerCase()) - running = compactionStatus.run_status - } while (running) + return compactionStatus.run_status; + }); } } @@ -84,19 +84,15 @@ suite("test_schema_change_varchar_to_datev2") { qt_sql_1 """select * from ${tbName} ORDER BY `k1`;""" sql """ alter table ${tbName} modify column `k3` date; """ - int max_try_time = 1000 - while (max_try_time--){ + int max_try_secs = 300 + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(100) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); + sql """sync""" qt_sql_2 """select * from ${tbName} ORDER BY `k1`;""" diff --git a/regression-test/suites/schema_change_p0/decimalv2/test_agg_keys_schema_change_decimalv2.groovy b/regression-test/suites/schema_change_p0/decimalv2/test_agg_keys_schema_change_decimalv2.groovy index 88798db3c6c552..295a034d8eee90 100644 --- a/regression-test/suites/schema_change_p0/decimalv2/test_agg_keys_schema_change_decimalv2.groovy +++ b/regression-test/suites/schema_change_p0/decimalv2/test_agg_keys_schema_change_decimalv2.groovy @@ -16,6 +16,8 @@ // under the License. import org.codehaus.groovy.runtime.IOGroovyMethods +import java.util.concurrent.TimeUnit +import org.awaitility.Awaitility suite("test_agg_keys_schema_change_decimalv2", "nonConcurrent") { def config_row = sql """ ADMIN SHOW FRONTEND CONFIG LIKE 'disable_decimalv2'; """ @@ -62,9 +64,7 @@ suite("test_agg_keys_schema_change_decimalv2", "nonConcurrent") { // wait for all compactions done for (String[] tablet in tablets) { - boolean running = true - do { - Thread.sleep(100) + Awaitility.await().untilAsserted(() -> { String tablet_id = tablet[0] backend_id = tablet[2] (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) @@ -72,8 +72,8 @@ suite("test_agg_keys_schema_change_decimalv2", "nonConcurrent") { assertEquals(code, 0) def compactionStatus = parseJson(out.trim()) assertEquals("success", compactionStatus.status.toLowerCase()) - running = compactionStatus.run_status - } while (running) + return compactionStatus.run_status; + }); } } @@ -98,19 +98,15 @@ suite("test_agg_keys_schema_change_decimalv2", "nonConcurrent") { qt_sql1 """select * from ${tbName} ORDER BY 1,2,3,4;""" sql """ alter table ${tbName} add column `decimalv2v3` decimalv2(27,9) """ - int max_try_time = 1000 - while (max_try_time--){ + int max_try_secs = 300 + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(500, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(1000) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); + sql """sync""" qt_sql2 """select * from ${tbName} ORDER BY 1,2,3,4;""" do_compact(tbName) @@ -118,112 +114,81 @@ suite("test_agg_keys_schema_change_decimalv2", "nonConcurrent") { qt_sql3 """select * from ${tbName} ORDER BY 1,2,3,4;""" sql """ alter table ${tbName} drop column `decimalv2v3` """ - max_try_time = 1000 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(500, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(1000) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); sql """sync""" qt_sql4 """select * from ${tbName} ORDER BY 1,2,3,4;""" // DECIMALV2(21,3) -> decimalv3 OK sql """ alter table ${tbName} modify column decimalv2k2 DECIMALV3(21,3) key """ - max_try_time = 1000 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(500, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(1000) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); sql """sync""" qt_sql5 """select * from ${tbName} ORDER BY 1,2,3,4;""" // DECIMALV2(21,3) -> decimalv3 OK sql """ alter table ${tbName} modify column decimalv2k3 DECIMALV3(38,10) key """ - max_try_time = 1000 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(500, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(1000) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); sql """sync""" qt_sql5_2 """select * from ${tbName} ORDER BY 1,2,3,4;""" // DECIMALV2(27,9) -> decimalv3, round scale part, not overflow sql """ alter table ${tbName} modify column decimalv2v1 DECIMALV3(26,8) sum """ - max_try_time = 1000 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(500, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(1000) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); + sql """sync""" qt_sql6 """select * from ${tbName} ORDER BY 1,2,3,4;""" // DECIMALV2(21,3) -> decimalv3, round scale part, overflow sql """ alter table ${tbName} modify column decimalv2v2 DECIMALV3(20,2) sum """ - max_try_time = 1000 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(500, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "CANCELLED") { - break - } else { - sleep(1000) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); + sql """sync""" qt_sql7 """select * from ${tbName} ORDER BY 1,2,3,4;""" // DECIMALV2(21,3) -> decimalv3, narrow integral, overflow sql """ alter table ${tbName} modify column decimalv2v2 DECIMALV3(20,3) sum """ - max_try_time = 1000 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(500, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "CANCELLED") { - break - } else { - sleep(1000) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); + sql """sync""" qt_sql8 """select * from ${tbName} ORDER BY 1,2,3,4;""" @@ -231,57 +196,42 @@ suite("test_agg_keys_schema_change_decimalv2", "nonConcurrent") { // DECIMALV3(21,3) -> decimalv2 OK sql """ alter table ${tbName} modify column decimalv2k2 DECIMALV2(21,3) key """ - max_try_time = 1000 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(500, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(1000) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); + sql """sync""" qt_sql9 """select * from ${tbName} ORDER BY 1,2,3,4;""" // DECIMALV3(26,8) -> decimalv2 sql """ alter table ${tbName} modify column decimalv2v1 DECIMALV2(25,7) sum """ - max_try_time = 1000 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(500, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(1000) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); + sql """sync""" qt_sql9_2 """select * from ${tbName} ORDER BY 1,2,3,4;""" // DECIMALV3(26,8) -> decimalv2, narrow integer sql """ alter table ${tbName} modify column decimalv2v1 DECIMALV2(25,8) sum """ - max_try_time = 1000 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(500, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(1000) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); + sql """sync""" qt_sql9_3 """select * from ${tbName} ORDER BY 1,2,3,4;""" diff --git a/regression-test/suites/schema_change_p0/decimalv3/test_agg_keys_schema_change_decimalv3.groovy b/regression-test/suites/schema_change_p0/decimalv3/test_agg_keys_schema_change_decimalv3.groovy index 41f2a32854a259..fd28d01ed4595f 100644 --- a/regression-test/suites/schema_change_p0/decimalv3/test_agg_keys_schema_change_decimalv3.groovy +++ b/regression-test/suites/schema_change_p0/decimalv3/test_agg_keys_schema_change_decimalv3.groovy @@ -16,6 +16,8 @@ // under the License. import org.codehaus.groovy.runtime.IOGroovyMethods +import java.util.concurrent.TimeUnit +import org.awaitility.Awaitility suite("test_agg_keys_schema_change_decimalv3") { def tbName = "test_agg_keys_schema_change_decimalv3" @@ -49,9 +51,7 @@ suite("test_agg_keys_schema_change_decimalv3") { // wait for all compactions done for (String[] tablet in tablets) { - boolean running = true - do { - Thread.sleep(100) + Awaitility.await().untilAsserted(() -> { String tablet_id = tablet[0] backend_id = tablet[2] (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) @@ -59,8 +59,8 @@ suite("test_agg_keys_schema_change_decimalv3") { assertEquals(code, 0) def compactionStatus = parseJson(out.trim()) assertEquals("success", compactionStatus.status.toLowerCase()) - running = compactionStatus.run_status - } while (running) + return compactionStatus.run_status; + }); } } @@ -82,91 +82,64 @@ suite("test_agg_keys_schema_change_decimalv3") { qt_sql """select * from ${tbName} ORDER BY `decimalv3k1`;""" sql """ alter table ${tbName} add column `decimalv3v3` DECIMALV3(38,4) """ - int max_try_time = 1000 - while (max_try_time--){ + int max_try_secs = 300 + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(1000) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); + sql """sync""" qt_sql """select * from ${tbName} ORDER BY `decimalv3k1`;""" do_compact(tbName) sql """sync""" qt_sql """select * from ${tbName} ORDER BY `decimalv3k1`;""" sql """ alter table ${tbName} drop column `decimalv3v3` """ - max_try_time = 1000 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "FINISHED") { - sleep(3000) - break - } else { - sleep(1000) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); sql """sync""" qt_sql """select * from ${tbName} ORDER BY `decimalv3k1`;""" sql """ alter table ${tbName} modify column decimalv3k2 DECIMALV3(19,3) key """ - max_try_time = 1000 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "CANCELLED") { - sleep(3000) - break - } else { - sleep(1000) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); + sql """sync""" qt_sql """select * from ${tbName} ORDER BY `decimalv3k1`;""" sql """ alter table ${tbName} modify column decimalv3k2 DECIMALV3(38,10) key """ - max_try_time = 1000 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "CANCELLED") { - sleep(3000) - break - } else { - sleep(1000) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); sql """sync""" qt_sql """select * from ${tbName} ORDER BY `decimalv3k1`;""" sql """ alter table ${tbName} modify column decimalv3k2 DECIMALV3(16,3) key """ - max_try_time = 1000 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tbName) if (result == "CANCELLED") { - sleep(3000) - break - } else { - sleep(1000) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); sql """sync""" qt_sql """select * from ${tbName} ORDER BY `decimalv3k1`;""" diff --git a/regression-test/suites/schema_change_p0/test_varchar_schema_change.groovy b/regression-test/suites/schema_change_p0/test_varchar_schema_change.groovy index 38bd996e89de9e..e4006e6e62b27f 100644 --- a/regression-test/suites/schema_change_p0/test_varchar_schema_change.groovy +++ b/regression-test/suites/schema_change_p0/test_varchar_schema_change.groovy @@ -16,6 +16,8 @@ // under the License. import org.codehaus.groovy.runtime.IOGroovyMethods +import java.util.concurrent.TimeUnit +import org.awaitility.Awaitility suite ("test_varchar_schema_change") { def getJobState = { tableName -> @@ -63,19 +65,14 @@ suite ("test_varchar_schema_change") { // } sql """ alter table ${tableName} modify column c2 varchar(30) """ - - int max_try_time = 1200 - while (max_try_time--){ + int max_try_secs = 300 + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tableName) if (result == "FINISHED") { - break - } else { - sleep(100) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); String[][] res = sql """ desc ${tableName} """ logger.info(res[2][1]) @@ -92,19 +89,14 @@ suite ("test_varchar_schema_change") { sql """ insert into ${tableName} values(55,'2019-11-21',21474,'123aa') """ sql """ alter table ${tableName} modify column c2 INT """ - max_try_time = 1200 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tableName) if (result == "CANCELLED" || result == "FINISHED") { assertEquals(result, "CANCELLED") - break - } else { - sleep(100) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); res = sql """ desc ${tableName} """ logger.info(res[2][1]) @@ -127,9 +119,7 @@ suite ("test_varchar_schema_change") { // wait for all compactions done for (String[] tablet in tablets) { - boolean running = true - do { - Thread.sleep(100) + Awaitility.await().untilAsserted(() -> { String tablet_id = tablet[0] backend_id = tablet[2] (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) @@ -137,8 +127,8 @@ suite ("test_varchar_schema_change") { assertEquals(code, 0) def compactionStatus = parseJson(out.trim()) assertEquals("success", compactionStatus.status.toLowerCase()) - running = compactionStatus.run_status - } while (running) + return compactionStatus.run_status; + }); } qt_sc " select * from ${tableName} order by 1,2; " @@ -150,18 +140,13 @@ suite ("test_varchar_schema_change") { modify column c2 varchar(40), modify column c3 varchar(6) DEFAULT '0' """ - max_try_time = 1200 - while (max_try_time--){ + Awaitility.await().atMost(max_try_secs, TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).await().until(() -> { String result = getJobState(tableName) if (result == "FINISHED") { - break - } else { - sleep(100) - if (max_try_time < 1){ - assertEquals(1,2) - } + return true; } - } + return false; + }); res = sql """ desc ${tableName} """ logger.info(res[2][1]) diff --git a/regression-test/suites/variant_p0/compaction/test_compaction.groovy b/regression-test/suites/variant_p0/compaction/test_compaction.groovy index 48d916e38e3909..2499de5712d934 100644 --- a/regression-test/suites/variant_p0/compaction/test_compaction.groovy +++ b/regression-test/suites/variant_p0/compaction/test_compaction.groovy @@ -16,6 +16,7 @@ // under the License. import org.codehaus.groovy.runtime.IOGroovyMethods +import org.awaitility.Awaitility suite("test_compaction_variant") { try { @@ -106,9 +107,7 @@ suite("test_compaction_variant") { // wait for all compactions done for (def tablet in tablets) { - boolean running = true - do { - Thread.sleep(1000) + Awaitility.await().untilAsserted(() -> { 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) @@ -116,8 +115,8 @@ suite("test_compaction_variant") { assertEquals(code, 0) def compactionStatus = parseJson(out.trim()) assertEquals("success", compactionStatus.status.toLowerCase()) - running = compactionStatus.run_status - } while (running) + return compactionStatus.run_status; + }); } int rowCount = 0 diff --git a/regression-test/suites/variant_p0/compaction_sparse_column.groovy b/regression-test/suites/variant_p0/compaction_sparse_column.groovy index 57e6e6ef963c97..0a7ea100373452 100644 --- a/regression-test/suites/variant_p0/compaction_sparse_column.groovy +++ b/regression-test/suites/variant_p0/compaction_sparse_column.groovy @@ -16,6 +16,7 @@ // under the License. import org.codehaus.groovy.runtime.IOGroovyMethods +import org.awaitility.Awaitility suite("test_compaction_sparse_column", "nonConcurrent") { def tableName = "test_compaction" @@ -125,9 +126,7 @@ suite("test_compaction_sparse_column", "nonConcurrent") { // wait for all compactions done for (def tablet in tablets) { - boolean running = true - do { - Thread.sleep(1000) + Awaitility.await().untilAsserted(() -> { 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) @@ -135,8 +134,8 @@ suite("test_compaction_sparse_column", "nonConcurrent") { assertEquals(code, 0) def compactionStatus = parseJson(out.trim()) assertEquals("success", compactionStatus.status.toLowerCase()) - running = compactionStatus.run_status - } while (running) + return compactionStatus.run_status; + }); } int rowCount = 0 From b805f9e8daa497e4912319eeeb3d3f870ac45032 Mon Sep 17 00:00:00 2001 From: feiniaofeiafei <53502832+feiniaofeiafei@users.noreply.github.com> Date: Thu, 29 Aug 2024 16:11:48 +0800 Subject: [PATCH 21/60] [improvement](nereids) support convert_tz in partition prune (#40047) --- .../rules/OneRangePartitionEvaluator.java | 14 +++ .../functions/scalar/ConvertTz.java | 19 +++- .../partition_prune/test_convert_tz.groovy | 99 +++++++++++++++++++ 3 files changed, 131 insertions(+), 1 deletion(-) create mode 100644 regression-test/suites/nereids_rules_p0/partition_prune/test_convert_tz.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java index 9de31fde6d5af9..c3345c346932a1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java @@ -43,6 +43,7 @@ import org.apache.doris.nereids.trees.expressions.Or; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.Monotonic; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ConvertTz; import org.apache.doris.nereids.trees.expressions.functions.scalar.Date; import org.apache.doris.nereids.trees.expressions.functions.scalar.DateTrunc; import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; @@ -638,6 +639,19 @@ public EvaluateRangeResult visitDate(Date date, EvaluateRangeInput context) { return computeMonotonicFunctionRange(result); } + @Override + public EvaluateRangeResult visitConvertTz(ConvertTz convertTz, EvaluateRangeInput context) { + EvaluateRangeResult result = super.visitConvertTz(convertTz, context); + if (!(result.result instanceof ConvertTz)) { + return result; + } + Expression converTzChild = convertTz.child(0); + if (partitionSlotContainsNull.containsKey(converTzChild)) { + partitionSlotContainsNull.put(convertTz, true); + } + return computeMonotonicFunctionRange(result); + } + private boolean isPartitionSlot(Slot slot) { return slotToType.containsKey(slot); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConvertTz.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConvertTz.java index 75270fa022be2d..11b44d8acee13a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConvertTz.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConvertTz.java @@ -22,6 +22,8 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.Monotonic; +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.StringLikeLiteral; import org.apache.doris.nereids.trees.expressions.shape.TernaryExpression; @@ -39,7 +41,7 @@ * ScalarFunction 'convert_tz'. This class is generated by GenerateFunction. */ public class ConvertTz extends ScalarFunction - implements TernaryExpression, ExplicitlyCastableSignature, AlwaysNullable { + implements TernaryExpression, ExplicitlyCastableSignature, AlwaysNullable, Monotonic { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT) @@ -84,4 +86,19 @@ public List getSignatures() { public R accept(ExpressionVisitor visitor, C context) { return visitor.visitConvertTz(this, context); } + + @Override + public boolean isPositive() { + return true; + } + + @Override + public int getMonotonicFunctionChildIndex() { + return 0; + } + + @Override + public Expression withConstantArgs(Literal literal) { + return new ConvertTz(literal, child(1), child(2)); + } } diff --git a/regression-test/suites/nereids_rules_p0/partition_prune/test_convert_tz.groovy b/regression-test/suites/nereids_rules_p0/partition_prune/test_convert_tz.groovy new file mode 100644 index 00000000000000..c309d10d067194 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/partition_prune/test_convert_tz.groovy @@ -0,0 +1,99 @@ +// 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_convert_tz") { + sql "set enable_fallback_to_original_planner=false" + sql "drop table if exists test_convert_tz;" + sql """CREATE TABLE test_convert_tz + ( + timestamp DATETIME NOT NULL + ) + ENGINE = olap + PARTITION BY range (timestamp) + ( + PARTITION `p1` VALUES LESS THAN ('2021-01-01'), + PARTITION `p2` VALUES LESS THAN ('2021-02-01'), + PARTITION `p3` VALUES LESS THAN ('2021-03-01') + ) DISTRIBUTED BY HASH (timestamp) + PROPERTIES( + "storage_format" = "DEFAULT", + "replication_num" = "1");""" + sql """INSERT INTO test_convert_tz (timestamp) + VALUES ('2020-12-31'), + ('2021-01-05'), + ('2021-01-15'), + ('2021-02-05'), + ('2021-02-15');""" + + explain { + sql "SELECT * FROM test_convert_tz WHERE convert_tz(timestamp, 'Asia/Shanghai', 'Europe/Paris') < '2021-01-01';" + contains("partitions=2/3 (p1,p2)") + } + explain { + sql "SELECT * FROM test_convert_tz WHERE convert_tz(timestamp, 'Asia/Shanghai', 'Europe/Paris') > '2021-01-01';"; + contains("partitions=2/3 (p2,p3)") + } + + explain { + sql """SELECT * FROM test_convert_tz WHERE convert_tz(timestamp, 'Asia/Shanghai', 'Europe/Paris') < '2021-02-24' + and convert_tz(timestamp, 'Asia/Shanghai', 'Europe/Paris') > '2021-01-01';""" + contains("partitions=2/3 (p2,p3)") + } + + explain { + sql """SELECT * FROM test_convert_tz WHERE convert_tz(timestamp, 'Asia/Shanghai', 'Europe/Paris') < '2021-02-24' + or convert_tz(timestamp, 'Asia/Shanghai', 'Europe/Paris') > '2021-01-01';""" + contains("partitions=3/3 (p1,p2,p3)") + } + + explain { + sql "SELECT * FROM test_convert_tz WHERE convert_tz(timestamp, 'Asia/Beijing', 'Europe/Paris') is null;"; + contains("partitions=3/3 (p1,p2,p3)") + } + + explain { + sql "SELECT * FROM test_convert_tz WHERE convert_tz(timestamp, 'Asia/Beijing', 'Europe/Paris') is not null;"; + contains("partitions=3/3 (p1,p2,p3)") + } + + explain { + sql "SELECT * FROM test_convert_tz WHERE date_trunc(convert_tz(timestamp, 'Asia/Beijing', 'Europe/Paris'), 'month') <'2021-01-01';"; + contains("partitions=3/3 (p1,p2,p3)") + } + + explain { + sql "SELECT * FROM test_convert_tz WHERE date_trunc(convert_tz(timestamp, 'Asia/Shanghai', 'Europe/Paris'), 'month') <'2021-01-01';"; + contains("partitions=2/3 (p1,p2)") + } + + explain { + sql "SELECT * FROM test_convert_tz WHERE convert_tz(date_trunc(timestamp, 'month'), 'Asia/Shanghai', 'Europe/Paris') <'2021-01-01';"; + contains("partitions=2/3 (p1,p2)") + } + for (int i = 0; i < 2; i++) { + if (i == 0) { + sql "set disable_nereids_rules = 'REWRITE_FILTER_EXPRESSION'" + } else { + sql "set disable_nereids_rules = ''" + } + explain { + sql "SELECT * FROM test_convert_tz WHERE not convert_tz(timestamp, 'Asia/Shanghai', 'Europe/Paris') <= '2021-01-01';"; + contains("partitions=2/3 (p2,p3)") + } + } +} \ No newline at end of file From 56fca531f6664151505ab2e41577c8927dd23c5d Mon Sep 17 00:00:00 2001 From: Gabriel Date: Thu, 29 Aug 2024 16:51:45 +0800 Subject: [PATCH 22/60] [fix](local shuffle) Fix hash shuffle local exchanger (#40036) --- .../pipeline/exec/aggregation_sink_operator.h | 3 +- be/src/pipeline/exec/analytic_sink_operator.h | 5 +- .../distinct_streaming_aggregation_operator.h | 5 +- be/src/pipeline/exec/hashjoin_build_sink.h | 3 + .../pipeline/exec/hashjoin_probe_operator.h | 3 + be/src/pipeline/exec/operator.h | 6 ++ .../partitioned_aggregation_sink_operator.h | 3 + .../partitioned_hash_join_probe_operator.h | 3 + .../partitioned_hash_join_sink_operator.h | 3 + .../pipeline/exec/set_probe_sink_operator.h | 2 + be/src/pipeline/exec/set_sink_operator.h | 1 + be/src/pipeline/exec/sort_sink_operator.h | 3 +- .../local_exchange_sink_operator.cpp | 14 +++-- .../local_exchange_sink_operator.h | 2 +- .../local_exchange/local_exchanger.cpp | 22 +------ be/src/pipeline/pipeline_fragment_context.cpp | 63 +++++++++++++++---- be/src/pipeline/pipeline_fragment_context.h | 5 +- .../join/test_join_local_shuffle.groovy | 6 +- 18 files changed, 106 insertions(+), 46 deletions(-) diff --git a/be/src/pipeline/exec/aggregation_sink_operator.h b/be/src/pipeline/exec/aggregation_sink_operator.h index 579b9eda1a6e9f..f7b225311a3c06 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.h +++ b/be/src/pipeline/exec/aggregation_sink_operator.h @@ -149,11 +149,12 @@ class AggSinkOperatorX final : public DataSinkOperatorX { ? DataDistribution(ExchangeType::PASSTHROUGH) : DataSinkOperatorX::required_data_distribution(); } - return _is_colocate && _require_bucket_distribution + return _is_colocate && _require_bucket_distribution && !_followed_by_shuffled_join ? DataDistribution(ExchangeType::BUCKET_HASH_SHUFFLE, _partition_exprs) : DataDistribution(ExchangeType::HASH_SHUFFLE, _partition_exprs); } bool require_data_distribution() const override { return _is_colocate; } + bool require_shuffled_data_distribution() const override { return !_probe_expr_ctxs.empty(); } size_t get_revocable_mem_size(RuntimeState* state) const; AggregatedDataVariants* get_agg_data(RuntimeState* state) { diff --git a/be/src/pipeline/exec/analytic_sink_operator.h b/be/src/pipeline/exec/analytic_sink_operator.h index 47080b823803b3..6d713996b9cd72 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.h +++ b/be/src/pipeline/exec/analytic_sink_operator.h @@ -82,7 +82,7 @@ class AnalyticSinkOperatorX final : public DataSinkOperatorXrequire_data_distribution(); } + bool require_shuffled_data_distribution() const override { + return _agg_sink_operator->require_shuffled_data_distribution(); + } Status set_child(OperatorXPtr child) override { RETURN_IF_ERROR(DataSinkOperatorX::set_child(child)); diff --git a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h index 6ee718a3354b27..a63ddb3e69d784 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h +++ b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h @@ -166,6 +166,9 @@ class PartitionedHashJoinProbeOperatorX final _distribution_partition_exprs)); } + bool require_shuffled_data_distribution() const override { + return _join_op != TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN; + } bool is_shuffled_hash_join() const override { return _join_distribution == TJoinDistributionType::PARTITIONED; } diff --git a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h index 1592c29cdb05de..252c53be12d057 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h +++ b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h @@ -116,6 +116,9 @@ class PartitionedHashJoinSinkOperatorX _distribution_partition_exprs); } + bool require_shuffled_data_distribution() const override { + return _join_op != TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN; + } bool is_shuffled_hash_join() const override { return _join_distribution == TJoinDistributionType::PARTITIONED; } diff --git a/be/src/pipeline/exec/set_probe_sink_operator.h b/be/src/pipeline/exec/set_probe_sink_operator.h index 93a862fa1cbe39..f21d58425814c8 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.h +++ b/be/src/pipeline/exec/set_probe_sink_operator.h @@ -98,6 +98,8 @@ class SetProbeSinkOperatorX final : public DataSinkOperatorX create_shared_state() const override { return nullptr; } private: diff --git a/be/src/pipeline/exec/set_sink_operator.h b/be/src/pipeline/exec/set_sink_operator.h index 09a1fa09e7ccbf..ac0757e4467ac0 100644 --- a/be/src/pipeline/exec/set_sink_operator.h +++ b/be/src/pipeline/exec/set_sink_operator.h @@ -95,6 +95,7 @@ class SetSinkOperatorX final : public DataSinkOperatorX diff --git a/be/src/pipeline/exec/sort_sink_operator.h b/be/src/pipeline/exec/sort_sink_operator.h index b842a56f2adc31..3188bfe3990084 100644 --- a/be/src/pipeline/exec/sort_sink_operator.h +++ b/be/src/pipeline/exec/sort_sink_operator.h @@ -64,7 +64,7 @@ class SortSinkOperatorX final : public DataSinkOperatorX { Status sink(RuntimeState* state, vectorized::Block* in_block, bool eos) override; DataDistribution required_data_distribution() const override { if (_is_analytic_sort) { - return _is_colocate && _require_bucket_distribution + return _is_colocate && _require_bucket_distribution && !_followed_by_shuffled_join ? DataDistribution(ExchangeType::BUCKET_HASH_SHUFFLE, _partition_exprs) : DataDistribution(ExchangeType::HASH_SHUFFLE, _partition_exprs); } else if (_merge_by_exchange) { @@ -73,6 +73,7 @@ class SortSinkOperatorX final : public DataSinkOperatorX { } return DataSinkOperatorX::required_data_distribution(); } + bool require_shuffled_data_distribution() const override { return _is_analytic_sort; } bool require_data_distribution() const override { return _is_colocate; } size_t get_revocable_mem_size(RuntimeState* state) const; diff --git a/be/src/pipeline/local_exchange/local_exchange_sink_operator.cpp b/be/src/pipeline/local_exchange/local_exchange_sink_operator.cpp index f0a51696075925..98b1a719a49da7 100644 --- a/be/src/pipeline/local_exchange/local_exchange_sink_operator.cpp +++ b/be/src/pipeline/local_exchange/local_exchange_sink_operator.cpp @@ -36,16 +36,16 @@ std::vector LocalExchangeSinkLocalState::dependencies() const { } Status LocalExchangeSinkOperatorX::init(ExchangeType type, const int num_buckets, - const bool is_shuffled_hash_join, + const bool should_disable_bucket_shuffle, const std::map& shuffle_idx_to_instance_idx) { _name = "LOCAL_EXCHANGE_SINK_OPERATOR (" + get_exchange_type_name(type) + ")"; _type = type; - if (_type == ExchangeType::HASH_SHUFFLE || _type == ExchangeType::BUCKET_HASH_SHUFFLE) { + if (_type == ExchangeType::HASH_SHUFFLE) { // For shuffle join, if data distribution has been broken by previous operator, we // should use a HASH_SHUFFLE local exchanger to shuffle data again. To be mentioned, // we should use map shuffle idx to instance idx because all instances will be // distributed to all BEs. Otherwise, we should use shuffle idx directly. - if (is_shuffled_hash_join) { + if (should_disable_bucket_shuffle) { std::for_each(shuffle_idx_to_instance_idx.begin(), shuffle_idx_to_instance_idx.end(), [&](const auto& item) { DCHECK(item.first != -1); @@ -58,9 +58,11 @@ Status LocalExchangeSinkOperatorX::init(ExchangeType type, const int num_buckets } } _partitioner.reset(new vectorized::Crc32HashPartitioner( - _type == ExchangeType::HASH_SHUFFLE || _bucket_seq_to_instance_idx.empty() - ? _num_partitions - : num_buckets)); + _num_partitions)); + RETURN_IF_ERROR(_partitioner->init(_texprs)); + } else if (_type == ExchangeType::BUCKET_HASH_SHUFFLE) { + _partitioner.reset( + new vectorized::Crc32HashPartitioner(num_buckets)); RETURN_IF_ERROR(_partitioner->init(_texprs)); } return Status::OK(); diff --git a/be/src/pipeline/local_exchange/local_exchange_sink_operator.h b/be/src/pipeline/local_exchange/local_exchange_sink_operator.h index faa48d209f4b1e..e0e7688307c386 100644 --- a/be/src/pipeline/local_exchange/local_exchange_sink_operator.h +++ b/be/src/pipeline/local_exchange/local_exchange_sink_operator.h @@ -102,7 +102,7 @@ class LocalExchangeSinkOperatorX final : public DataSinkOperatorX& shuffle_idx_to_instance_idx) override; Status prepare(RuntimeState* state) override; diff --git a/be/src/pipeline/local_exchange/local_exchanger.cpp b/be/src/pipeline/local_exchange/local_exchanger.cpp index 1bcd9f34ba854c..f4630f328bb70d 100644 --- a/be/src/pipeline/local_exchange/local_exchanger.cpp +++ b/be/src/pipeline/local_exchange/local_exchanger.cpp @@ -239,28 +239,8 @@ Status ShuffleExchanger::_split_rows(RuntimeState* state, const uint32_t* __rest new_block_wrapper->unref(local_state._shared_state, local_state._channel_id); } } - } else if (bucket_seq_to_instance_idx.empty()) { - /** - * If type is `BUCKET_HASH_SHUFFLE` and `_bucket_seq_to_instance_idx` is empty, which - * means no scan operators is included in this fragment so we also need a `HASH_SHUFFLE` here. - */ - const auto& map = local_state._parent->cast() - ._shuffle_idx_to_instance_idx; - DCHECK(!map.empty()); - new_block_wrapper->ref(map.size()); - for (const auto& it : map) { - DCHECK(it.second >= 0 && it.second < _num_partitions) - << it.first << " : " << it.second << " " << _num_partitions; - uint32_t start = local_state._partition_rows_histogram[it.first]; - uint32_t size = local_state._partition_rows_histogram[it.first + 1] - start; - if (size > 0) { - _enqueue_data_and_set_ready(it.second, local_state, - {new_block_wrapper, {row_idx, start, size}}); - } else { - new_block_wrapper->unref(local_state._shared_state, local_state._channel_id); - } - } } else { + DCHECK(!bucket_seq_to_instance_idx.empty()); new_block_wrapper->ref(_num_partitions); for (size_t i = 0; i < _num_partitions; i++) { uint32_t start = local_state._partition_rows_histogram[i]; diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index c3942e8286e8e3..3b94b258d31073 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -616,7 +616,7 @@ Status PipelineFragmentContext::_build_pipelines(ObjectPool* pool, int node_idx = 0; RETURN_IF_ERROR(_create_tree_helper(pool, request.fragment.plan.nodes, request, descs, nullptr, - &node_idx, root, cur_pipe, 0)); + &node_idx, root, cur_pipe, 0, false)); if (node_idx + 1 != request.fragment.plan.nodes.size()) { return Status::InternalError( @@ -630,7 +630,8 @@ Status PipelineFragmentContext::_create_tree_helper(ObjectPool* pool, const doris::TPipelineFragmentParams& request, const DescriptorTbl& descs, OperatorXPtr parent, int* node_idx, OperatorXPtr* root, - PipelinePtr& cur_pipe, int child_idx) { + PipelinePtr& cur_pipe, int child_idx, + const bool followed_by_shuffled_join) { // propagate error case if (*node_idx >= tnodes.size()) { return Status::InternalError( @@ -640,9 +641,11 @@ Status PipelineFragmentContext::_create_tree_helper(ObjectPool* pool, const TPlanNode& tnode = tnodes[*node_idx]; int num_children = tnodes[*node_idx].num_children; + bool current_followed_by_shuffled_join = followed_by_shuffled_join; OperatorXPtr op = nullptr; RETURN_IF_ERROR(_create_operator(pool, tnodes[*node_idx], request, descs, op, cur_pipe, - parent == nullptr ? -1 : parent->node_id(), child_idx)); + parent == nullptr ? -1 : parent->node_id(), child_idx, + followed_by_shuffled_join)); // assert(parent != nullptr || (node_idx == 0 && root_expr != nullptr)); if (parent != nullptr) { @@ -651,12 +654,30 @@ Status PipelineFragmentContext::_create_tree_helper(ObjectPool* pool, } else { *root = op; } + /** + * `ExchangeType::HASH_SHUFFLE` should be used if an operator is followed by a shuffled hash join. + * + * For plan: + * LocalExchange(id=0) -> Aggregation(id=1) -> ShuffledHashJoin(id=2) + * Exchange(id=3) -> ShuffledHashJoinBuild(id=2) + * We must ensure data distribution of `LocalExchange(id=0)` is same as Exchange(id=3). + * + * If an operator's is followed by a local exchange without shuffle (e.g. passthrough), a + * shuffled local exchanger will be used before join so it is not followed by shuffle join. + */ + auto require_shuffled_data_distribution = + cur_pipe->operator_xs().empty() + ? cur_pipe->sink_x()->require_shuffled_data_distribution() + : op->require_shuffled_data_distribution(); + current_followed_by_shuffled_join = + (followed_by_shuffled_join || op->is_shuffled_hash_join()) && + require_shuffled_data_distribution; // rely on that tnodes is preorder of the plan for (int i = 0; i < num_children; i++) { ++*node_idx; RETURN_IF_ERROR(_create_tree_helper(pool, tnodes, request, descs, op, node_idx, nullptr, - cur_pipe, i)); + cur_pipe, i, current_followed_by_shuffled_join)); // we are expecting a child, but have used all nodes // this means we have been given a bad tree and must fail @@ -692,15 +713,30 @@ Status PipelineFragmentContext::_add_local_exchange_impl( // 1. Create a new pipeline with local exchange sink. DataSinkOperatorXPtr sink; auto sink_id = next_sink_operator_id(); - const bool is_shuffled_hash_join = operator_xs.size() > idx - ? operator_xs[idx]->is_shuffled_hash_join() - : cur_pipe->sink_x()->is_shuffled_hash_join(); + + /** + * `bucket_seq_to_instance_idx` is empty if no scan operator is contained in this fragment. + * So co-located operators(e.g. Agg, Analytic) should use `HASH_SHUFFLE` instead of `BUCKET_HASH_SHUFFLE`. + */ + const bool followed_by_shuffled_join = + operator_xs.size() > idx ? operator_xs[idx]->followed_by_shuffled_join() + : cur_pipe->sink_x()->followed_by_shuffled_join(); + const bool should_disable_bucket_shuffle = + bucket_seq_to_instance_idx.empty() && + shuffle_idx_to_instance_idx.find(-1) == shuffle_idx_to_instance_idx.end() && + followed_by_shuffled_join; sink.reset(new LocalExchangeSinkOperatorX( - sink_id, local_exchange_id, is_shuffled_hash_join ? _total_instances : _num_instances, + sink_id, local_exchange_id, + should_disable_bucket_shuffle ? _total_instances : _num_instances, data_distribution.partition_exprs, bucket_seq_to_instance_idx)); + if (should_disable_bucket_shuffle && + data_distribution.distribution_type == ExchangeType::BUCKET_HASH_SHUFFLE) { + data_distribution.distribution_type = ExchangeType::HASH_SHUFFLE; + } RETURN_IF_ERROR(new_pip->set_sink(sink)); RETURN_IF_ERROR(new_pip->sink_x()->init(data_distribution.distribution_type, num_buckets, - is_shuffled_hash_join, shuffle_idx_to_instance_idx)); + should_disable_bucket_shuffle, + shuffle_idx_to_instance_idx)); // 2. Create and initialize LocalExchangeSharedState. std::shared_ptr shared_state = @@ -711,7 +747,7 @@ Status PipelineFragmentContext::_add_local_exchange_impl( case ExchangeType::HASH_SHUFFLE: shared_state->exchanger = ShuffleExchanger::create_unique( std::max(cur_pipe->num_tasks(), _num_instances), - is_shuffled_hash_join ? _total_instances : _num_instances, + should_disable_bucket_shuffle ? _total_instances : _num_instances, _runtime_state->query_options().__isset.local_exchange_free_blocks_limit ? _runtime_state->query_options().local_exchange_free_blocks_limit : 0); @@ -1123,7 +1159,8 @@ Status PipelineFragmentContext::_create_operator(ObjectPool* pool, const TPlanNo const doris::TPipelineFragmentParams& request, const DescriptorTbl& descs, OperatorXPtr& op, PipelinePtr& cur_pipe, int parent_idx, - int child_idx) { + int child_idx, + const bool followed_by_shuffled_join) { // We directly construct the operator from Thrift because the given array is in the order of preorder traversal. // Therefore, here we need to use a stack-like structure. _pipeline_parent_map.pop(cur_pipe, parent_idx, child_idx); @@ -1214,6 +1251,7 @@ Status PipelineFragmentContext::_create_operator(ObjectPool* pool, const TPlanNo !tnode.agg_node.grouping_exprs.empty() && !group_by_limit_opt) { op.reset(new DistinctStreamingAggOperatorX(pool, next_operator_id(), tnode, descs, _require_bucket_distribution)); + op->set_followed_by_shuffled_join(followed_by_shuffled_join); _require_bucket_distribution = _require_bucket_distribution || op->require_data_distribution(); RETURN_IF_ERROR(cur_pipe->add_operator(op)); @@ -1245,6 +1283,7 @@ Status PipelineFragmentContext::_create_operator(ObjectPool* pool, const TPlanNo sink.reset(new AggSinkOperatorX(pool, next_sink_operator_id(), tnode, descs, _require_bucket_distribution)); } + sink->set_followed_by_shuffled_join(followed_by_shuffled_join); _require_bucket_distribution = _require_bucket_distribution || sink->require_data_distribution(); sink->set_dests_id({op->operator_id()}); @@ -1387,6 +1426,7 @@ Status PipelineFragmentContext::_create_operator(ObjectPool* pool, const TPlanNo sink.reset(new SortSinkOperatorX(pool, next_sink_operator_id(), tnode, descs, _require_bucket_distribution)); } + sink->set_followed_by_shuffled_join(followed_by_shuffled_join); _require_bucket_distribution = _require_bucket_distribution || sink->require_data_distribution(); sink->set_dests_id({op->operator_id()}); @@ -1426,6 +1466,7 @@ Status PipelineFragmentContext::_create_operator(ObjectPool* pool, const TPlanNo DataSinkOperatorXPtr sink; sink.reset(new AnalyticSinkOperatorX(pool, next_sink_operator_id(), tnode, descs, _require_bucket_distribution)); + sink->set_followed_by_shuffled_join(followed_by_shuffled_join); _require_bucket_distribution = _require_bucket_distribution || sink->require_data_distribution(); sink->set_dests_id({op->operator_id()}); diff --git a/be/src/pipeline/pipeline_fragment_context.h b/be/src/pipeline/pipeline_fragment_context.h index 7597c3ce9b55d0..06c882674412ee 100644 --- a/be/src/pipeline/pipeline_fragment_context.h +++ b/be/src/pipeline/pipeline_fragment_context.h @@ -145,12 +145,13 @@ class PipelineFragmentContext : public TaskExecutionContext { Status _create_tree_helper(ObjectPool* pool, const std::vector& tnodes, const doris::TPipelineFragmentParams& request, const DescriptorTbl& descs, OperatorXPtr parent, int* node_idx, - OperatorXPtr* root, PipelinePtr& cur_pipe, int child_idx); + OperatorXPtr* root, PipelinePtr& cur_pipe, int child_idx, + const bool followed_by_shuffled_join); Status _create_operator(ObjectPool* pool, const TPlanNode& tnode, const doris::TPipelineFragmentParams& request, const DescriptorTbl& descs, OperatorXPtr& op, PipelinePtr& cur_pipe, - int parent_idx, int child_idx); + int parent_idx, int child_idx, const bool followed_by_shuffled_join); template Status _build_operators_for_set_operation_node(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, OperatorXPtr& op, diff --git a/regression-test/suites/nereids_p0/join/test_join_local_shuffle.groovy b/regression-test/suites/nereids_p0/join/test_join_local_shuffle.groovy index c66131b57dcfc9..29fe192e2b5368 100644 --- a/regression-test/suites/nereids_p0/join/test_join_local_shuffle.groovy +++ b/regression-test/suites/nereids_p0/join/test_join_local_shuffle.groovy @@ -16,6 +16,10 @@ // under the License. suite("test_join_local_shuffle", "query,p0") { + sql "DROP TABLE IF EXISTS test_join_local_shuffle_1;" + sql "DROP TABLE IF EXISTS test_join_local_shuffle_2;" + sql "DROP TABLE IF EXISTS test_join_local_shuffle_3;" + sql "DROP TABLE IF EXISTS test_join_local_shuffle_4;" sql "SET enable_nereids_planner=true" sql "SET enable_fallback_to_original_planner=false" sql """ @@ -72,7 +76,7 @@ suite("test_join_local_shuffle", "query,p0") { sql "insert into test_join_local_shuffle_2 values(2, 0);" sql "insert into test_join_local_shuffle_3 values(2, 0);" sql "insert into test_join_local_shuffle_4 values(0, 1);" - qt_sql " select /*+SET_VAR(disable_join_reorder=true,enable_local_shuffle=true) */ * from (select c1, max(c2) from (select b.c1 c1, b.c2 c2 from test_join_local_shuffle_3 a join [shuffle] test_join_local_shuffle_1 b on a.c2 = b.c1 join [broadcast] test_join_local_shuffle_4 c on b.c1 = c.c1) t1 group by c1) t, test_join_local_shuffle_2 where t.c1 = test_join_local_shuffle_2.c2; " + qt_sql " select /*+SET_VAR(disable_join_reorder=true,enable_local_shuffle=true) */ * from (select c1, max(c2) from (select b.c1 c1, b.c2 c2 from test_join_local_shuffle_3 a join [shuffle] test_join_local_shuffle_1 b on a.c2 = b.c1 join [broadcast] test_join_local_shuffle_4 c on b.c1 = c.c1) t1 group by c1) t join [shuffle] test_join_local_shuffle_2 on t.c1 = test_join_local_shuffle_2.c2; " sql "DROP TABLE IF EXISTS test_join_local_shuffle_1;" sql "DROP TABLE IF EXISTS test_join_local_shuffle_2;" From 80b921344f6b96088b2a81c8768829bcb0cf0aae Mon Sep 17 00:00:00 2001 From: daidai <2017501503@qq.com> Date: Thu, 29 Aug 2024 18:23:40 +0800 Subject: [PATCH 23/60] [fix](parquet)Fix the be core issue when reading parquet unsigned types. (#39926) ## Proposed changes Since Doris does not have an unsigned type, we convert parquet uint32 type to doris bigint (int64) type. When reading the parquet file, the byte size stored in parquet and the byte size of the data type mapped by doris are inconsistent, resulting in be core. Fix: When reading, we read according to the byte size stored in parquet, and then convert it to the data type mapped by doris. Mapping relationship description: parquet -> doris UInt8 -> Int16 UInt16 -> Int32 UInt32 -> Int64 UInt64 -> Int128. --- .../format/parquet/parquet_column_convert.cpp | 18 +- .../format/parquet/parquet_column_convert.h | 65 +++ .../vec/exec/format/parquet/schema_desc.cpp | 58 ++- be/src/vec/exec/format/parquet/schema_desc.h | 10 +- .../test_outfile_expr_generate_col_name.out | 6 +- ...st_local_tvf_parquet_unsigned_integers.out | 440 ++++++++++++++++++ .../tvf/unsigned_integers_1.parquet | Bin 0 -> 3202 bytes .../tvf/unsigned_integers_2.parquet | Bin 0 -> 3218 bytes .../tvf/unsigned_integers_3.parquet | Bin 0 -> 4727 bytes .../tvf/unsigned_integers_4.parquet | Bin 0 -> 704 bytes ...local_tvf_parquet_unsigned_integers.groovy | 102 ++++ 11 files changed, 673 insertions(+), 26 deletions(-) create mode 100644 regression-test/data/external_table_p0/tvf/test_local_tvf_parquet_unsigned_integers.out create mode 100644 regression-test/data/external_table_p0/tvf/unsigned_integers_1.parquet create mode 100644 regression-test/data/external_table_p0/tvf/unsigned_integers_2.parquet create mode 100644 regression-test/data/external_table_p0/tvf/unsigned_integers_3.parquet create mode 100644 regression-test/data/external_table_p0/tvf/unsigned_integers_4.parquet create mode 100644 regression-test/suites/external_table_p0/tvf/test_local_tvf_parquet_unsigned_integers.groovy diff --git a/be/src/vec/exec/format/parquet/parquet_column_convert.cpp b/be/src/vec/exec/format/parquet/parquet_column_convert.cpp index 57f1f54b7b91f5..2fb0afea82ae8a 100644 --- a/be/src/vec/exec/format/parquet/parquet_column_convert.cpp +++ b/be/src/vec/exec/format/parquet/parquet_column_convert.cpp @@ -66,7 +66,9 @@ ColumnPtr PhysicalToLogicalConverter::get_physical_column(tparquet::Type::type s src_physical_type = tparquet::Type::INT32; src_logical_type = TypeDescriptor(PrimitiveType::TYPE_INT); } - if (is_consistent() && _logical_converter->is_consistent()) { + + if (!_convert_params->is_type_compatibility && is_consistent() && + _logical_converter->is_consistent()) { if (_cached_src_physical_type == nullptr) { _cached_src_physical_type = DataTypeFactory::instance().create_data_type( src_logical_type, dst_logical_type->is_nullable()); @@ -246,7 +248,19 @@ std::unique_ptr PhysicalToLogicalConverter::get_conv } PrimitiveType src_logical_primitive = src_logical_type.type; - if (is_parquet_native_type(src_logical_primitive)) { + if (field_schema->is_type_compatibility) { + if (src_logical_type == TYPE_SMALLINT) { + physical_converter.reset(new UnsignedIntegerConverter()); + } else if (src_logical_type == TYPE_INT) { + physical_converter.reset(new UnsignedIntegerConverter()); + } else if (src_logical_type == TYPE_BIGINT) { + physical_converter.reset(new UnsignedIntegerConverter()); + } else if (src_logical_type == TYPE_LARGEINT) { + physical_converter.reset(new UnsignedIntegerConverter()); + } else { + physical_converter.reset(new UnsupportedConverter(src_physical_type, src_logical_type)); + } + } else if (is_parquet_native_type(src_logical_primitive)) { if (is_string_type(src_logical_primitive) && src_physical_type == tparquet::Type::FIXED_LEN_BYTE_ARRAY) { // for FixedSizeBinary diff --git a/be/src/vec/exec/format/parquet/parquet_column_convert.h b/be/src/vec/exec/format/parquet/parquet_column_convert.h index 551bf7e14edbc8..91b81121aa4303 100644 --- a/be/src/vec/exec/format/parquet/parquet_column_convert.h +++ b/be/src/vec/exec/format/parquet/parquet_column_convert.h @@ -40,6 +40,9 @@ struct ConvertParams { DecimalScaleParams decimal_scale; FieldSchema* field_schema = nullptr; + //For UInt8 -> Int16,UInt16 -> Int32,UInt32 -> Int64,UInt64 -> Int128. + bool is_type_compatibility = false; + /** * Some frameworks like paimon maybe writes non-standard parquet files. Timestamp field doesn't have * logicalType or converted_type to indicates its precision. We have to reset the time mask. @@ -108,6 +111,7 @@ struct ConvertParams { t.from_unixtime(0, *ctz); offset_days = t.day() == 31 ? -1 : 0; } + is_type_compatibility = field_schema_->is_type_compatibility; } template @@ -273,6 +277,67 @@ class LittleIntPhysicalConverter : public PhysicalToLogicalConverter { } }; +template +struct UnsignedTypeTraits; + +template <> +struct UnsignedTypeTraits { + using UnsignedCppType = UInt8; + //https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#unsigned-integers + //INT(8, false), INT(16, false), and INT(32, false) must annotate an int32 primitive type and INT(64, false) + //must annotate an int64 primitive type. + using StorageCppType = Int32; + using StorageColumnType = vectorized::ColumnInt32; +}; + +template <> +struct UnsignedTypeTraits { + using UnsignedCppType = UInt16; + using StorageCppType = Int32; + using StorageColumnType = vectorized::ColumnInt32; +}; + +template <> +struct UnsignedTypeTraits { + using UnsignedCppType = UInt32; + using StorageCppType = Int32; + using StorageColumnType = vectorized::ColumnInt32; +}; + +template <> +struct UnsignedTypeTraits { + using UnsignedCppType = UInt64; + using StorageCppType = Int64; + using StorageColumnType = vectorized::ColumnInt64; +}; + +template +class UnsignedIntegerConverter : public PhysicalToLogicalConverter { + Status physical_convert(ColumnPtr& src_physical_col, ColumnPtr& src_logical_column) override { + using UnsignedCppType = typename UnsignedTypeTraits::UnsignedCppType; + using StorageCppType = typename UnsignedTypeTraits::StorageCppType; + using StorageColumnType = typename UnsignedTypeTraits::StorageColumnType; + using DstColumnType = typename PrimitiveTypeTraits::ColumnType; + + ColumnPtr from_col = remove_nullable(src_physical_col); + MutableColumnPtr to_col = remove_nullable(src_logical_column)->assume_mutable(); + auto& src_data = static_cast(from_col.get())->get_data(); + + size_t rows = src_data.size(); + size_t start_idx = to_col->size(); + to_col->resize(start_idx + rows); + auto& data = static_cast(*to_col.get()).get_data(); + + for (int i = 0; i < rows; i++) { + StorageCppType src_value = src_data[i]; + auto unsigned_value = static_cast(src_value); + data[start_idx + i] = unsigned_value; + } + + return Status::OK(); + } +}; + class FixedSizeBinaryConverter : public PhysicalToLogicalConverter { private: int _type_length; diff --git a/be/src/vec/exec/format/parquet/schema_desc.cpp b/be/src/vec/exec/format/parquet/schema_desc.cpp index 08692de8743c06..9097b65718f53d 100644 --- a/be/src/vec/exec/format/parquet/schema_desc.cpp +++ b/be/src/vec/exec/format/parquet/schema_desc.cpp @@ -191,16 +191,19 @@ void FieldDescriptor::parse_physical_field(const tparquet::SchemaElement& physic physical_field->physical_type = physical_schema.type; _physical_fields.push_back(physical_field); physical_field->physical_column_index = _physical_fields.size() - 1; - physical_field->type = get_doris_type(physical_schema); + auto type = get_doris_type(physical_schema); + physical_field->type = type.first; + physical_field->is_type_compatibility = type.second; } -TypeDescriptor FieldDescriptor::get_doris_type(const tparquet::SchemaElement& physical_schema) { - TypeDescriptor type; - type.type = INVALID_TYPE; +std::pair FieldDescriptor::get_doris_type( + const tparquet::SchemaElement& physical_schema) { + std::pair ans = {INVALID_TYPE, false}; + TypeDescriptor& type = ans.first; if (physical_schema.__isset.logicalType) { - type = convert_to_doris_type(physical_schema.logicalType); + ans = convert_to_doris_type(physical_schema.logicalType); } else if (physical_schema.__isset.converted_type) { - type = convert_to_doris_type(physical_schema); + ans = convert_to_doris_type(physical_schema); } // use physical type instead if (type.type == INVALID_TYPE) { @@ -233,7 +236,7 @@ TypeDescriptor FieldDescriptor::get_doris_type(const tparquet::SchemaElement& ph break; } } - return type; + return ans; } // Copy from org.apache.iceberg.avro.AvroSchemaUtil#validAvroName @@ -302,8 +305,11 @@ void FieldDescriptor::iceberg_sanitize(const std::vector& read_colu } } -TypeDescriptor FieldDescriptor::convert_to_doris_type(tparquet::LogicalType logicalType) { - TypeDescriptor type; +std::pair FieldDescriptor::convert_to_doris_type( + tparquet::LogicalType logicalType) { + std::pair ans = {INVALID_TYPE, false}; + TypeDescriptor& type = ans.first; + bool& is_type_compatibility = ans.second; if (logicalType.__isset.STRING) { type = TypeDescriptor(TYPE_STRING); } else if (logicalType.__isset.DECIMAL) { @@ -313,16 +319,25 @@ TypeDescriptor FieldDescriptor::convert_to_doris_type(tparquet::LogicalType logi type = TypeDescriptor(TYPE_DATEV2); } else if (logicalType.__isset.INTEGER) { if (logicalType.INTEGER.isSigned) { - if (logicalType.INTEGER.bitWidth <= 32) { + if (logicalType.INTEGER.bitWidth <= 8) { + type = TypeDescriptor(TYPE_TINYINT); + } else if (logicalType.INTEGER.bitWidth <= 16) { + type = TypeDescriptor(TYPE_SMALLINT); + } else if (logicalType.INTEGER.bitWidth <= 32) { type = TypeDescriptor(TYPE_INT); } else { type = TypeDescriptor(TYPE_BIGINT); } } else { - if (logicalType.INTEGER.bitWidth <= 16) { + is_type_compatibility = true; + if (logicalType.INTEGER.bitWidth <= 8) { + type = TypeDescriptor(TYPE_SMALLINT); + } else if (logicalType.INTEGER.bitWidth <= 16) { type = TypeDescriptor(TYPE_INT); - } else { + } else if (logicalType.INTEGER.bitWidth <= 32) { type = TypeDescriptor(TYPE_BIGINT); + } else { + type = TypeDescriptor(TYPE_LARGEINT); } } } else if (logicalType.__isset.TIME) { @@ -344,12 +359,14 @@ TypeDescriptor FieldDescriptor::convert_to_doris_type(tparquet::LogicalType logi } else { type = TypeDescriptor(INVALID_TYPE); } - return type; + return ans; } -TypeDescriptor FieldDescriptor::convert_to_doris_type( +std::pair FieldDescriptor::convert_to_doris_type( const tparquet::SchemaElement& physical_schema) { - TypeDescriptor type; + std::pair ans = {INVALID_TYPE, false}; + TypeDescriptor& type = ans.first; + bool& is_type_compatibility = ans.second; switch (physical_schema.converted_type) { case tparquet::ConvertedType::type::UTF8: type = TypeDescriptor(TYPE_STRING); @@ -378,28 +395,33 @@ TypeDescriptor FieldDescriptor::convert_to_doris_type( type = TypeDescriptor(TYPE_TINYINT); break; case tparquet::ConvertedType::type::UINT_8: + is_type_compatibility = true; [[fallthrough]]; case tparquet::ConvertedType::type::INT_16: type = TypeDescriptor(TYPE_SMALLINT); break; case tparquet::ConvertedType::type::UINT_16: + is_type_compatibility = true; [[fallthrough]]; case tparquet::ConvertedType::type::INT_32: type = TypeDescriptor(TYPE_INT); break; case tparquet::ConvertedType::type::UINT_32: - [[fallthrough]]; - case tparquet::ConvertedType::type::UINT_64: + is_type_compatibility = true; [[fallthrough]]; case tparquet::ConvertedType::type::INT_64: type = TypeDescriptor(TYPE_BIGINT); break; + case tparquet::ConvertedType::type::UINT_64: + is_type_compatibility = true; + type = TypeDescriptor(TYPE_LARGEINT); + break; default: LOG(WARNING) << "Not supported parquet ConvertedType: " << physical_schema.converted_type; type = TypeDescriptor(INVALID_TYPE); break; } - return type; + return ans; } Status FieldDescriptor::parse_group_field(const std::vector& t_schemas, diff --git a/be/src/vec/exec/format/parquet/schema_desc.h b/be/src/vec/exec/format/parquet/schema_desc.h index 50e526bd7300a2..ca726ef1b57590 100644 --- a/be/src/vec/exec/format/parquet/schema_desc.h +++ b/be/src/vec/exec/format/parquet/schema_desc.h @@ -49,6 +49,9 @@ struct FieldSchema { int16_t repeated_parent_def_level = 0; std::vector children; + //For UInt8 -> Int16,UInt16 -> Int32,UInt32 -> Int64,UInt64 -> Int128. + bool is_type_compatibility = false; + FieldSchema() = default; ~FieldSchema() = default; FieldSchema(const FieldSchema& fieldSchema) = default; @@ -84,12 +87,13 @@ class FieldDescriptor { Status parse_node_field(const std::vector& t_schemas, size_t curr_pos, FieldSchema* node_field); - TypeDescriptor convert_to_doris_type(tparquet::LogicalType logicalType); + std::pair convert_to_doris_type(tparquet::LogicalType logicalType); - TypeDescriptor convert_to_doris_type(const tparquet::SchemaElement& physical_schema); + std::pair convert_to_doris_type( + const tparquet::SchemaElement& physical_schema); public: - TypeDescriptor get_doris_type(const tparquet::SchemaElement& physical_schema); + std::pair get_doris_type(const tparquet::SchemaElement& physical_schema); // org.apache.iceberg.avro.AvroSchemaUtil#sanitize will encode special characters, // we have to decode these characters diff --git a/regression-test/data/export_p0/outfile/outfile_expr/test_outfile_expr_generate_col_name.out b/regression-test/data/export_p0/outfile/outfile_expr/test_outfile_expr_generate_col_name.out index 406bc7660ffd68..5748cedd2289c5 100644 --- a/regression-test/data/export_p0/outfile/outfile_expr/test_outfile_expr_generate_col_name.out +++ b/regression-test/data/export_p0/outfile/outfile_expr/test_outfile_expr_generate_col_name.out @@ -100,12 +100,12 @@ id int Yes false \N NONE 9 1 string 27 false 5 true 1 -- !desc_s3 -- -__add_5 int Yes false \N NONE -__bit_or_7 int Yes false \N NONE +__add_5 smallint Yes false \N NONE +__bit_or_7 tinyint Yes false \N NONE __cast_3 bigint Yes false \N NONE __greater_than_4 boolean Yes false \N NONE __in_predicate_6 boolean Yes false \N NONE -__literal_1 int Yes false \N NONE +__literal_1 tinyint Yes false \N NONE __literal_2 text Yes false \N NONE id int Yes false \N NONE diff --git a/regression-test/data/external_table_p0/tvf/test_local_tvf_parquet_unsigned_integers.out b/regression-test/data/external_table_p0/tvf/test_local_tvf_parquet_unsigned_integers.out new file mode 100644 index 00000000000000..4cda2746a00b37 --- /dev/null +++ b/regression-test/data/external_table_p0/tvf/test_local_tvf_parquet_unsigned_integers.out @@ -0,0 +1,440 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !test_1 -- +0 254 54979 3876010132 12292188520939801104 +1 194 41087 2468877118 11276751473207154134 +2 204 40618 3112417582 12531644712494408881 +3 252 42885 2959241092 12524633273560617315 +4 168 34898 3576407414 13096188234563161622 +5 252 39188 3127662355 16817880057727309281 +6 146 45695 2457180674 13258241842726882717 +7 223 54096 3818049848 13392705476834798352 +8 247 49103 4235043353 14882039847048390015 +9 171 37711 4121304297 12690575475523254418 +10 208 45522 2849886325 9887016810088790216 +11 140 57808 4249890046 11248669065541052626 +12 154 63353 2494834510 10992021804074820099 +13 169 62034 3442632722 13741258842709060503 +14 201 39884 2544963866 13079528300040741505 +15 208 48887 2865890129 15763071531082484590 +16 200 58596 2907876388 13993629507485604147 +17 210 50277 3777672898 9348401130412952441 +18 212 51965 3618857131 16174811648348878672 +19 161 39818 2368776725 15511547001336881917 +20 160 41750 2980821608 11023149785626200227 +21 141 57340 4284126112 15552326510762976216 +22 139 58391 4117149789 15878585387857525976 +23 212 59593 2459376464 15173162289502083954 +24 165 53957 3427228999 17619046273898992624 +25 229 49948 2600007878 15455673680657024177 +26 134 64599 3738904119 16329279862825806683 +27 205 41073 2442068534 14375087414295500534 +28 204 60681 3422219729 18264453353631472102 +29 136 47361 3588730082 13238158010948418606 +30 252 42761 4175633691 15734976010511914446 +31 150 45199 4078820644 15090555688768508964 +32 210 57428 4009891336 16805687728956091249 +33 235 34379 3963852622 13291314989308428602 +34 226 33850 3643451160 15146799101548124057 +35 133 54253 2734990478 12652028700805136484 +36 166 61697 3958278249 9924263812189602832 +37 137 55610 2782254392 18017977508881163787 +38 166 41727 3012921589 12285785983985756200 +39 194 46987 2351267776 18191890018150428322 +40 201 61160 3877183539 17825711874965469887 +41 209 54646 3739634163 12447543494188025837 +42 157 65086 2250645811 15257822118590463416 +43 223 56812 4101782180 9308776360130037320 +44 135 61061 2806848998 15345311951702932353 +45 147 39862 3521569945 17679765098696012133 +46 229 38351 2299955463 11589155413771055105 +47 141 49268 2740241757 16646921996087125692 +48 144 53152 3733884127 12946127273932789697 +49 200 48517 4024062219 15103440093398422258 + +-- !test_2 -- +id int Yes false \N NONE +uint8_column smallint Yes false \N NONE +uint16_column int Yes false \N NONE +uint32_column bigint Yes false \N NONE +uint64_column largeint Yes false \N NONE + +-- !test_3 -- +id int Yes false \N NONE +uint8_column smallint Yes false \N NONE +uint16_column int Yes false \N NONE +uint32_column bigint Yes false \N NONE +uint64_column largeint Yes false \N NONE + +-- !test_4 -- +id int Yes false \N NONE +uint8_column smallint Yes false \N NONE +uint16_column int Yes false \N NONE +uint32_column bigint Yes false \N NONE +uint64_column largeint Yes false \N NONE + +-- !test_5 -- +0 200 48438 3775964178 9957179629640951554 +1 197 37330 2548711656 11127988488306710308 +2 200 65117 3419624570 11652168873218179696 +3 223 39174 3669818470 17675778338328486831 +4 196 60825 3064802389 18435946529637287047 +5 148 58165 3414732886 10219427643252484377 +6 217 61318 4075782121 10508828167797335037 +7 185 46040 2798591473 16764958584953558745 +8 148 50700 2541953946 11228894632595899400 +9 235 41262 3398155979 14622381114650660637 +10 189 49552 3754066331 16077913453789464999 +11 138 41102 2488033055 11226600460306403745 +12 132 62435 3745892606 17429590483490742063 +13 164 34858 3380810793 15447251686769296941 +14 246 44518 4213847006 13168159509128150529 +15 128 34475 2584892403 14470296482506793549 +16 180 40732 2667143993 13453719924509014943 +17 206 49824 3079026622 9242535560261271930 +18 250 61860 3723105814 9754643556067415401 +19 248 51724 4229819070 15295668751829551718 +20 237 38710 2951615403 14702842917502593482 +21 133 46707 2464573031 15860122391678970590 +22 249 41930 4210407904 12677893784378713520 +23 218 34250 3657784995 14768434928936445583 +24 134 52064 4274319068 16672757830801563734 +25 179 54232 3587434213 10613547124477746521 +26 217 63838 2348239122 17998346098073086386 +27 190 52362 3442840997 13122160861538572795 +28 131 39957 2321439682 16792774292797573856 +29 143 50902 2914375790 10557924491128546903 +30 228 33365 2438054546 14302876531585763284 +31 251 59126 2308219390 12753986538521770387 +32 197 58579 2647214662 16916351783057256258 +33 246 62028 4202894981 16869494938800942234 +34 229 47800 3963077237 12000510020655611310 +35 195 43221 2980563838 17324881735239531242 +36 210 46389 3302751013 10936691069329977133 +37 190 60582 4294458835 13465474203101539625 +38 234 58068 2575498858 18264769986785189204 +39 244 55282 2242973260 14725356984149511589 +40 224 33882 3169401634 9641824149700693760 +41 210 48824 2805949235 17877575698845246971 +42 250 49871 4015422133 11571502995585068959 +43 253 53173 3315146396 15527276320077174431 +44 205 48824 2153923483 18227485227719579199 +45 202 65101 2286014232 10596463304445669324 +46 144 39272 3664679383 14619550616500589262 +47 207 58788 2459180916 15787272910593406395 +48 236 61302 2814354943 15797302386492157450 +49 222 44025 2267428102 16226182608093468653 + +-- !test_6 -- +0 254 65535 4294967294 18446744073709551614 +1 254 65534 4294967295 18446744073709551615 +2 254 65535 4294967294 18446744073709551615 +3 255 65535 4294967294 18446744073709551615 +4 255 65534 4294967295 18446744073709551615 +5 254 65534 4294967295 18446744073709551614 +6 254 65534 4294967294 18446744073709551614 +7 255 65534 4294967294 18446744073709551614 +8 255 65534 4294967295 18446744073709551614 +9 255 65535 4294967294 18446744073709551614 + +-- !test_7 -- +id bigint Yes false \N NONE +mediumint_unsigned bigint Yes false \N NONE +int_unsigned bigint Yes false \N NONE + +-- !test_8 -- +1 111 111 +2 222 222 + +-- !test_9 -- +16 200 58596 2907876388 13993629507485604147 +49 200 48517 4024062219 15103440093398422258 + +-- !test_10 -- +38 166 41727 3012921589 12285785983985756200 + +-- !test_11 -- +46 229 38351 2299955463 11589155413771055105 + +-- !test_12 -- +49 200 48517 4024062219 15103440093398422258 + +-- !test_13 -- +49 222 44025 2267428102 16226182608093468653 + +-- !test_14 -- +38 234 58068 2575498858 18264769986785189204 + +-- !test_15 -- +14 246 44518 4213847006 13168159509128150529 + +-- !test_16 -- +25 179 54232 3587434213 10613547124477746521 + +-- !test_17 -- +500 + +-- !test_18 -- +500 + +-- !test_19 -- +500 + +-- !test_20 -- +500 + +-- !test_21 -- +500 + +-- !test_22 -- +0 254 65535 4294967294 18446744073709551614 +5 254 65534 4294967295 18446744073709551614 +6 254 65534 4294967294 18446744073709551614 +7 255 65534 4294967294 18446744073709551614 +8 255 65534 4294967295 18446744073709551614 +9 255 65535 4294967294 18446744073709551614 +10 255 65535 4294967294 18446744073709551614 +11 254 65535 4294967294 18446744073709551614 +12 255 65535 4294967294 18446744073709551614 +14 254 65535 4294967295 18446744073709551614 +16 255 65535 4294967295 18446744073709551614 +17 255 65535 4294967294 18446744073709551614 +18 255 65534 4294967295 18446744073709551614 +20 255 65535 4294967294 18446744073709551614 +21 255 65534 4294967294 18446744073709551614 +22 255 65534 4294967294 18446744073709551614 +24 255 65534 4294967294 18446744073709551614 +25 254 65535 4294967295 18446744073709551614 +27 255 65534 4294967294 18446744073709551614 +29 255 65534 4294967295 18446744073709551614 +30 255 65534 4294967295 18446744073709551614 +32 254 65534 4294967294 18446744073709551614 +37 254 65534 4294967295 18446744073709551614 +38 254 65535 4294967294 18446744073709551614 +40 255 65535 4294967294 18446744073709551614 +41 255 65534 4294967294 18446744073709551614 +43 254 65534 4294967295 18446744073709551614 +44 254 65534 4294967294 18446744073709551614 +45 255 65534 4294967295 18446744073709551614 +46 254 65534 4294967295 18446744073709551614 +53 254 65534 4294967294 18446744073709551614 +56 254 65535 4294967294 18446744073709551614 +58 255 65534 4294967295 18446744073709551614 +60 254 65535 4294967295 18446744073709551614 +62 255 65534 4294967295 18446744073709551614 +63 255 65535 4294967294 18446744073709551614 +66 255 65534 4294967295 18446744073709551614 +68 254 65535 4294967295 18446744073709551614 +71 254 65535 4294967295 18446744073709551614 +73 254 65535 4294967295 18446744073709551614 +75 254 65534 4294967294 18446744073709551614 +76 255 65534 4294967294 18446744073709551614 +77 254 65535 4294967295 18446744073709551614 +80 254 65534 4294967295 18446744073709551614 +81 255 65534 4294967294 18446744073709551614 +84 255 65534 4294967295 18446744073709551614 +86 255 65535 4294967295 18446744073709551614 +87 254 65534 4294967295 18446744073709551614 +89 255 65535 4294967294 18446744073709551614 +91 254 65534 4294967294 18446744073709551614 +92 255 65534 4294967294 18446744073709551614 +94 254 65535 4294967294 18446744073709551614 +98 255 65534 4294967295 18446744073709551614 +99 254 65535 4294967294 18446744073709551614 +100 255 65534 4294967295 18446744073709551614 +102 255 65534 4294967295 18446744073709551614 +103 255 65534 4294967295 18446744073709551614 +106 254 65534 4294967295 18446744073709551614 +108 254 65535 4294967295 18446744073709551614 +109 255 65535 4294967294 18446744073709551614 +110 254 65534 4294967295 18446744073709551614 +112 255 65535 4294967295 18446744073709551614 +115 255 65534 4294967295 18446744073709551614 +117 255 65535 4294967294 18446744073709551614 +120 254 65535 4294967294 18446744073709551614 +121 254 65535 4294967295 18446744073709551614 +123 254 65535 4294967294 18446744073709551614 +124 254 65535 4294967295 18446744073709551614 +130 255 65534 4294967294 18446744073709551614 +131 254 65535 4294967295 18446744073709551614 +132 254 65535 4294967294 18446744073709551614 +133 254 65534 4294967295 18446744073709551614 +139 254 65535 4294967295 18446744073709551614 +141 255 65535 4294967295 18446744073709551614 +148 255 65535 4294967294 18446744073709551614 +149 254 65534 4294967295 18446744073709551614 +150 254 65534 4294967295 18446744073709551614 +151 254 65534 4294967294 18446744073709551614 +152 254 65535 4294967295 18446744073709551614 +157 255 65534 4294967295 18446744073709551614 +158 255 65535 4294967294 18446744073709551614 +167 255 65534 4294967295 18446744073709551614 +168 254 65535 4294967294 18446744073709551614 +169 254 65534 4294967294 18446744073709551614 +173 255 65534 4294967294 18446744073709551614 +174 254 65535 4294967295 18446744073709551614 +177 254 65535 4294967295 18446744073709551614 +184 255 65535 4294967295 18446744073709551614 +186 254 65535 4294967295 18446744073709551614 +187 255 65535 4294967295 18446744073709551614 +189 254 65534 4294967295 18446744073709551614 +190 254 65535 4294967295 18446744073709551614 +191 254 65535 4294967294 18446744073709551614 +192 254 65535 4294967294 18446744073709551614 +194 255 65534 4294967294 18446744073709551614 +195 255 65534 4294967295 18446744073709551614 +196 254 65535 4294967295 18446744073709551614 +197 255 65535 4294967295 18446744073709551614 +204 255 65535 4294967295 18446744073709551614 +207 255 65535 4294967295 18446744073709551614 +210 255 65535 4294967294 18446744073709551614 +211 255 65535 4294967294 18446744073709551614 +213 254 65534 4294967295 18446744073709551614 +214 255 65534 4294967295 18446744073709551614 +216 254 65535 4294967295 18446744073709551614 +217 255 65535 4294967295 18446744073709551614 +222 255 65534 4294967295 18446744073709551614 +226 255 65535 4294967294 18446744073709551614 +227 254 65534 4294967294 18446744073709551614 +228 254 65535 4294967295 18446744073709551614 +229 255 65535 4294967294 18446744073709551614 +231 254 65534 4294967294 18446744073709551614 +232 254 65534 4294967294 18446744073709551614 +233 255 65535 4294967295 18446744073709551614 +240 255 65534 4294967295 18446744073709551614 +241 254 65535 4294967294 18446744073709551614 +243 254 65534 4294967294 18446744073709551614 +244 255 65534 4294967295 18446744073709551614 +246 255 65534 4294967294 18446744073709551614 +248 255 65535 4294967295 18446744073709551614 +251 254 65535 4294967294 18446744073709551614 +253 255 65534 4294967294 18446744073709551614 +256 255 65534 4294967294 18446744073709551614 +257 255 65535 4294967294 18446744073709551614 +258 254 65534 4294967294 18446744073709551614 +261 254 65535 4294967295 18446744073709551614 +262 254 65535 4294967295 18446744073709551614 +264 255 65534 4294967294 18446744073709551614 +266 254 65535 4294967295 18446744073709551614 +267 255 65534 4294967294 18446744073709551614 +269 254 65535 4294967294 18446744073709551614 +270 255 65534 4294967294 18446744073709551614 +273 254 65535 4294967294 18446744073709551614 +279 255 65535 4294967295 18446744073709551614 +281 254 65534 4294967294 18446744073709551614 +283 255 65534 4294967295 18446744073709551614 +285 254 65535 4294967294 18446744073709551614 +287 255 65535 4294967294 18446744073709551614 +288 255 65534 4294967294 18446744073709551614 +290 254 65534 4294967295 18446744073709551614 +291 255 65535 4294967295 18446744073709551614 +292 255 65534 4294967295 18446744073709551614 +295 254 65535 4294967294 18446744073709551614 +296 255 65534 4294967295 18446744073709551614 +298 255 65535 4294967294 18446744073709551614 +301 255 65534 4294967295 18446744073709551614 +302 254 65534 4294967294 18446744073709551614 +307 254 65535 4294967294 18446744073709551614 +308 254 65535 4294967294 18446744073709551614 +309 254 65535 4294967295 18446744073709551614 +313 254 65534 4294967295 18446744073709551614 +317 254 65534 4294967294 18446744073709551614 +319 255 65535 4294967295 18446744073709551614 +320 254 65535 4294967295 18446744073709551614 +321 254 65535 4294967295 18446744073709551614 +322 254 65535 4294967295 18446744073709551614 +325 255 65535 4294967295 18446744073709551614 +326 254 65534 4294967295 18446744073709551614 +327 255 65535 4294967295 18446744073709551614 +328 255 65534 4294967294 18446744073709551614 +331 254 65535 4294967294 18446744073709551614 +332 255 65535 4294967294 18446744073709551614 +334 255 65534 4294967294 18446744073709551614 +336 254 65535 4294967295 18446744073709551614 +337 254 65535 4294967295 18446744073709551614 +339 255 65535 4294967295 18446744073709551614 +341 254 65535 4294967294 18446744073709551614 +342 255 65534 4294967294 18446744073709551614 +345 254 65534 4294967294 18446744073709551614 +346 255 65535 4294967294 18446744073709551614 +347 255 65534 4294967295 18446744073709551614 +349 255 65534 4294967295 18446744073709551614 +350 254 65535 4294967295 18446744073709551614 +353 255 65534 4294967294 18446744073709551614 +355 255 65535 4294967294 18446744073709551614 +356 254 65535 4294967295 18446744073709551614 +357 254 65535 4294967295 18446744073709551614 +361 254 65535 4294967295 18446744073709551614 +363 254 65535 4294967295 18446744073709551614 +367 255 65535 4294967294 18446744073709551614 +368 254 65534 4294967295 18446744073709551614 +372 254 65535 4294967294 18446744073709551614 +373 255 65535 4294967294 18446744073709551614 +376 255 65535 4294967294 18446744073709551614 +377 254 65535 4294967295 18446744073709551614 +379 255 65535 4294967294 18446744073709551614 +382 254 65534 4294967295 18446744073709551614 +385 254 65535 4294967294 18446744073709551614 +389 254 65534 4294967294 18446744073709551614 +390 255 65535 4294967295 18446744073709551614 +391 254 65535 4294967294 18446744073709551614 +393 255 65534 4294967295 18446744073709551614 +395 254 65535 4294967294 18446744073709551614 +396 254 65534 4294967294 18446744073709551614 +398 254 65535 4294967294 18446744073709551614 +400 254 65534 4294967294 18446744073709551614 +401 254 65534 4294967294 18446744073709551614 +402 255 65534 4294967295 18446744073709551614 +403 254 65534 4294967295 18446744073709551614 +408 255 65535 4294967294 18446744073709551614 +409 254 65534 4294967295 18446744073709551614 +410 254 65534 4294967294 18446744073709551614 +411 255 65535 4294967294 18446744073709551614 +412 254 65535 4294967294 18446744073709551614 +413 255 65534 4294967294 18446744073709551614 +415 254 65534 4294967294 18446744073709551614 +417 254 65534 4294967295 18446744073709551614 +418 254 65535 4294967294 18446744073709551614 +420 255 65535 4294967294 18446744073709551614 +424 255 65535 4294967294 18446744073709551614 +426 254 65535 4294967295 18446744073709551614 +430 255 65535 4294967294 18446744073709551614 +431 255 65534 4294967295 18446744073709551614 +432 255 65535 4294967294 18446744073709551614 +434 254 65535 4294967294 18446744073709551614 +440 255 65534 4294967294 18446744073709551614 +441 254 65535 4294967294 18446744073709551614 +443 254 65535 4294967295 18446744073709551614 +444 255 65535 4294967295 18446744073709551614 +450 254 65534 4294967294 18446744073709551614 +454 255 65534 4294967294 18446744073709551614 +456 255 65535 4294967294 18446744073709551614 +458 255 65535 4294967295 18446744073709551614 +459 254 65534 4294967294 18446744073709551614 +461 255 65535 4294967294 18446744073709551614 +462 255 65535 4294967295 18446744073709551614 +463 254 65534 4294967294 18446744073709551614 +464 255 65534 4294967295 18446744073709551614 +465 254 65535 4294967294 18446744073709551614 +466 255 65534 4294967295 18446744073709551614 +468 254 65534 4294967295 18446744073709551614 +470 254 65535 4294967294 18446744073709551614 +471 255 65534 4294967294 18446744073709551614 +472 254 65535 4294967295 18446744073709551614 +473 255 65534 4294967295 18446744073709551614 +474 255 65535 4294967295 18446744073709551614 +475 255 65535 4294967294 18446744073709551614 +477 254 65534 4294967294 18446744073709551614 +480 255 65535 4294967295 18446744073709551614 +483 254 65535 4294967294 18446744073709551614 +485 255 65535 4294967294 18446744073709551614 +486 254 65534 4294967294 18446744073709551614 +487 254 65534 4294967295 18446744073709551614 +488 255 65535 4294967295 18446744073709551614 +489 254 65535 4294967295 18446744073709551614 +490 255 65535 4294967294 18446744073709551614 +494 254 65534 4294967294 18446744073709551614 +495 255 65534 4294967295 18446744073709551614 +496 254 65534 4294967295 18446744073709551614 +497 255 65535 4294967294 18446744073709551614 +499 255 65534 4294967295 18446744073709551614 + diff --git a/regression-test/data/external_table_p0/tvf/unsigned_integers_1.parquet b/regression-test/data/external_table_p0/tvf/unsigned_integers_1.parquet new file mode 100644 index 0000000000000000000000000000000000000000..8f87903255ff3b7bcd258740502c280e2c3b2793 GIT binary patch literal 3202 zcmcJS3s6&68pjVw3{kKR%8dzx1zSj2MC2t!>N;M)8ZD22T2y2!5K6GTj0CWcWm(xO ztAc`nvg%MDr~wh`DrymxTG3kNA)wV_H7cn17SN$;_kS+62FvVpI_?>M=bpzs-}%1( z`Eu_?#fG{u0%jGTS<4S#)C?Ix$Z?Z<^#sKP@BkkOzz`q=ra%OSf?>c6n1kWqHSjtR z0}B9iWF)Wz43L0PKni5Q3RnX* zCSrDJ!o1GU<~!!7<2Gva74coQ$uwr7_{R`A!;_6CLa|KBi&HZ^iCS_;An6k*4)YbF z|Fxma8OjreN1YVo0&fM(b{IEgKpHcx` zi84URqbjLso}<|r`_ud>&HikGF4EUa({N08v2)09sg*$!xP@jIQ4|h)p`^JkE-}UX zgXpAqT|#2NrhG{sUvW#MSTB^nwBMjY{wAfrZX+b4gb@8k1h|cmP(*FH4r5;uqB=uJ z!y!U~5n*!`1iehiVyGMX0K)$pLe~;<^(rATM+tfQEg}EHm}D~!1g9_x<0q~Y@(t9M ze@#djPH7?b8G0K(q5ZgGuiQ#PW?mxXJt*#zMaXUJXL29y?+N*N6Aq=Ckn48{No^%$ z5|q5v4j<9qi+vgj2uV4N18l(9!NJ9+Yor2d8D)U@zc-CniC>66oQm(}WvqUiKw=Om zI$l#$m~rYGZTS1tU*m1sIccP0zRUh4`?HdwDpR+PX&o6Be}Gr4%kyr^yCU0^@Z^TP zv`5$2zT=s*>S@qh|5-Ca<9^D@qN*r_->kPU58YX}*XeTE{M~06z3WQNfwHQS{JB3z z?EA1sWnGjvHDc;_#~wc}ep_+jq8BfBb<^#>lZNunR%@FpkL}xS9*QKqyFo>Mx1`&g zzR%er*c`X)PM3FU>(-uS1-lPtIIQrnlXX6v)^X8ewBqwEKA(1X*aU7{ceSI*G+UA# z@y*Q5Kd#8RXclt*?qI1;{g4Xi%0H_93du|Ad$|8reS@V!Z^5Z=wBexYmk1fX@cH9> z6~8;zL_GcQq**0}`ZnfLOSMJD`l>BoEJ~k!V`p^8#`VQXjXNbdC&Y~=hxS~a)%(?H zdCZLsr@DLg)I6&empLVuYk%0E;jlDaU9$9a-s5wDjz8H{9#7xCC-&g@?B?qk^)~~o z*Yl%%b{AzXUt9FiUwh?sU)E+C)}E+rTCyi``K0B;Dvq4@SZAfY7Vc5E_@wR30JoY< zRoX6b#^Ur#PYt^0M#lf_W~Fn*3~Armb60LQ=JvLnJy+B1q_mj#gZ6^$=)E1QoA1r& z%9@z4q`NBX=9n1;pS6W|m9=NF8w|aLQzoBX{kQs#IoE=7K1hG+^67|A@}jD%Pcs){ zRqYYy?JWc3+W7dUg=MSz9-N$T!{^A#N4XPI_N@#1$$HseJe$>j&%SFhv|yLm|BOqx z#er74mznua!||ILJuNZ!ZP)Cw5AQKa-l*0-Y$>Vf(0GO)8q#^BsE+Ss*s=RT zbL^~I_u{5O(n$0?2C3+|)jyg}dRsXo_~o6%{g}Nxjnk>uRFWl9+_O=Xj-u%_*v5mU zl`MWdkqQ?@Yhn^2nPGTwIq>KkufH+if$bm(C?6>r0r$C^20l+5q?i;BocuN29jq{E z$4v@gexUOXa!gL9lM=N|E}QpC-@UK$^`m%|PgBLKd^jlnAE_aECX=WYB1w%EKJ#R9 zsW}@O8XV@+pDEHy{Bg9nv%W0Lh5%aLW7#y;HuWn)>tSZ>T0Hw4m_AInaoSM;A> z(VGSQJ?XfXR!Yv4H*HeGtl!7jgm$O+P2u{qV!0;QVqJ0Zb&Cue?;5lq^usys6Ben% z+;ntqZYq}2X?R3kr43d?`*040vrii=8$>I0SiDZ{AGmn7dr)eW^8J)3de?L<-JiQZ zUWH~F#zLlXPI5u~8akq&qssS_qY^{XgW@MFh;mPNiwdKm`3Ga=IL_(71!x(^!0OHs(8OJZnx~_GLDe*S{>&IiWU$6Z6E~0fCvl+VlV>efRR8Ki~^&<7@!Bn z0+=J?!CQa<5-|@`2|R_XW$B zEIhbk@!fR+#@m(Q2f_{?i@aYJLt`dv;xIWQkl7HCo=hqTS26;LQqnGz^a|O(YqR2i zx1r1w$`eLFoz%kx-U^wE0_K{)jak4yiLbOe3eIEvENBE3ARSx>-+=qzZy*J10(szP zupXQOHJ}%CgNGmq^niM>8Jq-VU?bQ8Zh<~<8+-?9K^y1-O`sqA08}6qTmTQiP4K$X zwosR@mkQUTM$u50h_$_>6&t3g%BR<8JJ&k-sC@!5N>*ystu+*FZXUjkYOF)=Vas%0 zs;e6x9u@cgvf${*_!UvS&YueF2UNjM9U)7hdP+4!+E0iYHhmKDpzaV-2gTi<5b|{?Ay>Y_-o=C* zgrb+533+mrkVTs?hbMFadlW-$cla3wKlAW}R>Q}hBSQ;MNRec-%vqsdzvzrQ(cYG3F4ahdkP zJqOC{HlLPA+I}o+GkA7neEadxkUMEj-#qQjT{^7(MtjA;?K^c>#?AR3spI4OQC%Dw(vX42R>_hjw1{uZ;3QX14dud7|B3S@`w ziJjiNY^r-@pTU?(h3uq>qX<-w{&k=sm2f+E<}7qcu73uK$uHE1Pb)?)r;~#n%zt zl|6m_DM{!4@ODX)DsleI`y*TZScCgPE=g&Z&&6~`iaxE}Wq+#8-0^vlL-f%HUR~vF znGX_9JN;4Kc5K}V7ngvPgKfQ;c}5GPI?Rq|80>DZ++$KW>f*|to23g3{%o84+1md6 zZMp@SLZ$yT zom7T$M)>PHivKa&PS;GQpNC5}iP_YNY~=)+PD5=xR9eX*ZHQE~DmZNUiarIY*Gh1MRcgIs*$g6JyOSh_#IKGGA-H5l`n!#l|b@muJKf{wcQ z#so!qee4lw9}#5zu~m>C4b5pD?zG^Yc8@?yGalZQMm%(b!FcA)*0kYI^}{-U%n#~^ z4#W$kdb?2FQM_IT^{ZYE{J4gF>a|16BGj;l=Ysp!4BDdxxcPj|S466a35;DCzdX(? qI3~s{Fg7-Njbiok*j3@tQ3@+Nb4zne* literal 0 HcmV?d00001 diff --git a/regression-test/data/external_table_p0/tvf/unsigned_integers_3.parquet b/regression-test/data/external_table_p0/tvf/unsigned_integers_3.parquet new file mode 100644 index 0000000000000000000000000000000000000000..ff8dae5ecece1714c6e4dd6c3e27121eff1c9edf GIT binary patch literal 4727 zcmcK8XH*nd1BY>3h=L^&ba7D>qG1I@rPxzs0deWYC}0;*iU>nw9M5vzz(bi~;@ z_TuI?VkHeFr4t>W)VEP$X$29Kp@NUl8bYNi)X)~~&>kJ|F*-sWpMdt1I-xVf&_owV zpoOmJ25odl59mM_pQ0yv;WOw#ir&zN0SwUxM(B%v=nrF3XB z1-`^r7!GsDU;#^5VFau(5^~tU7Iv_Q103N5XSl!>qwqCGV+`Ej4iAjQIE=>xOvEHi z#uRwM3*MNDY4CwB{NRu22*3;kA_&3w1|bN=OoU+;!V!Vln1i|a7T;kW=3@aCVi6*- z7)$UymLdwv5RDit#|o^(Dy+sDtVJx=VLjpyj|6PMMkFE$o3I(lNWm7QA`R)tz*cNS zCbE!?9ONPo`6xgkwxbBeD8UYtVkdTCH};?md$AAuaRB8wh(oBrVN~J>s!)v@)S?bY zaSX>%j}thFQ#g$ua0X{_4(D+J7tw%AXvAe)!Bt$tbu{4ye#B3>iDulwZQQ|K+{4ef zj|X^&M|g}U_yte#48P(ze#7th1ApQ#ynrzITR{Y6sNf^CMjNO?4Q&hCU2nh(0hvU-UzN7{dgnFv9>0 z#2^gD5DdjIe2y>hCBDLNm_r5&Si%Y;V2zQG!v?mngFPJJ2q!qh1+Ex{uQ3{9;0AYi zU@XRAJSJcwCSfwBz!P5Z##Bs$4}9SVe@sUJW*`tj2*x)EK`3S-46_i92+YPD%*D6( z4)ZV{3$PH25Q)WDg72{uQCNm(#9%pAU?o;zHP&D)VzCbE5r=prU;{QH5lPsD%}7QH zwjdR0NJj>?VjD7%g>2*?7kS7>0Sd7lMJPrIcAykHu?xGg2W8lceb|o!D91q@LIn<^ z5=T&lYSf?>bvTM+IF5Rpz)76KY5agQIE!;Qj|;en23$fTF5?QW;u@}_2{-T~e!@*O z;}&k?4({R}e#U(~z(YL3V~CZ+3)IBXYDU6aUR~T;+*(_rB2-9JgxRjCr2p0)2g+5R z3>PU=Xl~O=ZN8F7Rbrm9V4OJKN=35pmX(uuDJzu^ha%o3hiztaK zb@8%J`mUMo2UC~CbTRg{TWwE`T zQ{@{hGNRY_aWC?{a3~`tzW?MyQCgN;mv1ohJ(cNEv2{h_;Na`k8!Wf2-1K?Gqvi`0 z+g2qHUm#NJYL&Tqi)ECC!Pvu@Ytlxp?rF2pDr;?qU4p6K#lu;#+niD?mUSJGy)J81 zrfb&N%Ix(yZiSvT8%N~C<&E1Fa_eGcPJF?n@;i6aX(@0{Xj&Q@r(9;*ikWmuiZ))hC-rw(V|Pl6f*s9&0$Ewj}HHHs>_kO}0C-&*Zol z`#09^$T?Rqxgxr|U1{!xV&Bu*6Y5Iy8g>RZ)o!xencr9z@%VOQ-Ohq52No!|>tVmE z@LEMwr#=&p?%Li|wYrzxX8YYmKh`CfO}~6}ck#{o6w8<%4tq*&oz5JUGx6A-9e2(Z zdev=qC@a0!uq*V=exP$j_=+5_~wZ;yJW|Gdw#idzIghT z7L%w5C$Y#MjC?tmTg^`x9R7THn$z9D^&xRj4Gvfxp4@d#rO`dD{J4oBQ$5d^ zC^s}F%*`8_RQQX#QGJPFUzfD64YZopob(LKSZTJTt+d(bO{n%^et|QGO!5i}2%8bu z5^u3cbEQZc-A8)SQ2QVK`0q?F==ERtw@g%#Nr?7trbk=6{SM3v^$wg+mh{xx-P}i` zDgWuP8K&)f2Oq6%eE!oCj~U~-T&ox+uPV>Bbm{P9KS^hQDix)yDA^STI%JkZbS%tpp^WxGTE`HWH z!8Iq-t8cRWtDrgY*Fr~WntU?3^+EgbIo?N}NxRK1oD?(7Ty$4-JwG90xAm9tkvSct zd(8e3%FOgVp)Q$7b4^}`dfk!thSC*(_Gnjv1)>3>$)|SVaH3lXFM>! zHSD(=Wg7Y?G>5227Z27JtGt|IiYGT{z&jz+MVk2{Y1}~R*#Sbx_jTxfVYLDVDoI+0 zdihSBF-6>#d8j86W^ha4ol&o+X`kCkl^NMm_wMM?lTfMtZsqHzpx2d3T}7n|&A;1w zJ#8w@g-X6Nw6wLxf7maR4?2rK?DYAtf|=8 zWxzwM^zCv-D9Mv)}{=p=2qO*$7+qUv1eN4XXjd z@epApS2Q(5C<+bBEa&hc!tMym5sH0bB~ug?C7CfB3zI2s1C1U1U8i}N4h{8oGYJ#w z#?InWp_(2ET@?y4Utyolu_)RX3No3au<~{b2=lgbnC))rILlK$W~QfbmTide`HJ%c zoGFHL%*960Nkxd)(Mcg3m5&Mb40MTb3>fO~X&PbT=_W+8a^lMVicUNDQ+hj(qN%s? z2p`ar$I1HbHavv;$9W#qx7Y5DhXWY+NYrMAN Qf1THBB{zO+I`fy)-?|Wn`2YX_ literal 0 HcmV?d00001 diff --git a/regression-test/data/external_table_p0/tvf/unsigned_integers_4.parquet b/regression-test/data/external_table_p0/tvf/unsigned_integers_4.parquet new file mode 100644 index 0000000000000000000000000000000000000000..010d15497ec96f1cb3b4ce99183e8ce8b7883ace GIT binary patch literal 704 zcmWG=3^EjD5Y-WN&=F+;GT21fWF$D47$87`gAvLS1JUx`91t#Za#BJ!-Caf7D*pd1_`#}> backends = sql """ show backends """ + def dataFilePath = context.config.dataPath + "/external_table_p0/tvf/" + + assertTrue(backends.size() > 0) + + def be_id = backends[0][0] + // cluster mode need to make sure all be has this data + def outFilePath="/" + def transFile01="${dataFilePath}/unsigned_integers_1.parquet" + def transFile02="${dataFilePath}/unsigned_integers_2.parquet" + def transFile03="${dataFilePath}/unsigned_integers_3.parquet" + def transFile04="${dataFilePath}/unsigned_integers_4.parquet" + + for (List backend : backends) { + def be_host = backend[1] + scpFiles ("root", be_host, transFile01, outFilePath, false); + scpFiles ("root", be_host, transFile02, outFilePath, false); + scpFiles ("root", be_host, transFile03, outFilePath, false); + scpFiles ("root", be_host, transFile04, outFilePath, false); + } + + def file1 = outFilePath + "unsigned_integers_1.parquet"; + def file2 = outFilePath + "unsigned_integers_2.parquet"; + def file3 = outFilePath + "unsigned_integers_3.parquet"; + def file4 = outFilePath + "unsigned_integers_4.parquet"; + + + + + qt_test_1 """ select * from local( "file_path" = "${file1}", "backend_id" = "${be_id}", "format" = "parquet") order by id ;""" + + qt_test_2 """ desc function local( "file_path" = "${file1}", "backend_id" = "${be_id}", "format" = "parquet");""" + + qt_test_3 """ desc function local( "file_path" = "${file2}", "backend_id" = "${be_id}", "format" = "parquet");""" + + qt_test_4 """ desc function local( "file_path" = "${file3}", "backend_id" = "${be_id}", "format" = "parquet");""" + + qt_test_5 """ select * from local( "file_path" = "${file2}", "backend_id" = "${be_id}", "format" = "parquet") order by id ;""" + + qt_test_6 """ select * from local( "file_path" = "${file3}", "backend_id" = "${be_id}", "format" = "parquet") order by id limit 10;""" + + qt_test_7 """ desc function local( "file_path" = "${file4}", "backend_id" = "${be_id}", "format" = "parquet");""" + + qt_test_8 """ select * from local( "file_path" = "${file4}", "backend_id" = "${be_id}", "format" = "parquet") order by id ;""" + + + + qt_test_9 """ select * from local( "file_path" = "${file1}", "backend_id" = "${be_id}", "format" = "parquet") where uint8_column = 200 order by id ;""" + + qt_test_10 """ select * from local( "file_path" = "${file1}", "backend_id" = "${be_id}", "format" = "parquet") where uint16_column = 41727 order by id ;""" + + qt_test_11 """ select * from local( "file_path" = "${file1}", "backend_id" = "${be_id}", "format" = "parquet") where uint32_column = 2299955463 order by id ;""" + + qt_test_12 """ select * from local( "file_path" = "${file1}", "backend_id" = "${be_id}", "format" = "parquet") where uint64_column = 15103440093398422258 order by id ;""" + + + + qt_test_13 """ select * from local( "file_path" = "${file2}", "backend_id" = "${be_id}", "format" = "parquet") where uint8_column = 222 order by id ;""" + + qt_test_14 """ select * from local( "file_path" = "${file2}", "backend_id" = "${be_id}", "format" = "parquet") where uint16_column = 58068 order by id ;""" + + qt_test_15 """ select * from local( "file_path" = "${file2}", "backend_id" = "${be_id}", "format" = "parquet") where uint32_column = 4213847006 order by id ;""" + + qt_test_16 """ select * from local( "file_path" = "${file2}", "backend_id" = "${be_id}", "format" = "parquet") where uint64_column = 10613547124477746521 order by id ;""" + + + qt_test_17 """ select count(id) from local( "file_path" = "${file3}", "backend_id" = "${be_id}", "format" = "parquet") ;""" + + qt_test_18 """ select count(uint8_column) from local( "file_path" = "${file3}", "backend_id" = "${be_id}", "format" = "parquet") ;""" + + qt_test_19 """ select count(uint16_column) from local( "file_path" = "${file3}", "backend_id" = "${be_id}", "format" = "parquet") ;""" + + qt_test_20 """ select count(uint32_column) from local( "file_path" = "${file3}", "backend_id" = "${be_id}", "format" = "parquet") ;""" + + qt_test_21 """ select count(uint64_column) from local( "file_path" = "${file3}", "backend_id" = "${be_id}", "format" = "parquet") ;""" + + qt_test_22 """ select * from local( "file_path" = "${file3}", "backend_id" = "${be_id}", "format" = "parquet") where uint64_column = 18446744073709551614 order by id ;""" + + + +} From 561866873a11435fac6224afe584deb6937a2d0b Mon Sep 17 00:00:00 2001 From: zzzxl <33418555+zzzxl1993@users.noreply.github.com> Date: Thu, 29 Aug 2024 20:06:16 +0800 Subject: [PATCH 24/60] [opt](inverted index) Controls whether the in_list can execute fast_execute. (#40022) 1. compound inlist is allowed to execute fast_execute by configuring in_list_value_count_threshold --- be/src/olap/rowset/segment_v2/segment_iterator.cpp | 13 ++++++++++--- be/src/vec/exprs/vexpr.cpp | 3 +++ be/src/vec/exprs/vexpr.h | 1 + be/src/vec/exprs/vin_predicate.cpp | 2 ++ .../java/org/apache/doris/qe/SessionVariable.java | 10 ++++++++++ gensrc/thrift/PaloInternalService.thrift | 2 ++ .../test_index_inlist_fault_injection.out | 6 ++++++ .../test_index_inlist_fault_injection.groovy | 6 ++++++ 8 files changed, 40 insertions(+), 3 deletions(-) diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp index c79621647414c7..e4398abdb3d331 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -946,10 +946,17 @@ bool SegmentIterator::_check_apply_by_inverted_index(ColumnPredicate* pred, bool return false; } - if ((pred->type() == PredicateType::IN_LIST || pred->type() == PredicateType::NOT_IN_LIST) && - pred->predicate_params()->marked_by_runtime_filter) { + if (pred->type() == PredicateType::IN_LIST || pred->type() == PredicateType::NOT_IN_LIST) { + auto predicate_param = pred->predicate_params(); // in_list or not_in_list predicate produced by runtime filter - return false; + if (predicate_param->marked_by_runtime_filter) { + return false; + } + // the in_list or not_in_list value count cannot be greater than threshold + int32_t threshold = _opts.runtime_state->query_options().in_list_value_count_threshold; + if (pred_in_compound && predicate_param->values.size() > threshold) { + return false; + } } // UNTOKENIZED strings exceed ignore_above, they are written as null, causing range query errors diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp index b66c8aa80a7e6f..5cb0607411d469 100644 --- a/be/src/vec/exprs/vexpr.cpp +++ b/be/src/vec/exprs/vexpr.cpp @@ -652,6 +652,9 @@ std::string VExpr::gen_predicate_result_sign(Block& block, const ColumnNumbers& pred_result_sign += BeConsts::BLOCK_TEMP_COLUMN_PREFIX + column_name + "_" + function_name + "_"; if (function_name == "in" || function_name == "not_in") { + if (arguments.size() - 1 > _in_list_value_count_threshold) { + return pred_result_sign; + } // Generating 'result_sign' from 'inlist' requires sorting the values. std::set values; for (size_t i = 1; i < arguments.size(); i++) { diff --git a/be/src/vec/exprs/vexpr.h b/be/src/vec/exprs/vexpr.h index 0409327aef1dc0..110748efab3f9e 100644 --- a/be/src/vec/exprs/vexpr.h +++ b/be/src/vec/exprs/vexpr.h @@ -329,6 +329,7 @@ class VExpr { uint32_t _index_unique_id = 0; bool _can_fast_execute = false; bool _enable_inverted_index_query = true; + uint32_t _in_list_value_count_threshold = 10; }; } // namespace vectorized diff --git a/be/src/vec/exprs/vin_predicate.cpp b/be/src/vec/exprs/vin_predicate.cpp index 4affec791a4b0f..4d518f9f923184 100644 --- a/be/src/vec/exprs/vin_predicate.cpp +++ b/be/src/vec/exprs/vin_predicate.cpp @@ -28,6 +28,7 @@ #include #include "common/status.h" +#include "runtime/runtime_state.h" #include "vec/core/block.h" #include "vec/core/column_numbers.h" #include "vec/core/column_with_type_and_name.h" @@ -79,6 +80,7 @@ Status VInPredicate::prepare(RuntimeState* state, const RowDescriptor& desc, VExpr::register_function_context(state, context); _prepare_finished = true; _can_fast_execute = can_fast_execute(); + _in_list_value_count_threshold = state->query_options().in_list_value_count_threshold; return Status::OK(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 8af95511233101..4140835b657004 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -652,6 +652,8 @@ public class SessionVariable implements Serializable, Writable { public static final String ENABLE_MATCH_WITHOUT_INVERTED_INDEX = "enable_match_without_inverted_index"; public static final String ENABLE_FALLBACK_ON_MISSING_INVERTED_INDEX = "enable_fallback_on_missing_inverted_index"; + public static final String IN_LIST_VALUE_COUNT_THRESHOLD = "in_list_value_count_threshold"; + /** * If set false, user couldn't submit analyze SQL and FE won't allocate any related resources. */ @@ -2113,6 +2115,13 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { }) public boolean enableFallbackOnMissingInvertedIndex = true; + @VariableMgr.VarAttr(name = IN_LIST_VALUE_COUNT_THRESHOLD, description = { + "in条件value数量大于这个threshold后将不会走fast_execute", + "When the number of values in the IN condition exceeds this threshold," + + " fast_execute will not be used." + }) + public int inListValueCountThreshold = 10; + public void setEnableEsParallelScroll(boolean enableESParallelScroll) { this.enableESParallelScroll = enableESParallelScroll; } @@ -3691,6 +3700,7 @@ public TQueryOptions toThrift() { tResult.setKeepCarriageReturn(keepCarriageReturn); tResult.setEnableSegmentCache(enableSegmentCache); + tResult.setInListValueCountThreshold(inListValueCountThreshold); return tResult; } diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index 3b0b0944a58fa1..85e4ade4ca4adc 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -333,6 +333,8 @@ struct TQueryOptions { 126: optional i32 runtime_bloom_filter_max_size = 16777216; + 127: optional i32 in_list_value_count_threshold = 10; + // For cloud, to control if the content would be written into file cache // In write path, to control if the content would be written into file cache. // In read path, read from file cache or remote storage when execute query. diff --git a/regression-test/data/fault_injection_p0/test_index_inlist_fault_injection.out b/regression-test/data/fault_injection_p0/test_index_inlist_fault_injection.out index 528b4008084f7e..8409a168a0043a 100644 --- a/regression-test/data/fault_injection_p0/test_index_inlist_fault_injection.out +++ b/regression-test/data/fault_injection_p0/test_index_inlist_fault_injection.out @@ -65,3 +65,9 @@ -- !sql -- 2 +-- !sql -- +852 + +-- !sql -- +852 + diff --git a/regression-test/suites/fault_injection_p0/test_index_inlist_fault_injection.groovy b/regression-test/suites/fault_injection_p0/test_index_inlist_fault_injection.groovy index 8d22c001ed0692..a9b3d5135212d4 100644 --- a/regression-test/suites/fault_injection_p0/test_index_inlist_fault_injection.groovy +++ b/regression-test/suites/fault_injection_p0/test_index_inlist_fault_injection.groovy @@ -117,6 +117,12 @@ suite("test_index_inlist_fault_injection", "nonConcurrent") { qt_sql """ select count() from ${indexTbName} where (clientip = '2.1.0.0' or clientip = NULL and clientip = '40.135.0.0'); """ sql """ set enable_common_expr_pushdown = true; """ + + sql """ set in_list_value_count_threshold = 0; """ + qt_sql """ select count() from ${indexTbName} where (clientip in ('40.135.0.0', '232.0.0.0', '26.1.0.0', '247.37.0.0') or status = 200); """ + sql """ set in_list_value_count_threshold = 10; """ + qt_sql """ select count() from ${indexTbName} where (clientip in ('40.135.0.0', '232.0.0.0', '26.1.0.0', '247.37.0.0') or status = 200); """ + } finally { } } finally { From 6fae4724d319fdf5f5dde1a3576ad287f23ebe90 Mon Sep 17 00:00:00 2001 From: yiguolei <676222867@qq.com> Date: Thu, 29 Aug 2024 20:20:32 +0800 Subject: [PATCH 25/60] [enhancement](exception) catch exception for streamload and validate column (#40092) --- be/src/http/action/http_stream.cpp | 58 ++++++++++--------- be/src/http/action/stream_load.cpp | 35 +++++------ be/src/io/file_factory.cpp | 1 + be/src/io/fs/stream_load_pipe.cpp | 4 +- .../runtime/stream_load/stream_load_context.h | 8 ++- be/src/util/byte_buffer.h | 9 +-- be/src/vec/sink/vtablet_block_convertor.cpp | 11 ++-- be/src/vec/sink/vtablet_block_convertor.h | 13 ++++- be/test/util/byte_buffer2_test.cpp | 3 +- 9 files changed, 75 insertions(+), 67 deletions(-) diff --git a/be/src/http/action/http_stream.cpp b/be/src/http/action/http_stream.cpp index 7dbae6df731d84..c6176c52815459 100644 --- a/be/src/http/action/http_stream.cpp +++ b/be/src/http/action/http_stream.cpp @@ -237,37 +237,39 @@ void HttpStreamAction::on_chunk_data(HttpRequest* req) { SCOPED_ATTACH_TASK(ExecEnv::GetInstance()->stream_load_pipe_tracker()); int64_t start_read_data_time = MonotonicNanos(); + Status st = ctx->allocate_schema_buffer(); + if (!st.ok()) { + ctx->status = st; + return; + } while (evbuffer_get_length(evbuf) > 0) { - try { - auto bb = ByteBuffer::allocate(128 * 1024); - auto remove_bytes = evbuffer_remove(evbuf, bb->ptr, bb->capacity); - bb->pos = remove_bytes; - bb->flip(); - auto st = ctx->body_sink->append(bb); - // schema_buffer stores 1M of data for parsing column information - // need to determine whether to cache for the first time - if (ctx->is_read_schema) { - if (ctx->schema_buffer()->pos + remove_bytes < config::stream_tvf_buffer_size) { - ctx->schema_buffer()->put_bytes(bb->ptr, remove_bytes); - } else { - LOG(INFO) << "use a portion of data to request fe to obtain column information"; - ctx->is_read_schema = false; - ctx->status = process_put(req, ctx); - } - } - if (!st.ok() && !ctx->status.ok()) { - LOG(WARNING) << "append body content failed. errmsg=" << st << ", " << ctx->brief(); - ctx->status = st; - return; - } - ctx->receive_bytes += remove_bytes; - } catch (const doris::Exception& e) { - if (e.code() == doris::ErrorCode::MEM_ALLOC_FAILED) { - ctx->status = Status::MemoryLimitExceeded( - fmt::format("PreCatch error code:{}, {}, ", e.code(), e.to_string())); + ByteBufferPtr bb; + st = ByteBuffer::allocate(128 * 1024, &bb); + if (!st.ok()) { + ctx->status = st; + return; + } + auto remove_bytes = evbuffer_remove(evbuf, bb->ptr, bb->capacity); + bb->pos = remove_bytes; + bb->flip(); + st = ctx->body_sink->append(bb); + // schema_buffer stores 1M of data for parsing column information + // need to determine whether to cache for the first time + if (ctx->is_read_schema) { + if (ctx->schema_buffer()->pos + remove_bytes < config::stream_tvf_buffer_size) { + ctx->schema_buffer()->put_bytes(bb->ptr, remove_bytes); + } else { + LOG(INFO) << "use a portion of data to request fe to obtain column information"; + ctx->is_read_schema = false; + ctx->status = process_put(req, ctx); } - ctx->status = Status::Error(e.code(), e.to_string()); } + if (!st.ok()) { + LOG(WARNING) << "append body content failed. errmsg=" << st << ", " << ctx->brief(); + ctx->status = st; + return; + } + ctx->receive_bytes += remove_bytes; } // after all the data has been read and it has not reached 1M, it will execute here if (ctx->is_read_schema) { diff --git a/be/src/http/action/stream_load.cpp b/be/src/http/action/stream_load.cpp index d0c5dff2075c6f..1a9420dea637db 100644 --- a/be/src/http/action/stream_load.cpp +++ b/be/src/http/action/stream_load.cpp @@ -344,25 +344,22 @@ void StreamLoadAction::on_chunk_data(HttpRequest* req) { int64_t start_read_data_time = MonotonicNanos(); while (evbuffer_get_length(evbuf) > 0) { - try { - auto bb = ByteBuffer::allocate(128 * 1024); - auto remove_bytes = evbuffer_remove(evbuf, bb->ptr, bb->capacity); - bb->pos = remove_bytes; - bb->flip(); - auto st = ctx->body_sink->append(bb); - if (!st.ok()) { - LOG(WARNING) << "append body content failed. errmsg=" << st << ", " << ctx->brief(); - ctx->status = st; - return; - } - ctx->receive_bytes += remove_bytes; - } catch (const doris::Exception& e) { - if (e.code() == doris::ErrorCode::MEM_ALLOC_FAILED) { - ctx->status = Status::MemoryLimitExceeded( - fmt::format("PreCatch error code:{}, {}, ", e.code(), e.to_string())); - } - ctx->status = Status::Error(e.code(), e.to_string()); - } + ByteBufferPtr bb; + Status st = ByteBuffer::allocate(128 * 1024, &bb); + if (!st.ok()) { + ctx->status = st; + return; + } + auto remove_bytes = evbuffer_remove(evbuf, bb->ptr, bb->capacity); + bb->pos = remove_bytes; + bb->flip(); + st = ctx->body_sink->append(bb); + if (!st.ok()) { + LOG(WARNING) << "append body content failed. errmsg=" << st << ", " << ctx->brief(); + ctx->status = st; + return; + } + ctx->receive_bytes += remove_bytes; } int64_t read_data_time = MonotonicNanos() - start_read_data_time; int64_t last_receive_and_read_data_cost_nanos = ctx->receive_and_read_data_cost_nanos; diff --git a/be/src/io/file_factory.cpp b/be/src/io/file_factory.cpp index f4ce573c53533d..86907886f17933 100644 --- a/be/src/io/file_factory.cpp +++ b/be/src/io/file_factory.cpp @@ -206,6 +206,7 @@ Status FileFactory::create_pipe_reader(const TUniqueId& load_id, io::FileReaderS return Status::InternalError("unknown stream load id: {}", UniqueId(load_id).to_string()); } if (need_schema) { + RETURN_IF_ERROR(stream_load_ctx->allocate_schema_buffer()); // Here, a portion of the data is processed to parse column information auto pipe = std::make_shared( io::kMaxPipeBufferedBytes /* max_buffered_bytes */, 64 * 1024 /* min_chunk_size */, diff --git a/be/src/io/fs/stream_load_pipe.cpp b/be/src/io/fs/stream_load_pipe.cpp index ce91a2e839113c..0dc27e009d08ee 100644 --- a/be/src/io/fs/stream_load_pipe.cpp +++ b/be/src/io/fs/stream_load_pipe.cpp @@ -113,7 +113,7 @@ Status StreamLoadPipe::read_one_message(std::unique_ptr* data, size_t Status StreamLoadPipe::append_and_flush(const char* data, size_t size, size_t proto_byte_size) { SCOPED_ATTACH_TASK(ExecEnv::GetInstance()->stream_load_pipe_tracker()); ByteBufferPtr buf; - RETURN_IF_ERROR_OR_CATCH_EXCEPTION(ByteBuffer::create_and_allocate(buf, 128 * 1024)); + RETURN_IF_ERROR(ByteBuffer::allocate(128 * 1024, &buf)); buf->put_bytes(data, size); buf->flip(); return _append(buf, proto_byte_size); @@ -148,7 +148,7 @@ Status StreamLoadPipe::append(const char* data, size_t size) { size_t chunk_size = std::max(_min_chunk_size, size - pos); chunk_size = BitUtil::RoundUpToPowerOfTwo(chunk_size); SCOPED_ATTACH_TASK(ExecEnv::GetInstance()->stream_load_pipe_tracker()); - RETURN_IF_ERROR_OR_CATCH_EXCEPTION(ByteBuffer::create_and_allocate(_write_buf, chunk_size)); + RETURN_IF_ERROR(ByteBuffer::allocate(chunk_size, &_write_buf)); _write_buf->put_bytes(data + pos, size - pos); return Status::OK(); } diff --git a/be/src/runtime/stream_load/stream_load_context.h b/be/src/runtime/stream_load/stream_load_context.h index 95e56e0b3faf4b..9d1601372f877d 100644 --- a/be/src/runtime/stream_load/stream_load_context.h +++ b/be/src/runtime/stream_load/stream_load_context.h @@ -121,15 +121,17 @@ class StreamLoadContext { bool is_mow_table() const; - ByteBufferPtr schema_buffer() { + Status allocate_schema_buffer() { if (_schema_buffer == nullptr) { SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER( ExecEnv::GetInstance()->stream_load_pipe_tracker()); - _schema_buffer = ByteBuffer::allocate(config::stream_tvf_buffer_size); + return ByteBuffer::allocate(config::stream_tvf_buffer_size, &_schema_buffer); } - return _schema_buffer; + return Status::OK(); } + ByteBufferPtr schema_buffer() { return _schema_buffer; } + public: static const int default_txn_id = -1; // load type, eg: ROUTINE LOAD/MANUAL LOAD diff --git a/be/src/util/byte_buffer.h b/be/src/util/byte_buffer.h index 1499f51c05316b..aafd4506087d76 100644 --- a/be/src/util/byte_buffer.h +++ b/be/src/util/byte_buffer.h @@ -34,13 +34,8 @@ struct ByteBuffer; using ByteBufferPtr = std::shared_ptr; struct ByteBuffer : private Allocator { - static ByteBufferPtr allocate(size_t size) { - ByteBufferPtr ptr(new ByteBuffer(size)); - return ptr; - } - - static Status create_and_allocate(ByteBufferPtr& ptr, size_t size) { - ptr = ByteBufferPtr(new ByteBuffer(size)); + static Status allocate(const size_t size, ByteBufferPtr* ptr) { + RETURN_IF_CATCH_EXCEPTION({ *ptr = ByteBufferPtr(new ByteBuffer(size)); }); return Status::OK(); } diff --git a/be/src/vec/sink/vtablet_block_convertor.cpp b/be/src/vec/sink/vtablet_block_convertor.cpp index feb6633511ea76..617668c035af45 100644 --- a/be/src/vec/sink/vtablet_block_convertor.cpp +++ b/be/src/vec/sink/vtablet_block_convertor.cpp @@ -182,12 +182,11 @@ DecimalType OlapTableBlockConvertor::_get_decimalv3_min_or_max(const TypeDescrip return DecimalType(value); } -Status OlapTableBlockConvertor::_validate_column(RuntimeState* state, const TypeDescriptor& type, - bool is_nullable, vectorized::ColumnPtr column, - size_t slot_index, bool* stop_processing, - fmt::memory_buffer& error_prefix, - const uint32_t row_count, - vectorized::IColumn::Permutation* rows) { +Status OlapTableBlockConvertor::_internal_validate_column( + RuntimeState* state, const TypeDescriptor& type, bool is_nullable, + vectorized::ColumnPtr column, size_t slot_index, bool* stop_processing, + fmt::memory_buffer& error_prefix, const uint32_t row_count, + vectorized::IColumn::Permutation* rows) { DCHECK((rows == nullptr) || (rows->size() == row_count)); fmt::memory_buffer error_msg; auto set_invalid_and_append_error_msg = [&](int row) { diff --git a/be/src/vec/sink/vtablet_block_convertor.h b/be/src/vec/sink/vtablet_block_convertor.h index 0db340ce6c27d4..7f866c38032775 100644 --- a/be/src/vec/sink/vtablet_block_convertor.h +++ b/be/src/vec/sink/vtablet_block_convertor.h @@ -69,7 +69,18 @@ class OlapTableBlockConvertor { Status _validate_column(RuntimeState* state, const TypeDescriptor& type, bool is_nullable, vectorized::ColumnPtr column, size_t slot_index, bool* stop_processing, fmt::memory_buffer& error_prefix, const uint32_t row_count, - vectorized::IColumn::Permutation* rows = nullptr); + vectorized::IColumn::Permutation* rows = nullptr) { + RETURN_IF_CATCH_EXCEPTION({ + return _internal_validate_column(state, type, is_nullable, column, slot_index, + stop_processing, error_prefix, row_count, rows); + }); + } + + Status _internal_validate_column(RuntimeState* state, const TypeDescriptor& type, + bool is_nullable, vectorized::ColumnPtr column, + size_t slot_index, bool* stop_processing, + fmt::memory_buffer& error_prefix, const uint32_t row_count, + vectorized::IColumn::Permutation* rows = nullptr); // make input data valid for OLAP table // return number of invalid/filtered rows. diff --git a/be/test/util/byte_buffer2_test.cpp b/be/test/util/byte_buffer2_test.cpp index 04b62cd5fe8f0e..73c38c9e404340 100644 --- a/be/test/util/byte_buffer2_test.cpp +++ b/be/test/util/byte_buffer2_test.cpp @@ -32,7 +32,8 @@ class ByteBufferTest : public testing::Test { }; TEST_F(ByteBufferTest, normal) { - auto buf = ByteBuffer::allocate(4); + ByteBufferPtr buf; + Status st = ByteBuffer::allocate(4, &buf); EXPECT_EQ(0, buf->pos); EXPECT_EQ(4, buf->limit); EXPECT_EQ(4, buf->capacity); From 41e7c9b7bfbd4ef50f7e6e602ad02d17b2bce799 Mon Sep 17 00:00:00 2001 From: TengJianPing <18241664+jacktengg@users.noreply.github.com> Date: Thu, 29 Aug 2024 20:26:27 +0800 Subject: [PATCH 26/60] [improvement](regression) improve p1 test time by moving spill cases to p2 (#40079) --- .../spill/partitioned_agg_fault_injection.groovy | 4 ++-- .../spill/partitioned_hash_join_fault_injection.groovy | 4 ++-- .../spill/spill_sort_fault_injection.groovy | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) rename regression-test/suites/{tpch_sf0.1_p1 => tpch_sf1_p2}/spill/partitioned_agg_fault_injection.groovy (97%) rename regression-test/suites/{tpch_sf0.1_p1 => tpch_sf1_p2}/spill/partitioned_hash_join_fault_injection.groovy (98%) rename regression-test/suites/{tpch_sf0.1_p1 => tpch_sf1_p2}/spill/spill_sort_fault_injection.groovy (98%) diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/partitioned_agg_fault_injection.groovy b/regression-test/suites/tpch_sf1_p2/spill/partitioned_agg_fault_injection.groovy similarity index 97% rename from regression-test/suites/tpch_sf0.1_p1/spill/partitioned_agg_fault_injection.groovy rename to regression-test/suites/tpch_sf1_p2/spill/partitioned_agg_fault_injection.groovy index 285141d1043f8b..f80b0a987bd134 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/partitioned_agg_fault_injection.groovy +++ b/regression-test/suites/tpch_sf1_p2/spill/partitioned_agg_fault_injection.groovy @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -suite("partitioned_agg_fault_injection", "nonConcurrent") { +suite("partitioned_agg_fault_injection", "p2, nonConcurrent") { multi_sql """ - use regression_test_tpch_unique_sql_zstd_bucket1_p0; + use regression_test_tpch_sf1_p2; set enable_force_spill=true; set min_revocable_mem=1024; """ diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/partitioned_hash_join_fault_injection.groovy b/regression-test/suites/tpch_sf1_p2/spill/partitioned_hash_join_fault_injection.groovy similarity index 98% rename from regression-test/suites/tpch_sf0.1_p1/spill/partitioned_hash_join_fault_injection.groovy rename to regression-test/suites/tpch_sf1_p2/spill/partitioned_hash_join_fault_injection.groovy index 3e0cc930eb9172..b79e584ca97b5f 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/partitioned_hash_join_fault_injection.groovy +++ b/regression-test/suites/tpch_sf1_p2/spill/partitioned_hash_join_fault_injection.groovy @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -suite("partitioned_hash_join_fault_injection", "nonConcurrent") { +suite("partitioned_hash_join_fault_injection", "p2, nonConcurrent") { multi_sql """ - use regression_test_tpch_unique_sql_zstd_bucket1_p0; + use regression_test_tpch_sf1_p2; set enable_force_spill=true; set min_revocable_mem=1024; """ diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/spill_sort_fault_injection.groovy b/regression-test/suites/tpch_sf1_p2/spill/spill_sort_fault_injection.groovy similarity index 98% rename from regression-test/suites/tpch_sf0.1_p1/spill/spill_sort_fault_injection.groovy rename to regression-test/suites/tpch_sf1_p2/spill/spill_sort_fault_injection.groovy index 1613935b5c2bb2..3030b123e45170 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/spill_sort_fault_injection.groovy +++ b/regression-test/suites/tpch_sf1_p2/spill/spill_sort_fault_injection.groovy @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -suite("spill_sort_fault_injection", "nonConcurrent") { +suite("spill_sort_fault_injection", "p2, nonConcurrent") { multi_sql """ - use regression_test_tpch_unique_sql_zstd_bucket1_p0; + use regression_test_tpch_sf1_p2; set enable_force_spill=true; set min_revocable_mem=1024; """ From da8b43ed6e558249942c21b01a691144e6fa5b93 Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Thu, 29 Aug 2024 21:31:20 +0800 Subject: [PATCH 27/60] [opt](Nereids) forbid one step topn with distribute as child (#40066) --- .../properties/ChildrenPropertiesRegulator.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java index 686350ea91d283..9985b9c567f8fe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java @@ -537,11 +537,21 @@ public Boolean visitPhysicalTopN(PhysicalTopN topN, Void context // process must shuffle visit(topN, context); + int sortPhaseNum = jobContext.getCascadesContext().getConnectContext().getSessionVariable().sortPhaseNum; + // if control sort phase, forbid nothing + if (sortPhaseNum == 1 || sortPhaseNum == 2) { + return true; + } // If child is DistributionSpecGather, topN should forbid two-phase topN if (topN.getSortPhase() == SortPhase.LOCAL_SORT && childrenProperties.get(0).getDistributionSpec().equals(DistributionSpecGather.INSTANCE)) { return false; } + // forbid one step topn with distribute as child + if (topN.getSortPhase() == SortPhase.GATHER_SORT + && children.get(0).getPlan() instanceof PhysicalDistribute) { + return false; + } return true; } From 02223465594efef89d6c355afb4f37fffde09246 Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Thu, 29 Aug 2024 21:39:17 +0800 Subject: [PATCH 28/60] [opt](Nereids) support grant, revoke, admin and transaction statements syntax (#39866) --- .../org/apache/doris/nereids/DorisParser.g4 | 112 +++++++++++++++--- .../AdminCancelRebalanceDiskStmt.java | 2 +- .../analysis/AdminCancelRepairTableStmt.java | 2 +- .../doris/analysis/AdminCheckTabletsStmt.java | 2 +- .../doris/analysis/AdminCleanTrashStmt.java | 2 +- .../doris/analysis/AdminCompactTableStmt.java | 2 +- .../doris/analysis/AdminCopyTabletStmt.java | 2 +- .../analysis/AdminRebalanceDiskStmt.java | 2 +- .../doris/analysis/AdminRepairTableStmt.java | 2 +- .../doris/analysis/AdminSetConfigStmt.java | 2 +- .../AdminSetPartitionVersionStmt.java | 2 +- .../analysis/AdminSetReplicaStatusStmt.java | 2 +- .../analysis/AdminSetReplicaVersionStmt.java | 2 +- .../analysis/AdminSetTableStatusStmt.java | 2 +- .../org/apache/doris/analysis/GrantStmt.java | 2 +- .../org/apache/doris/analysis/RevokeStmt.java | 2 +- .../doris/analysis/TransactionBeginStmt.java | 2 +- .../doris/analysis/TransactionCommitStmt.java | 2 +- .../analysis/TransactionRollbackStmt.java | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 20 +--- .../org/apache/doris/qe/StmtExecutor.java | 2 +- 21 files changed, 121 insertions(+), 49 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 5480c2a3215b5d..f79088b73aacd3 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 @@ -40,6 +40,7 @@ statement | DROP (PROCEDURE | PROC) (IF EXISTS)? name=multipartIdentifier #dropProcedure | SHOW PROCEDURE STATUS (LIKE pattern=valueExpression | whereClause)? #showProcedureStatus | SHOW CREATE PROCEDURE name=multipartIdentifier #showCreateProcedure + // FIXME: like should be wildWhere? FRONTEND should not contain FROM backendid | ADMIN? SHOW type=(FRONTEND | BACKEND) CONFIG (LIKE pattern=valueExpression)? (FROM backendId=INTEGER_VALUE)? #showConfig ; @@ -65,6 +66,9 @@ unsupportedStatement | unsupportedDropStatement | unsupportedStatsStatement | unsupportedAlterStatement + | unsupportedGrantRevokeStatement + | unsupportedAdminStatement + | unsupportedTransactionStatement ; materailizedViewStatement @@ -168,6 +172,71 @@ supportedDropStatement : DROP CATALOG RECYCLE BIN WHERE idType=STRING_LITERAL EQ id=INTEGER_VALUE #dropCatalogRecycleBin ; +unsupportedAdminStatement + : ADMIN SHOW REPLICA STATUS FROM baseTableRef wildWhere? #adminShowReplicaStatus + | ADMIN SHOW REPLICA DISTRIBUTION FROM baseTableRef #adminShowReplicaDistribution + | ADMIN SET REPLICA STATUS PROPERTIES LEFT_PAREN propertyItemList RIGHT_PAREN #adminSetReplicaStatus + | ADMIN SET REPLICA VERSION PROPERTIES LEFT_PAREN propertyItemList RIGHT_PAREN #adminSetReplicaVersion + | ADMIN REPAIR TABLE baseTableRef #adminRepairTable + | ADMIN CANCEL REPAIR TABLE baseTableRef #adminCancelRepairTable + | ADMIN COMPACT TABLE baseTableRef wildWhere? #adminCompactTable + | ADMIN SET (FRONTEND | (ALL FRONTENDS)) CONFIG + (LEFT_PAREN propertyItemList RIGHT_PAREN)? ALL? #adminSetFrontendConfig + | ADMIN CHECK tabletList properties=propertyClause? #adminCheckTablets + | ADMIN REBALANCE DISK (ON LEFT_PAREN backends+=STRING_LITERAL + (COMMA backends+=STRING_LITERAL) RIGHT_PAREN)? #adminRebalanceDisk + | ADMIN CANCEL REBALANCE DISK (ON LEFT_PAREN backends+=STRING_LITERAL + (COMMA backends+=STRING_LITERAL) RIGHT_PAREN)? #adminCancelRebalanceDisk + | ADMIN CLEAN TRASH (ON LEFT_PAREN backends+=STRING_LITERAL + (COMMA backends+=STRING_LITERAL) RIGHT_PAREN)? #adminCleanTrash + | ADMIN SET TABLE name=multipartIdentifier + PARTITION VERSION properties=propertyClause? #adminSetPartitionVersion + | ADMIN DIAGNOSE TABLET tabletId=INTEGER_VALUE #adminDiagnoseTablet + | ADMIN SHOW TABLET STORAGE FORMAT VERBOSE? #adminShowTabletStorageFormat + | ADMIN COPY TABLET tabletId=INTEGER_VALUE properties=propertyClause? #adminCopyTablet + | ADMIN SET TABLE name=multipartIdentifier STATUS properties=propertyClause? #adminSetTableStatus + ; + +baseTableRef + : multipartIdentifier optScanParams? tableSnapshot? specifiedPartition? + tabletList? tableAlias sample? relationHint? + ; + +wildWhere + : LIKE STRING_LITERAL + | WHERE expression + ; + +unsupportedTransactionStatement + : BEGIN (WITH LABEL identifier?)? #transactionBegin + | COMMIT WORK? (AND NO? CHAIN)? (NO? RELEASE)? #transcationCommit + | ROLLBACK WORK? (AND NO? CHAIN)? (NO? RELEASE)? #transactionRollback + ; + +unsupportedGrantRevokeStatement + : GRANT privilegeList ON multipartIdentifierOrAsterisk + TO (userIdentify | ROLE STRING_LITERAL) #grantTablePrivilege + | GRANT privilegeList ON + (RESOURCE | CLUSTER | STAGE | STORAGE VAULT | WORKLOAD GROUP) + identifierOrTextOrAsterisk TO (userIdentify | ROLE STRING_LITERAL) #grantResourcePrivilege + | GRANT roles+=STRING_LITERAL (COMMA roles+=STRING_LITERAL)* TO userIdentify #grantRole + | REVOKE privilegeList ON multipartIdentifierOrAsterisk + FROM (userIdentify | ROLE STRING_LITERAL) #grantTablePrivilege + | REVOKE privilegeList ON + (RESOURCE | CLUSTER | STAGE | STORAGE VAULT | WORKLOAD GROUP) + identifierOrTextOrAsterisk FROM (userIdentify | ROLE STRING_LITERAL) #grantResourcePrivilege + | REVOKE roles+=STRING_LITERAL (COMMA roles+=STRING_LITERAL)* FROM userIdentify #grantRole + ; + +privilege + : name=identifier columns=identifierList? + | ALL + ; + +privilegeList + : privilege (COMMA privilege)* + ; + unsupportedAlterStatement : ALTER TABLE tableName=multipartIdentifier alterTableClause (COMMA alterTableClause)* #alterTable @@ -188,14 +257,17 @@ unsupportedAlterStatement | ALTER RESOURCE name=identifierOrText properties=propertyClause? #alterResource | ALTER COLOCATE GROUP name=multipartIdentifier SET LEFT_PAREN propertyItemList RIGHT_PAREN #alterColocateGroup - | ALTER WORKLOAD GROUP name=identifierOrText properties=propertyClause? #alterWorkloadGroup - | ALTER WORKLOAD POLICY name=identifierOrText properties=propertyClause? #alterWorkloadPolicy + | ALTER WORKLOAD GROUP name=identifierOrText + properties=propertyClause? #alterWorkloadGroup + | ALTER WORKLOAD POLICY name=identifierOrText + properties=propertyClause? #alterWorkloadPolicy | ALTER ROUTINE LOAD FOR name=multipartIdentifier properties=propertyClause? (FROM type=identifier LEFT_PAREN propertyItemList RIGHT_PAREN)? #alterRoutineLoad | ALTER SQL_BLOCK_RULE name=identifier properties=propertyClause? #alterSqlBlockRule | ALTER TABLE name=multipartIdentifier SET LEFT_PAREN propertyItemList RIGHT_PAREN #alterTableProperties - | ALTER STORAGE POLICY name=identifierOrText properties=propertyClause #alterStoragePlicy + | ALTER STORAGE POLICY name=identifierOrText + properties=propertyClause #alterStoragePlicy | ALTER USER (IF EXISTS)? grantUserIdentify passwordOption (COMMENT STRING_LITERAL)? #alterUser | ALTER REPOSITORY name=identifier properties=propertyClause? #alterRepository @@ -428,14 +500,14 @@ functionArgument unsupportedSetStatement : SET (optionWithType | optionWithoutType) - (COMMA (optionWithType | optionWithoutType))* #setOptions - | SET identifier AS DEFAULT STORAGE VAULT #setDefaultStorageVault - | SET PROPERTY (FOR user=identifierOrText)? propertyItemList #setUserProperties + (COMMA (optionWithType | optionWithoutType))* #setOptions + | SET identifier AS DEFAULT STORAGE VAULT #setDefaultStorageVault + | SET PROPERTY (FOR user=identifierOrText)? propertyItemList #setUserProperties | SET (GLOBAL | LOCAL | SESSION)? TRANSACTION ( transactionAccessMode | isolationLevel | transactionAccessMode COMMA isolationLevel - | isolationLevel COMMA transactionAccessMode) #setTransaction + | isolationLevel COMMA transactionAccessMode) #setTransaction ; optionWithType @@ -529,7 +601,7 @@ dataDesc (PARTITION partition=identifierList)? (COLUMNS TERMINATED BY comma=STRING_LITERAL)? (LINES TERMINATED BY separator=STRING_LITERAL)? - (FORMAT AS format=identifierOrStringLiteral)? + (FORMAT AS format=identifierOrText)? (columns=identifierList)? (columnsFromPath=colFromPath)? (columnMapping=colMappingList)? @@ -571,19 +643,29 @@ mvPartition | partitionExpr = functionCallExpression ; -identifierOrStringLiteral +identifierOrText : identifier | STRING_LITERAL ; -identifierOrText - : errorCapturingIdentifier +identifierOrTextOrAsterisk + : identifier | STRING_LITERAL - | LEADING_STRING + | ASTERISK + ; + +multipartIdentifierOrAsterisk + : parts+=identifierOrAsterisk (DOT parts+=identifierOrAsterisk)* + ; + +identifierOrAsterisk + : identifierOrText + | ASTERISK ; userIdentify - : user=identifierOrText (ATSIGN (host=identifierOrText | LEFT_PAREN host=identifierOrText RIGHT_PAREN))? + : user=identifierOrText (ATSIGN (host=identifierOrText + | LEFT_PAREN host=identifierOrText RIGHT_PAREN))? ; grantUserIdentify @@ -880,8 +962,8 @@ optScanParams ; relationPrimary - : multipartIdentifier optScanParams? materializedViewName? specifiedPartition? - tabletList? tableAlias sample? tableSnapshot? relationHint? lateralView* #tableName + : multipartIdentifier optScanParams? materializedViewName? tableSnapshot? specifiedPartition? + tabletList? tableAlias sample? relationHint? lateralView* #tableName | LEFT_PAREN query RIGHT_PAREN tableAlias lateralView* #aliasedQuery | tvfName=identifier LEFT_PAREN (properties=propertyItemList)? diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCancelRebalanceDiskStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCancelRebalanceDiskStmt.java index 874a1af1368801..59180d158e0341 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCancelRebalanceDiskStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCancelRebalanceDiskStmt.java @@ -34,7 +34,7 @@ import java.util.List; import java.util.Map; -public class AdminCancelRebalanceDiskStmt extends DdlStmt { +public class AdminCancelRebalanceDiskStmt extends DdlStmt implements NotFallbackInParser { private List backends = Lists.newArrayList(); public AdminCancelRebalanceDiskStmt(List backends) throws UserException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCancelRepairTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCancelRepairTableStmt.java index b7944e38e65b90..e4efa1b58d5c55 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCancelRepairTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCancelRepairTableStmt.java @@ -30,7 +30,7 @@ import java.util.List; -public class AdminCancelRepairTableStmt extends DdlStmt { +public class AdminCancelRepairTableStmt extends DdlStmt implements NotFallbackInParser { private TableRef tblRef; private List partitions = Lists.newArrayList(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCheckTabletsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCheckTabletsStmt.java index cf1d9f5bdf0bd0..56efbdc8a6300e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCheckTabletsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCheckTabletsStmt.java @@ -32,7 +32,7 @@ import java.util.Map; // ADMIN CHECK TABLET (id1, id2, ...) PROPERTIES ("type" = "check_consistency"); -public class AdminCheckTabletsStmt extends DdlStmt { +public class AdminCheckTabletsStmt extends DdlStmt implements NotFallbackInParser { private List tabletIds; private Map properties; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCleanTrashStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCleanTrashStmt.java index b6d5cc2ce1931a..5dd9419376d450 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCleanTrashStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCleanTrashStmt.java @@ -34,7 +34,7 @@ import java.util.List; import java.util.Map; -public class AdminCleanTrashStmt extends DdlStmt { +public class AdminCleanTrashStmt extends DdlStmt implements NotFallbackInParser { private List backends = Lists.newArrayList(); public AdminCleanTrashStmt(List backends) throws UserException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCompactTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCompactTableStmt.java index 6abcae6c05cc6f..5d82a735353aad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCompactTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCompactTableStmt.java @@ -30,7 +30,7 @@ import java.util.List; -public class AdminCompactTableStmt extends DdlStmt { +public class AdminCompactTableStmt extends DdlStmt implements NotFallbackInParser { private TableRef tblRef; private Expr where; 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 475219ca343c53..072ee4bb12de2c 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 @@ -34,7 +34,7 @@ import java.util.Map; // ADMIN COPY TABLET 10110 PROPERTIES('version' = '1000', backend_id = '10001'); -public class AdminCopyTabletStmt extends ShowStmt { +public class AdminCopyTabletStmt extends ShowStmt implements NotFallbackInParser { public static final String PROP_VERSION = "version"; public static final String PROP_BACKEND_ID = "backend_id"; public static final String PROP_EXPIRATION = "expiration_minutes"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminRebalanceDiskStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminRebalanceDiskStmt.java index aa48a9a1fc11f7..7185826343f99a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminRebalanceDiskStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminRebalanceDiskStmt.java @@ -35,7 +35,7 @@ import java.util.List; import java.util.Map; -public class AdminRebalanceDiskStmt extends DdlStmt { +public class AdminRebalanceDiskStmt extends DdlStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(AdminRebalanceDiskStmt.class); private List backends = Lists.newArrayList(); private long timeoutS = 0; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminRepairTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminRepairTableStmt.java index 098517d28d8d19..a87c6e02b65d17 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminRepairTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminRepairTableStmt.java @@ -30,7 +30,7 @@ import java.util.List; -public class AdminRepairTableStmt extends DdlStmt { +public class AdminRepairTableStmt extends DdlStmt implements NotFallbackInParser { private TableRef tblRef; private List partitions = Lists.newArrayList(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetConfigStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetConfigStmt.java index 0de5ee0807d0ee..458116924d7ee5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetConfigStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetConfigStmt.java @@ -33,7 +33,7 @@ import java.util.Map; // admin set frontend config ("key" = "value"); -public class AdminSetConfigStmt extends DdlStmt { +public class AdminSetConfigStmt extends DdlStmt implements NotFallbackInParser { private boolean applyToAll; private NodeType type; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetPartitionVersionStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetPartitionVersionStmt.java index a1edeb7a2c8ef8..63e0fed32c45da 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetPartitionVersionStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetPartitionVersionStmt.java @@ -35,7 +35,7 @@ * "partition_id" = "20010", * "visible_version" = "101" */ -public class AdminSetPartitionVersionStmt extends DdlStmt { +public class AdminSetPartitionVersionStmt extends DdlStmt implements NotFallbackInParser { private long partitionId = -1; private long visibleVersion = -1; private final TableName tableName; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetReplicaStatusStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetReplicaStatusStmt.java index 84288dabd6852e..dcc2f29fca59cb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetReplicaStatusStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetReplicaStatusStmt.java @@ -35,7 +35,7 @@ * "backend_id" = "10001" * "status" = "drop"/"bad"/"ok" */ -public class AdminSetReplicaStatusStmt extends DdlStmt { +public class AdminSetReplicaStatusStmt extends DdlStmt implements NotFallbackInParser { public static final String TABLET_ID = "tablet_id"; public static final String BACKEND_ID = "backend_id"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetReplicaVersionStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetReplicaVersionStmt.java index 82ce9eff7158a0..23b16638477a06 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetReplicaVersionStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetReplicaVersionStmt.java @@ -37,7 +37,7 @@ * "last_success_version" = "100", * "last_failed_version" = "-1", */ -public class AdminSetReplicaVersionStmt extends DdlStmt { +public class AdminSetReplicaVersionStmt extends DdlStmt implements NotFallbackInParser { public static final String TABLET_ID = "tablet_id"; public static final String BACKEND_ID = "backend_id"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetTableStatusStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetTableStatusStmt.java index e58d2af12785ea..8e857f7507094e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetTableStatusStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminSetTableStatusStmt.java @@ -31,7 +31,7 @@ import java.util.Map; -public class AdminSetTableStatusStmt extends DdlStmt { +public class AdminSetTableStatusStmt extends DdlStmt implements NotFallbackInParser { public static final String TABLE_STATE = "state"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/GrantStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/GrantStmt.java index f1b3ab01d43bfc..c87c5daba2357c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/GrantStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/GrantStmt.java @@ -53,7 +53,7 @@ // GRANT privilege[(col1,col2...)] [, privilege] ON db.tbl TO user_identity [ROLE 'role']; // GRANT privilege [, privilege] ON RESOURCE 'resource' TO user_identity [ROLE 'role']; // GRANT role [, role] TO user_identity -public class GrantStmt extends DdlStmt { +public class GrantStmt extends DdlStmt implements NotFallbackInParser { private UserIdentity userIdent; // Indicates which permissions are granted to this role private String role; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/RevokeStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/RevokeStmt.java index 9b06623f82c20f..96c52f7353f4b8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/RevokeStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/RevokeStmt.java @@ -43,7 +43,7 @@ // REVOKE privilege[(col1,col2...)] [, privilege] ON db.tbl FROM user_identity [ROLE 'role']; // REVOKE privilege [, privilege] ON resource 'resource' FROM user_identity [ROLE 'role']; // REVOKE role [, role] FROM user_identity -public class RevokeStmt extends DdlStmt { +public class RevokeStmt extends DdlStmt implements NotFallbackInParser { private UserIdentity userIdent; // Indicates which permissions are revoked from this role private String role; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TransactionBeginStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TransactionBeginStmt.java index 58184b9bb21c1c..90f5fbb5159ece 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/TransactionBeginStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TransactionBeginStmt.java @@ -22,7 +22,7 @@ import org.apache.doris.common.util.DebugUtil; import org.apache.doris.transaction.TransactionEntry; -public class TransactionBeginStmt extends TransactionStmt { +public class TransactionBeginStmt extends TransactionStmt implements NotFallbackInParser { private String label = null; public TransactionBeginStmt() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TransactionCommitStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TransactionCommitStmt.java index cdfe53e9ce7c97..b17e767b683a02 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/TransactionCommitStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TransactionCommitStmt.java @@ -17,6 +17,6 @@ package org.apache.doris.analysis; -public class TransactionCommitStmt extends TransactionStmt { +public class TransactionCommitStmt extends TransactionStmt implements NotFallbackInParser { } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TransactionRollbackStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TransactionRollbackStmt.java index d285ba9fe1b3bc..081cd84592b1e0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/TransactionRollbackStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TransactionRollbackStmt.java @@ -17,6 +17,6 @@ package org.apache.doris.analysis; -public class TransactionRollbackStmt extends TransactionStmt { +public class TransactionRollbackStmt extends TransactionStmt implements NotFallbackInParser { } 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 af7d6cd63affd2..c7d43a696fcdaf 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 @@ -110,7 +110,6 @@ import org.apache.doris.nereids.DorisParser.HintStatementContext; import org.apache.doris.nereids.DorisParser.IdentifierContext; import org.apache.doris.nereids.DorisParser.IdentifierListContext; -import org.apache.doris.nereids.DorisParser.IdentifierOrTextContext; import org.apache.doris.nereids.DorisParser.IdentifierSeqContext; import org.apache.doris.nereids.DorisParser.InPartitionDefContext; import org.apache.doris.nereids.DorisParser.IndexDefContext; @@ -1130,7 +1129,7 @@ public LogicalPlan visitLoad(DorisParser.LoadContext ctx) { : LoadTask.MergeType.valueOf(ddc.mergeType().getText()); Optional fileFormat = ddc.format == null ? Optional.empty() - : Optional.of(visitIdentifierOrStringLiteral(ddc.format)); + : Optional.of(visitIdentifierOrText(ddc.format)); Optional separator = ddc.separator == null ? Optional.empty() : Optional.of(ddc.separator.getText() .substring(1, ddc.separator.getText().length() - 1)); Optional comma = ddc.comma == null ? Optional.empty() : Optional.of(ddc.comma.getText() @@ -1229,16 +1228,7 @@ public Command visitCreateRowPolicy(CreateRowPolicyContext ctx) { } @Override - public String visitIdentifierOrText(IdentifierOrTextContext ctx) { - if (ctx.STRING_LITERAL() != null) { - return ctx.STRING_LITERAL().getText().substring(1, ctx.STRING_LITERAL().getText().length() - 1); - } else { - return ctx.errorCapturingIdentifier().getText(); - } - } - - @Override - public String visitIdentifierOrStringLiteral(DorisParser.IdentifierOrStringLiteralContext ctx) { + public String visitIdentifierOrText(DorisParser.IdentifierOrTextContext ctx) { if (ctx.STRING_LITERAL() != null) { return ctx.STRING_LITERAL().getText().substring(1, ctx.STRING_LITERAL().getText().length() - 1); } else { @@ -3197,7 +3187,7 @@ private LogicalPlan withSelectHint(LogicalPlan logicalPlan, List leadingParameters = new ArrayList(); + List leadingParameters = new ArrayList<>(); for (HintAssignmentContext kv : hintStatement.parameters) { if (kv.key != null) { String parameterName = visitIdentifierOrText(kv.key); @@ -3210,7 +3200,7 @@ private LogicalPlan withSelectHint(LogicalPlan logicalPlan, List useRuleParameters = new ArrayList(); + List useRuleParameters = new ArrayList<>(); for (HintAssignmentContext kv : hintStatement.parameters) { if (kv.key != null) { String parameterName = visitIdentifierOrText(kv.key); @@ -3220,7 +3210,7 @@ private LogicalPlan withSelectHint(LogicalPlan logicalPlan, List noUseRuleParameters = new ArrayList(); + List noUseRuleParameters = new ArrayList<>(); for (HintAssignmentContext kv : hintStatement.parameters) { String parameterName = visitIdentifierOrText(kv.key); if (kv.key != null) { 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 812d9d00417375..71614770afb70f 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 @@ -713,7 +713,7 @@ private void executeByNereids(TUniqueId queryId) throws Exception { if (context.isTxnModel()) { if (!(logicalPlan instanceof BatchInsertIntoTableCommand || logicalPlan instanceof InsertIntoTableCommand || logicalPlan instanceof UpdateCommand || logicalPlan instanceof DeleteFromUsingCommand - || logicalPlan instanceof DeleteFromCommand)) { + || logicalPlan instanceof DeleteFromCommand || logicalPlan instanceof UnsupportedCommand)) { String errMsg = "This is in a transaction, only insert, update, delete, " + "commit, rollback is acceptable."; throw new NereidsException(errMsg, new AnalysisException(errMsg)); From c7e4439aed88c9a47838155cba87469cd78c6dbe Mon Sep 17 00:00:00 2001 From: Yongqiang YANG <98214048+dataroaring@users.noreply.github.com> Date: Thu, 29 Aug 2024 22:03:19 +0800 Subject: [PATCH 29/60] [refactor](env) refactor getClusterIdAndRole in CloudEnv to remove (#39838) remove duplicated code. --- .../apache/doris/cloud/catalog/CloudEnv.java | 221 +----------------- 1 file changed, 1 insertion(+), 220 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java index e52ea292932161..535b8ea582f845 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java @@ -38,25 +38,16 @@ import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.UserException; import org.apache.doris.common.io.CountingDataOutputStream; -import org.apache.doris.common.util.HttpURLUtil; -import org.apache.doris.common.util.NetUtils; -import org.apache.doris.ha.FrontendNodeType; -import org.apache.doris.httpv2.meta.MetaBaseAction; import org.apache.doris.mysql.privilege.PrivPredicate; -import org.apache.doris.persist.Storage; import org.apache.doris.qe.ConnectContext; -import org.apache.doris.system.Frontend; import org.apache.doris.system.SystemInfoService.HostInfo; import com.google.common.base.Preconditions; -import com.google.common.base.Strings; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import java.io.DataInputStream; -import java.io.File; import java.io.IOException; -import java.net.HttpURLConnection; import java.util.List; import java.util.Map; import java.util.Optional; @@ -165,7 +156,6 @@ private Cloud.NodeInfoPB getLocalTypeFromMetaService() { protected void getClusterIdAndRole() throws IOException { NodeInfoPB.NodeType type = NodeInfoPB.NodeType.UNKNOWN; - String feNodeNameFromMeta = ""; // cloud mode while (true) { Cloud.NodeInfoPB nodeInfoPB = null; @@ -184,9 +174,6 @@ protected void getClusterIdAndRole() throws IOException { continue; } type = nodeInfoPB.getNodeType(); - feNodeNameFromMeta = genFeNodeNameFromMeta( - Config.enable_fqdn_mode ? nodeInfoPB.getHost() : nodeInfoPB.getIp(), - nodeInfoPB.getEditLogPort(), nodeInfoPB.getCtime() * 1000); break; } @@ -196,214 +183,8 @@ protected void getClusterIdAndRole() throws IOException { LOG.warn("type current not support, please check it"); System.exit(-1); } - File roleFile = new File(super.imageDir, Storage.ROLE_FILE); - File versionFile = new File(super.imageDir, Storage.VERSION_FILE); - - // if helper node is point to self, or there is ROLE and VERSION file in local. - // get the node type from local - if ((type == NodeInfoPB.NodeType.FE_MASTER) || type != NodeInfoPB.NodeType.FE_OBSERVER - && (isMyself() || (roleFile.exists() && versionFile.exists()))) { - if (!isMyself()) { - LOG.info("find ROLE and VERSION file in local, ignore helper nodes: {}", helperNodes); - } - - // check file integrity, if has. - if ((roleFile.exists() && !versionFile.exists()) || (!roleFile.exists() && versionFile.exists())) { - throw new IOException("role file and version file must both exist or both not exist. " - + "please specific one helper node to recover. will exit."); - } - - // ATTN: - // If the version file and role file does not exist and the helper node is itself, - // this should be the very beginning startup of the cluster, so we create ROLE and VERSION file, - // set isFirstTimeStartUp to true, and add itself to frontends list. - // If ROLE and VERSION file is deleted for some reason, we may arbitrarily start this node as - // FOLLOWER, which may cause UNDEFINED behavior. - // Everything may be OK if the origin role is exactly FOLLOWER, - // but if not, FE process will exit somehow. - Storage storage = new Storage(this.imageDir); - if (!roleFile.exists()) { - // The very first time to start the first node of the cluster. - // It should became a Master node (Master node's role is also FOLLOWER, which means electable) - - // For compatibility. Because this is the very first time to start, so we arbitrarily choose - // a new name for this node - role = FrontendNodeType.FOLLOWER; - if (type == NodeInfoPB.NodeType.FE_MASTER) { - nodeName = feNodeNameFromMeta; - } else { - nodeName = genFeNodeName(selfNode.getHost(), selfNode.getPort(), false /* new style */); - } - - storage.writeFrontendRoleAndNodeName(role, nodeName); - LOG.info("very first time to start this node. role: {}, node name: {}", role.name(), nodeName); - } else { - role = storage.getRole(); - if (role == FrontendNodeType.REPLICA) { - // for compatibility - role = FrontendNodeType.FOLLOWER; - } - - nodeName = storage.getNodeName(); - if (Strings.isNullOrEmpty(nodeName)) { - // In normal case, if ROLE file exist, role and nodeName should both exist. - // But we will get a empty nodeName after upgrading. - // So for forward compatibility, we use the "old-style" way of naming: "ip_port", - // and update the ROLE file. - if (type == NodeInfoPB.NodeType.FE_MASTER) { - nodeName = feNodeNameFromMeta; - } else { - nodeName = genFeNodeName(selfNode.getHost(), selfNode.getPort(), true /* old style */); - } - storage.writeFrontendRoleAndNodeName(role, nodeName); - LOG.info("forward compatibility. role: {}, node name: {}", role.name(), nodeName); - } - // Notice: - // With the introduction of FQDN, the nodeName is no longer bound to an IP address, - // so consistency is no longer checked here. Otherwise, the startup will fail. - } - - Preconditions.checkNotNull(role); - Preconditions.checkNotNull(nodeName); - - if (!versionFile.exists()) { - clusterId = Config.cluster_id == -1 ? Storage.newClusterID() : Config.cluster_id; - token = Strings.isNullOrEmpty(Config.auth_token) ? Storage.newToken() : Config.auth_token; - storage = new Storage(clusterId, token, this.imageDir); - storage.writeClusterIdAndToken(); - - isFirstTimeStartUp = true; - Frontend self = new Frontend(role, nodeName, selfNode.getHost(), selfNode.getPort()); - // Set self alive to true, the BDBEnvironment.getReplicationGroupAdmin() will rely on this to get - // helper node, before the heartbeat thread is started. - self.setIsAlive(true); - // We don't need to check if frontends already contains self. - // frontends must be empty cause no image is loaded and no journal is replayed yet. - // And this frontend will be persisted later after opening bdbje environment. - frontends.put(nodeName, self); - LOG.info("add self frontend: {}", self); - } else { - clusterId = storage.getClusterID(); - if (storage.getToken() == null) { - token = Strings.isNullOrEmpty(Config.auth_token) ? Storage.newToken() : Config.auth_token; - LOG.info("new token={}", token); - storage.setToken(token); - storage.writeClusterIdAndToken(); - } else { - token = storage.getToken(); - } - isFirstTimeStartUp = false; - } - } else { - // cloud mode, type == NodeInfoPB.NodeType.FE_OBSERVER - // try to get role and node name from helper node, - // this loop will not end until we get certain role type and name - while (true) { - if (!getFeNodeTypeAndNameFromHelpers()) { - LOG.warn("current node is not added to the group. please add it first. " - + "sleep 5 seconds and retry, current helper nodes: {}", helperNodes); - try { - Thread.sleep(5000); - continue; - } catch (InterruptedException e) { - LOG.warn("", e); - System.exit(-1); - } - } - - if (role == FrontendNodeType.REPLICA) { - // for compatibility - role = FrontendNodeType.FOLLOWER; - } - break; - } - Preconditions.checkState(helperNodes.size() == 1); - Preconditions.checkNotNull(role); - Preconditions.checkNotNull(nodeName); - - HostInfo rightHelperNode = helperNodes.get(0); - - Storage storage = new Storage(this.imageDir); - if (roleFile.exists() && (role != storage.getRole() || !nodeName.equals(storage.getNodeName())) - || !roleFile.exists()) { - storage.writeFrontendRoleAndNodeName(role, nodeName); - } - if (!versionFile.exists()) { - // If the version file doesn't exist, download it from helper node - if (!getVersionFileFromHelper(rightHelperNode)) { - throw new IOException("fail to download version file from " - + rightHelperNode.getHost() + " will exit."); - } - - // NOTE: cluster_id will be init when Storage object is constructed, - // so we new one. - storage = new Storage(this.imageDir); - clusterId = storage.getClusterID(); - token = storage.getToken(); - if (Strings.isNullOrEmpty(token)) { - token = Config.auth_token; - } - LOG.info("get version file from helper, cluster id {}, token {}", clusterId, token); - } else { - // If the version file exist, read the cluster id and check the - // id with helper node to make sure they are identical - clusterId = storage.getClusterID(); - token = storage.getToken(); - LOG.info("check local cluster id {} and token via helper node", clusterId, token); - try { - String url = "http://" + NetUtils - .getHostPortInAccessibleFormat(rightHelperNode.getHost(), Config.http_port) + "/check"; - HttpURLConnection conn = HttpURLUtil.getConnectionWithNodeIdent(url); - conn.setConnectTimeout(2 * 1000); - conn.setReadTimeout(2 * 1000); - String clusterIdString = conn.getHeaderField(MetaBaseAction.CLUSTER_ID); - int remoteClusterId = Integer.parseInt(clusterIdString); - if (remoteClusterId != clusterId) { - LOG.error("cluster id is not equal with helper node {}. will exit. remote:{}, local:{}", - rightHelperNode.getHost(), clusterIdString, clusterId); - throw new IOException( - "cluster id is not equal with helper node " - + rightHelperNode.getHost() + ". will exit."); - } - String remoteToken = conn.getHeaderField(MetaBaseAction.TOKEN); - if (token == null && remoteToken != null) { - LOG.info("get token from helper node. token={}.", remoteToken); - token = remoteToken; - storage.writeClusterIdAndToken(); - storage.reload(); - } - if (Config.enable_token_check) { - Preconditions.checkNotNull(token); - Preconditions.checkNotNull(remoteToken); - if (!token.equals(remoteToken)) { - throw new IOException( - "token is not equal with helper node " + rightHelperNode.getHost() - + ", local token " + token + ", remote token " + remoteToken + ". will exit."); - } - } - } catch (Exception e) { - throw new IOException("fail to check cluster_id and token with helper node.", e); - } - } - - getNewImage(rightHelperNode); - } - - if (Config.cluster_id != -1 && clusterId != Config.cluster_id) { - throw new IOException("cluster id is not equal with config item cluster_id. will exit. " - + "If you are in recovery mode, please also modify the cluster_id in 'doris-meta/image/VERSION'"); - } - - if (role.equals(FrontendNodeType.FOLLOWER)) { - isElectable = true; - } else { - isElectable = false; - } - - Preconditions.checkState(helperNodes.size() == 1); - LOG.info("finished to get cluster id: {}, isElectable: {}, role: {}, node name: {}, token: {}", - clusterId, isElectable, role.name(), nodeName, token); + super.getClusterIdAndRole(); } @Override From 05dfd1cf7278a723f45f7c0632312225abdffb91 Mon Sep 17 00:00:00 2001 From: Siyang Tang <82279870+TangSiyang2001@users.noreply.github.com> Date: Thu, 29 Aug 2024 23:33:05 +0800 Subject: [PATCH 30/60] [enhancement](schema-change) Support light schema change on hash columns and agg key columns with varchar type to change length (#39319) ## Proposed changes 1. Schema change should rebuild distribution info after modifying columns, especially distribution columns. Or it may cause dynamic partition failed when checking distribution columns' equality. 2. Support hash key columns to do light schema change. For unique key or dup key columns, could not be enabled temporarily due to some historical reasons. See #39798 . --- .../doris/alter/SchemaChangeHandler.java | 9 +-- .../apache/doris/alter/SchemaChangeJobV2.java | 1 + .../java/org/apache/doris/catalog/Column.java | 2 +- .../doris/catalog/HashDistributionInfo.java | 4 + .../org/apache/doris/catalog/OlapTable.java | 24 ++++++ ...amic_partition_mod_distribution_key.groovy | 75 +++++++++++++++++++ ...har_schema_change_with_distribution.groovy | 52 ------------- 7 files changed, 107 insertions(+), 60 deletions(-) create mode 100644 regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition_mod_distribution_key.groovy delete mode 100644 regression-test/suites/schema_change_p0/test_varchar_schema_change_with_distribution.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index 3fbcd3f629b52f..87d76d8de39e6d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -614,13 +614,7 @@ private boolean processModifyColumn(ModifyColumnClause alterClause, OlapTable ol if (columnPos == null && col.getDataType() == PrimitiveType.VARCHAR && modColumn.getDataType() == PrimitiveType.VARCHAR) { col.checkSchemaChangeAllowed(modColumn); - // If col and modColumn is not key, it allow light schema change, - // of course, olapTable has been enable light schema change - if (modColumn.isKey() || col.isKey()) { - lightSchemaChange = false; - } else { - lightSchemaChange = olapTable.getEnableLightSchemaChange(); - } + lightSchemaChange = olapTable.getEnableLightSchemaChange(); } if (col.isClusterKey()) { throw new DdlException("Can not modify cluster key column: " + col.getName()); @@ -3047,6 +3041,7 @@ public void updateBaseIndexSchema(OlapTable olapTable, Map column) { + this.distributionColumns = column; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index 9a01c7c7fafb94..c4414e17d39d8e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -473,6 +473,30 @@ public void rebuildFullSchema() { } } + public void rebuildDistributionInfo() { + if (!Objects.equals(defaultDistributionInfo.getType(), DistributionInfoType.HASH)) { + return; + } + HashDistributionInfo distributionInfo = (HashDistributionInfo) defaultDistributionInfo; + Set originalColumnsNames = + distributionInfo.getDistributionColumns() + .stream() + .map(Column::getName) + .collect(Collectors.toSet()); + + List newDistributionColumns = getBaseSchema() + .stream() + .filter(column -> originalColumnsNames.contains(column.getName())) + .map(Column::new) + .collect(Collectors.toList()); + distributionInfo.setDistributionColumns(newDistributionColumns); + + getPartitions() + .stream() + .map(Partition::getDistributionInfo) + .forEach(info -> ((HashDistributionInfo) info).setDistributionColumns(newDistributionColumns)); + } + public boolean deleteIndexInfo(String indexName) { if (!indexNameToId.containsKey(indexName)) { return false; diff --git a/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition_mod_distribution_key.groovy b/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition_mod_distribution_key.groovy new file mode 100644 index 00000000000000..db44f59216bbdd --- /dev/null +++ b/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition_mod_distribution_key.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.apache.doris.regression.suite.ClusterOptions + +suite("test_dynamic_partition_mod_distribution_key") { + def options = new ClusterOptions() + options.setFeNum(2) + docker(options) { + // FIXME: for historical bugs, this case will fail if adding k2 as dup key or unique key + // see in https://github.com/apache/doris/issues/39798 + def keys = ["DUPLICATE KEY (k1)", "UNIQUE KEY (k1)", "AGGREGATE KEY (k1, k2)"] + def aggTypes = ["", "", "REPLACE"] + for (i in 0..<3) { + def key = keys.get(i) + def aggType = aggTypes.get(i) + def tableName = "test_dynamic_partition_mod_distribution_key" + sql """ DROP TABLE IF EXISTS ${tableName} """ + + sql """ + CREATE TABLE IF NOT EXISTS ${tableName} ( + k1 DATE NOT NULL, + k2 VARCHAR(20) NOT NULL, + v INT ${aggType} + ) ${key} + PARTITION BY RANGE(k1) () + DISTRIBUTED BY HASH(k1) BUCKETS 1 + PROPERTIES ( + "dynamic_partition.enable"="true", + "dynamic_partition.end"="3", + "dynamic_partition.buckets"="1", + "dynamic_partition.start"="-3", + "dynamic_partition.prefix"="p", + "dynamic_partition.time_unit"="DAY", + "dynamic_partition.create_history_partition"="true", + "dynamic_partition.replication_allocation" = "tag.location.default: 1") + """ + + sql """ alter table ${tableName} modify column k1 comment 'new_comment_for_k1' """ + sql """ alter table ${tableName} modify column k2 varchar(255) """ + + cluster.restartFrontends() + sleep(30000) + context.reconnectFe() + + sql """ ADMIN SET FRONTEND CONFIG ('dynamic_partition_check_interval_seconds' = '1') """ + sql """ alter table ${tableName} set('dynamic_partition.end'='5') """ + result = sql "show partitions from ${tableName}" + for (def retry = 0; retry < 10; retry++) { // at most wait 120s + if (result.size() == 9) { + break; + } + logger.info("wait dynamic partition scheduler, sleep 1s") + sleep(1000) // sleep 1s + result = sql "show partitions from ${tableName}" + } + assertEquals(9, result.size()) + } + } +} \ No newline at end of file diff --git a/regression-test/suites/schema_change_p0/test_varchar_schema_change_with_distribution.groovy b/regression-test/suites/schema_change_p0/test_varchar_schema_change_with_distribution.groovy deleted file mode 100644 index 5068f0aec438b0..00000000000000 --- a/regression-test/suites/schema_change_p0/test_varchar_schema_change_with_distribution.groovy +++ /dev/null @@ -1,52 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -suite ("test_varchar_schema_change_with_distribution") { - def tableName = "test_varchar_schema_change_with_distribution" - sql """ DROP TABLE IF EXISTS ${tableName} FORCE;""" - - sql """ - CREATE TABLE ${tableName} - ( - dt datetime NOT NULL COMMENT '分区日期', - citycode SMALLINT, - watts_range VARCHAR(20), - pv BIGINT SUM DEFAULT '0' - ) - AGGREGATE KEY(dt, citycode, watts_range) - PARTITION BY RANGE(dt)() - DISTRIBUTED BY HASH(dt, watts_range) BUCKETS 1 - PROPERTIES ( - "dynamic_partition.enable"="true", - "dynamic_partition.end"="3", - "dynamic_partition.buckets"="1", - "dynamic_partition.start"="-3", - "dynamic_partition.prefix"="p", - "dynamic_partition.time_unit"="HOUR", - "dynamic_partition.create_history_partition"="true", - "dynamic_partition.replication_allocation" = "tag.location.default: 1" - ); - """ - - test { - sql """ alter table ${tableName} modify column watts_range varchar(30) """ - exception "Can not modify distribution column" - } - - sql """ DROP TABLE IF EXISTS ${tableName} FORCE;""" - -} \ No newline at end of file From eae7cf0e991fa3dd54d4fbf7803edd398f8df5e6 Mon Sep 17 00:00:00 2001 From: zclllhhjj Date: Thu, 29 Aug 2024 23:52:05 +0800 Subject: [PATCH 31/60] [Fix](load) Lower lock granularity to reduce loading CPU pressure (#40134) ## Proposed changes Issue Number: close #xxx before when writing block, we have a lock to protect the whole process. now reduced it --- be/src/runtime/tablets_channel.cpp | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/be/src/runtime/tablets_channel.cpp b/be/src/runtime/tablets_channel.cpp index eac9693d8390da..329366766f86af 100644 --- a/be/src/runtime/tablets_channel.cpp +++ b/be/src/runtime/tablets_channel.cpp @@ -582,12 +582,15 @@ Status BaseTabletsChannel::_write_block_data( // add_batch may concurrency with inc_open but not under _lock. // so need to protect it with _tablet_writers_lock. - std::lock_guard l(_tablet_writers_lock); - - auto tablet_writer_it = _tablet_writers.find(tablet_id); - if (tablet_writer_it == _tablet_writers.end()) { - return Status::InternalError("unknown tablet to append data, tablet={}", tablet_id); + decltype(_tablet_writers.find(tablet_id)) tablet_writer_it; + { + std::lock_guard l(_tablet_writers_lock); + tablet_writer_it = _tablet_writers.find(tablet_id); + if (tablet_writer_it == _tablet_writers.end()) { + return Status::InternalError("unknown tablet to append data, tablet={}", tablet_id); + } } + Status st = write_func(tablet_writer_it->second.get()); if (!st.ok()) { auto err_msg = From 7c6ce2a2afb84676fc2ff4f080ab89760ceda5fa Mon Sep 17 00:00:00 2001 From: Gabriel Date: Fri, 30 Aug 2024 00:09:53 +0800 Subject: [PATCH 32/60] [fix](core) Fix `BUCKET_HASH_SHUFFLE` local exchanger (#40148) ## Proposed changes Issue Number: close #xxx --- be/src/pipeline/pipeline_fragment_context.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index 3b94b258d31073..19d94997531497 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -729,7 +729,7 @@ Status PipelineFragmentContext::_add_local_exchange_impl( sink_id, local_exchange_id, should_disable_bucket_shuffle ? _total_instances : _num_instances, data_distribution.partition_exprs, bucket_seq_to_instance_idx)); - if (should_disable_bucket_shuffle && + if (bucket_seq_to_instance_idx.empty() && data_distribution.distribution_type == ExchangeType::BUCKET_HASH_SHUFFLE) { data_distribution.distribution_type = ExchangeType::HASH_SHUFFLE; } From 2c5155bde5f496a5dc0c60875c4e6e24836530f1 Mon Sep 17 00:00:00 2001 From: TengJianPing <18241664+jacktengg@users.noreply.github.com> Date: Fri, 30 Aug 2024 10:11:45 +0800 Subject: [PATCH 33/60] [fix](ut) fix window_funnel UT failure (#40146) ## Proposed changes Issue Number: close #xxx --- .../aggregate_function_window_funnel.h | 114 +++++++----------- .../vec_window_funnel_test.cpp | 10 +- 2 files changed, 50 insertions(+), 74 deletions(-) diff --git a/be/src/vec/aggregate_functions/aggregate_function_window_funnel.h b/be/src/vec/aggregate_functions/aggregate_function_window_funnel.h index 5a31b88a69be46..1f6a6e74968f42 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_window_funnel.h +++ b/be/src/vec/aggregate_functions/aggregate_function_window_funnel.h @@ -83,14 +83,11 @@ struct WindowFunnelState { int64_t window; bool enable_mode; WindowFunnelMode window_funnel_mode; - mutable MutableColumnPtr timestamp_column; - mutable MutableColumns event_columns; + mutable vectorized::MutableBlock mutable_block; ColumnVector::Container* timestamp_column_data; std::vector::Container*> event_columns_datas; - Block block; SortDescription sort_description {1}; bool sorted; - bool is_merge; WindowFunnelState() { event_count = 0; @@ -101,30 +98,38 @@ struct WindowFunnelState { sort_description[0].direction = 1; sort_description[0].nulls_direction = -1; sorted = false; - is_merge = false; } WindowFunnelState(int arg_event_count) : WindowFunnelState() { - timestamp_column = ColumnVector::create(); + event_count = arg_event_count; + auto timestamp_column = ColumnVector::create(); timestamp_column_data = &assert_cast&>(*timestamp_column).get_data(); - event_count = arg_event_count; - event_columns.resize(event_count); + + MutableColumns event_columns; for (int i = 0; i < event_count; i++) { - event_columns[i] = ColumnVector::create(); + auto event_column = ColumnVector::create(); event_columns_datas.emplace_back( - &assert_cast&>(*event_columns[i]).get_data()); + &assert_cast&>(*event_column).get_data()); + event_columns.emplace_back(std::move(event_column)); } + Block tmp_block; + tmp_block.insert({std::move(timestamp_column), + DataTypeFactory::instance().create_data_type(TYPE_INDEX), "timestamp"}); + for (int i = 0; i < event_count; i++) { + tmp_block.insert({std::move(event_columns[i]), + DataTypeFactory::instance().create_data_type(TypeIndex::UInt8), + "event_" + std::to_string(i)}); + } + + mutable_block = MutableBlock(std::move(tmp_block)); } void reset() { window = 0; - timestamp_column->clear(); - for (auto& column : event_columns) { - column->clear(); - } - block.clear_column_data(); + mutable_block.clear(); + timestamp_column_data = nullptr; + event_columns_datas.clear(); sorted = false; - is_merge = false; } void add(const IColumn** arg_columns, ssize_t row_num, int64_t win, WindowFunnelMode mode) { @@ -144,24 +149,11 @@ struct WindowFunnelState { if (sorted) { return; } - if (!is_merge) { - Block tmp_block; - tmp_block.insert({std::move(timestamp_column), - DataTypeFactory::instance().create_data_type(TYPE_INDEX), - "timestamp"}); - for (int i = 0; i < event_count; i++) { - tmp_block.insert({std::move(event_columns[i]), - DataTypeFactory::instance().create_data_type(TypeIndex::UInt8), - "event_" + std::to_string(i)}); - } - block = tmp_block.clone_without_columns(); - sort_block(tmp_block, block, sort_description, 0); - } else { - auto tmp_block = block.clone_without_columns(); - sort_block(block, tmp_block, sort_description, 0); - block = std::move(tmp_block); - } + Block tmp_block = mutable_block.to_block(); + auto block = tmp_block.clone_without_columns(); + sort_block(tmp_block, block, sort_description, 0); + mutable_block = MutableBlock(std::move(block)); sorted = true; } @@ -174,9 +166,9 @@ struct WindowFunnelState { TimeInterval interval(SECOND, window, false); int column_idx = 1; - const auto& first_event_column = block.get_by_position(column_idx); + const auto& first_event_column = mutable_block.get_column_by_position(column_idx); const auto& first_event_data = - assert_cast&>(*first_event_column.column).get_data(); + assert_cast&>(*first_event_column).get_data(); auto match_row = simd::find_one(first_event_data.data(), start_row, row_count); start_row = match_row + 1; if (match_row < row_count) { @@ -189,9 +181,9 @@ struct WindowFunnelState { column_idx++; auto last_match_row = match_row; for (; column_idx < event_count + 1; column_idx++) { - const auto& event_column = block.get_by_position(column_idx); + const auto& event_column = mutable_block.get_column_by_position(column_idx); const auto& event_data = - assert_cast&>(*event_column.column).get_data(); + assert_cast&>(*event_column).get_data(); if constexpr (WINDOW_FUNNEL_MODE == WindowFunnelMode::FIXED) { ++match_row; if (event_data[match_row] == 1) { @@ -227,10 +219,9 @@ struct WindowFunnelState { for (int tmp_column_idx = 1; tmp_column_idx < column_idx; tmp_column_idx++) { const auto& tmp_event_column = - block.get_by_position(tmp_column_idx); + mutable_block.get_column_by_position(tmp_column_idx); const auto& tmp_event_data = - assert_cast&>( - *tmp_event_column.column) + assert_cast&>(*tmp_event_column) .get_data(); auto dup_match_row = simd::find_one(tmp_event_data.data(), last_match_row + 1, match_row); @@ -258,11 +249,11 @@ struct WindowFunnelState { int _get_internal() const { size_t start_row = 0; int max_found_event_count = 0; - const auto& ts_column = block.get_by_position(0).column->get_ptr(); + const auto& ts_column = mutable_block.get_column_by_position(0)->get_ptr(); const auto& timestamp_data = assert_cast&>(*ts_column).get_data().data(); - auto row_count = block.rows(); + auto row_count = mutable_block.rows(); while (start_row < row_count) { auto found_event_count = _match_event_list(start_row, row_count, timestamp_data); @@ -274,7 +265,7 @@ struct WindowFunnelState { return max_found_event_count; } int get() const { - auto row_count = block.rows(); + auto row_count = mutable_block.rows(); if (event_count == 0 || row_count == 0) { return 0; } @@ -294,16 +285,13 @@ struct WindowFunnelState { } void merge(const WindowFunnelState& other) { - is_merge = true; - MutableBlock mutable_block(&block); - if (!other.block.empty()) { - auto st = mutable_block.merge(other.block); + if (!other.mutable_block.empty()) { + auto st = mutable_block.merge(other.mutable_block.to_block()); if (!st.ok()) { throw doris::Exception(ErrorCode::INTERNAL_ERROR, st.to_string()); return; } } - block = mutable_block.to_block(); event_count = event_count > 0 ? event_count : other.event_count; window = window > 0 ? window : other.window; @@ -328,27 +316,13 @@ struct WindowFunnelState { size_t compressed_bytes = 0; Status status; std::string buff; - if (is_merge) { - // as the branch-2.1 is used the new impl of window funnel, and the be_exec_version is 5 - // but in branch-3.0 this be_exec_version have update to 7, so when upgrade from branch-2.1 to branch-3.0 - // maybe have error send the branch-3.0 version of version 7 to branch-2.1([0---version--5]) - status = block.serialize( - 5, &pblock, &uncompressed_bytes, &compressed_bytes, - segment_v2::CompressionTypePB::ZSTD); // ZSTD for better compression ratio - } else { - Block tmp_block; - tmp_block.insert({std::move(timestamp_column), - DataTypeFactory::instance().create_data_type(TYPE_INDEX), - "timestamp"}); - for (int i = 0; i < event_count; i++) { - tmp_block.insert({std::move(event_columns[i]), - DataTypeFactory::instance().create_data_type(TypeIndex::UInt8), - "event_" + std::to_string(i)}); - } - status = tmp_block.serialize( - 5, &pblock, &uncompressed_bytes, &compressed_bytes, - segment_v2::CompressionTypePB::ZSTD); // ZSTD for better compression ratio - } + Block block = mutable_block.to_block(); + // as the branch-2.1 is used the new impl of window funnel, and the be_exec_version is 5 + // but in branch-3.0 this be_exec_version have update to 7, so when upgrade from branch-2.1 to branch-3.0 + // maybe have error send the branch-3.0 version of version 7 to branch-2.1([0---version--5]) + status = block.serialize( + 5, &pblock, &uncompressed_bytes, &compressed_bytes, + segment_v2::CompressionTypePB::ZSTD); // ZSTD for better compression ratio if (!status.ok()) { throw doris::Exception(ErrorCode::INTERNAL_ERROR, status.to_string()); return; @@ -385,10 +359,12 @@ struct WindowFunnelState { throw doris::Exception(ErrorCode::INTERNAL_ERROR, "Failed to parse window_funnel data to block"); } + Block block; auto status = block.deserialize(pblock); if (!status.ok()) { throw doris::Exception(ErrorCode::INTERNAL_ERROR, status.to_string()); } + mutable_block = MutableBlock(std::move(block)); } }; diff --git a/be/test/vec/aggregate_functions/vec_window_funnel_test.cpp b/be/test/vec/aggregate_functions/vec_window_funnel_test.cpp index 5ea6ce205ba873..2877217c8f1142 100644 --- a/be/test/vec/aggregate_functions/vec_window_funnel_test.cpp +++ b/be/test/vec/aggregate_functions/vec_window_funnel_test.cpp @@ -147,10 +147,15 @@ TEST_F(VWindowFunnelTest, testSerialize) { agg_function->add(place, column, i, nullptr); } + ColumnVector column_result; + agg_function->insert_result_into(place, column_result); + EXPECT_EQ(column_result.get_data()[0], 3); + ColumnString buf; VectorBufferWriter buf_writer(buf); agg_function->serialize(place, buf_writer); buf_writer.commit(); + agg_function->destroy(place); std::unique_ptr memory2(new char[agg_function->size_of_data()]); AggregateDataPtr place2 = memory2.get(); @@ -159,11 +164,6 @@ TEST_F(VWindowFunnelTest, testSerialize) { VectorBufferReader buf_reader(buf.get_data_at(0)); agg_function->deserialize(place2, buf_reader, nullptr); - ColumnVector column_result; - agg_function->insert_result_into(place, column_result); - EXPECT_EQ(column_result.get_data()[0], 3); - agg_function->destroy(place); - ColumnVector column_result2; agg_function->insert_result_into(place2, column_result2); EXPECT_EQ(column_result2.get_data()[0], 3); From b53b8c9c115485f3912914e347e2c8dd45dde632 Mon Sep 17 00:00:00 2001 From: smallx Date: Fri, 30 Aug 2024 10:24:42 +0800 Subject: [PATCH 34/60] [fix](restore) Reset next version for remote table when restore (#40118) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit We should reset next version to visible version + 1 for all partitions of remote table, when restoring table that do not exist locally. 在高并发insert场景,CCR源端表的next version可能比visible version大比较多,目标集群restore全量快照后,切换到增量binlog后,commit事务时使用的version(来自next version)就可能比当前visible version大比较多。 此时,对于MoW表,就会出现publish version不连续,增量binlog一直无法publish的问题。事务状态会一直是`COMMITTED`,并伴随类似ErrMsg `wait for publishing partition 15027 version 1037597. self version: 1037627. table 15025`。 --- .../src/main/java/org/apache/doris/backup/RestoreJob.java | 3 +++ .../src/main/java/org/apache/doris/catalog/OlapTable.java | 6 ++++++ 2 files changed, 9 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java index b7880302442cb8..12a30a97277995 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java @@ -771,6 +771,9 @@ private void checkAndPrepareMeta() { return; } + // reset next version to visible version + 1 for all partitions + remoteOlapTbl.resetVersionForRestore(); + // Reset properties to correct values. remoteOlapTbl.resetPropertiesForRestore(reserveDynamicPartitionEnable, reserveReplica, replicaAlloc, isBeingSynced); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index c4414e17d39d8e..78d97e8e48bc90 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -653,6 +653,12 @@ public void resetPropertiesForRestore(boolean reserveDynamicPartitionEnable, boo setColocateGroup(null); } + public void resetVersionForRestore() { + for (Partition partition : idToPartition.values()) { + partition.setNextVersion(partition.getVisibleVersion() + 1); + } + } + public Status resetIdsForRestore(Env env, Database db, ReplicaAllocation restoreReplicaAlloc, boolean reserveReplica, String srcDbName) { // ATTN: The meta of the restore may come from different clusters, so the From 162ac86cfa6ab17367fd2aed34c2563fc7320524 Mon Sep 17 00:00:00 2001 From: 924060929 <924060929@qq.com> Date: Fri, 30 Aug 2024 10:45:38 +0800 Subject: [PATCH 35/60] [fix](test) fix unstable test (#40142) fix unstable test --- .../suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy | 1 + 1 file changed, 1 insertion(+) 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 3381b5653c4865..8284873693693d 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 @@ -144,6 +144,7 @@ suite("parse_sql_from_sql_cache") { set "partitions", "p1" inputIterator([[1, 3], [1, 4]].iterator()) } + sql "sync" // stream load can not use cache sql "select * from test_use_plan_cache5" From e78c38f0fda012ec2321141472fbec41fae1ad55 Mon Sep 17 00:00:00 2001 From: abmdocrt Date: Fri, 30 Aug 2024 10:50:32 +0800 Subject: [PATCH 36/60] [Fix](delete) Fix invalid `enable_mow_light_delete` propertiy default value (#40104) --- .../java/org/apache/doris/common/util/PropertyAnalyzer.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java index 760ee10542a646..e978dd7c4d35a9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java @@ -1469,11 +1469,11 @@ public static boolean analyzeUniqueKeyMergeOnWrite(Map propertie public static boolean analyzeEnableDeleteOnDeletePredicate(Map properties) throws AnalysisException { if (properties == null || properties.isEmpty()) { - return false; + return Config.enable_mow_light_delete; } String value = properties.get(PropertyAnalyzer.PROPERTIES_ENABLE_MOW_LIGHT_DELETE); if (value == null) { - return false; + return Config.enable_mow_light_delete; } properties.remove(PropertyAnalyzer.PROPERTIES_ENABLE_MOW_LIGHT_DELETE); if (value.equals("true")) { From 20af746a5c4a799ebaec1677361bdf8aab3c85ea Mon Sep 17 00:00:00 2001 From: starocean999 <40539150+starocean999@users.noreply.github.com> Date: Fri, 30 Aug 2024 10:56:25 +0800 Subject: [PATCH 37/60] [fix](nereids)change lag/lead function signature (#40060) remove CharType from the signature --- .../functions/window/RequireTrivialTypes.java | 2 -- .../lag_lead_signature.groovy | 35 +++++++++++++++++++ 2 files changed, 35 insertions(+), 2 deletions(-) create mode 100644 regression-test/suites/nereids_syntax_p0/lag_lead_signature.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/RequireTrivialTypes.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/RequireTrivialTypes.java index 34295732d58263..04d92fdeb11132 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/RequireTrivialTypes.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/RequireTrivialTypes.java @@ -19,7 +19,6 @@ import org.apache.doris.nereids.types.BigIntType; import org.apache.doris.nereids.types.BooleanType; -import org.apache.doris.nereids.types.CharType; import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.types.DateTimeType; import org.apache.doris.nereids.types.DateTimeV2Type; @@ -64,7 +63,6 @@ public interface RequireTrivialTypes { DateTimeV2Type.SYSTEM_DEFAULT, TimeType.INSTANCE, TimeV2Type.INSTANCE, - CharType.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT, StringType.INSTANCE ); diff --git a/regression-test/suites/nereids_syntax_p0/lag_lead_signature.groovy b/regression-test/suites/nereids_syntax_p0/lag_lead_signature.groovy new file mode 100644 index 00000000000000..5dde98a95a0f5c --- /dev/null +++ b/regression-test/suites/nereids_syntax_p0/lag_lead_signature.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. + +suite("lag_lead_signature") { + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + sql """drop table if exists lag_lead_signature_t""" + sql """ CREATE TABLE lag_lead_signature_t ( + `k` VARCHAR(200) NULL + ) ENGINE=OLAP + DUPLICATE KEY(`k`) + DISTRIBUTED BY HASH(`k`) BUCKETS 3 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql "insert into lag_lead_signature_t values ('44'), ('28');" + + sql "select lag(k, 1, 0) over(), lead(k, 1, 0) over() from lag_lead_signature_t;" + +} \ No newline at end of file From d580a0a912aad101740c4f385d331cef09954bb6 Mon Sep 17 00:00:00 2001 From: minghong Date: Fri, 30 Aug 2024 11:29:45 +0800 Subject: [PATCH 38/60] [opt](nereids) set lower bound for range-selectivity (#40089) ## Proposed changes Range selectivity is prone to producing outliers, so we add this threshold limit. The threshold estimation is calculated based on selecting one month out of fifty years. Issue Number: close #xxx --- .../doris/nereids/stats/FilterEstimation.java | 5 ++++ .../nereids/stats/FilterEstimationTest.java | 29 +++++++++++++++++++ 2 files changed, 34 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java index 2839a817700794..1628c3b7d72e3f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/FilterEstimation.java @@ -67,6 +67,9 @@ */ public class FilterEstimation extends ExpressionVisitor { public static final double DEFAULT_INEQUALITY_COEFFICIENT = 0.5; + // "Range selectivity is prone to producing outliers, so we add this threshold limit. + // The threshold estimation is calculated based on selecting one month out of fifty years." + public static final double RANGE_SELECTIVITY_THRESHOLD = 0.0016; public static final double DEFAULT_IN_COEFFICIENT = 1.0 / 3.0; public static final double DEFAULT_LIKE_COMPARISON_SELECTIVITY = 0.2; @@ -600,6 +603,8 @@ private Statistics estimateBinaryComparisonFilter(Expression leftExpr, DataType double sel = leftRange.overlapPercentWith(rightRange); if (!(dataType instanceof RangeScalable) && (sel != 0.0 && sel != 1.0)) { sel = DEFAULT_INEQUALITY_COEFFICIENT; + } else if (sel < RANGE_SELECTIVITY_THRESHOLD) { + sel = RANGE_SELECTIVITY_THRESHOLD; } sel = getNotNullSelectivity(leftStats, sel); updatedStatistics = context.statistics.withSel(sel); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java index d7c44e082cf52e..0158dd9587c70b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/FilterEstimationTest.java @@ -34,6 +34,7 @@ import org.apache.doris.nereids.trees.expressions.Or; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.functions.scalar.Left; +import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.literal.DateLiteral; import org.apache.doris.nereids.trees.expressions.literal.DoubleLiteral; import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; @@ -1365,4 +1366,32 @@ public void testStringRangeColToCol() { Statistics agrtc = new FilterEstimation().estimate(new GreaterThan(a, c), baseStats); Assertions.assertEquals(50, agrtc.getRowCount()); } + + @Test + public void testLargeRange() { + SlotReference a = new SlotReference("a", IntegerType.INSTANCE); + long tenB = 1000000000; + long row = 1600000000; + ColumnStatistic colStats = new ColumnStatisticBuilder() + .setAvgSizeByte(10) + .setCount(row) + .setNdv(10000) + .setMinExpr(new IntLiteral(0)) + .setMinValue(0) + .setMaxExpr(new IntLiteral(tenB)) + .setMaxValue(tenB) + .build(); + Statistics stats = new StatisticsBuilder() + .setRowCount(row) + .putColumnStatistics(a, colStats) + .build(); + Expression less = new LessThan(a, new IntegerLiteral(50000)); + FilterEstimation estimation = new FilterEstimation(); + Statistics out = estimation.estimate(less, stats); + Assertions.assertEquals(out.getRowCount(), row * FilterEstimation.RANGE_SELECTIVITY_THRESHOLD); + + Expression greater = new GreaterThan(a, new BigIntLiteral(tenB - 5000L)); + out = estimation.estimate(greater, stats); + Assertions.assertEquals(out.getRowCount(), row * FilterEstimation.RANGE_SELECTIVITY_THRESHOLD); + } } From 721d460dfd245f83be5eb1646931e47cd6f0b49f Mon Sep 17 00:00:00 2001 From: zhiqiang Date: Fri, 30 Aug 2024 11:45:14 +0800 Subject: [PATCH 39/60] [chore](cancel) Make enable_pipeline_task_leakage_detect false by default (#40145) add be.conf enable_pipeline_task_leakage_detect, default to false. --- be/src/common/config.cpp | 2 ++ be/src/common/config.h | 2 ++ be/src/runtime/fragment_mgr.cpp | 5 +++-- 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index f051bb3e8569b0..d6aaeaf6accf1c 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1320,6 +1320,8 @@ DEFINE_mInt64(pipeline_task_leakage_detect_period_secs, "60"); DEFINE_mInt32(snappy_compression_block_size, "262144"); DEFINE_mInt32(lz4_compression_block_size, "262144"); +DEFINE_mBool(enable_pipeline_task_leakage_detect, "false"); + // clang-format off #ifdef BE_TEST // test s3 diff --git a/be/src/common/config.h b/be/src/common/config.h index 69e17f843e6ba9..017eab0302ac90 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1414,6 +1414,8 @@ DECLARE_mInt64(pipeline_task_leakage_detect_period_secs); DECLARE_mInt32(snappy_compression_block_size); DECLARE_mInt32(lz4_compression_block_size); +DECLARE_mBool(enable_pipeline_task_leakage_detect); + #ifdef BE_TEST // test s3 DECLARE_String(test_s3_resource); diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index 32b7dcbcbb226e..a9fb5a2f35229d 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -951,8 +951,9 @@ void FragmentMgr::cancel_worker() { timespec now; clock_gettime(CLOCK_MONOTONIC, &now); - if (now.tv_sec - check_invalid_query_last_timestamp.tv_sec > - config::pipeline_task_leakage_detect_period_secs) { + if (config::enable_pipeline_task_leakage_detect && + now.tv_sec - check_invalid_query_last_timestamp.tv_sec > + config::pipeline_task_leakage_detect_period_secs) { check_invalid_query_last_timestamp = now; running_queries_on_all_fes = _get_all_running_queries_from_fe(); } else { From 263faeab678cd326ed68fefe061c722349d39b45 Mon Sep 17 00:00:00 2001 From: sparrow <38098988+biohazard4321@users.noreply.github.com> Date: Fri, 30 Aug 2024 12:21:14 +0800 Subject: [PATCH 40/60] [fix](cooldown)No need to rdlock inside get_cooldown_tablets, there's enough rdlock inside tablet internal function calls. (#39211) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Proposed changes There's a recurse rdlock inside get_cooldown;when using libcxx,libcxx‘s shared_mutex dosen't support recursive lock, so causes hangs below: #0 0x00007f788b101a35 in pthread_cond_wait@@GLIBC_2.3.2 () from /lib64/libpthread.so.0 #1 0x00005636b8ff996f in std::__1::condition_variable::wait(std::__1::unique_lock&) () #2 0x00005636b8ffad8b in std::__1::__shared_mutex_base::lock_shared() () #3 0x00005636addc524c in doris::Tablet::_has_data_to_cooldown() () #4 0x00005636addc353b in doris::Tablet::pick_cooldown_rowset() () #5 0x00005636addc56b9 in doris::Tablet::need_cooldown(long*, unsigned long*) () #6 0x00005636adde52f7 in doris::TabletManager::get_cooldown_tablets(std::__1::vector, std::__1::allocator > >*, std::__1::vector, std::__1::allocator > >*, std::__1::function const&)>) () #7 0x00005636adb0ed81 in doris::StorageEngine::_cooldown_tasks_producer_callback() () #8 0x00005636ae17a253 in doris::Thread::supervise_thread(void*) () #9 0x00007f788b0fdea5 in start_thread () from /lib64/libpthread.so.0 #10 0x00007f788b916b0d in clone () from /lib64/libc.so.6 review the code, there's no need a rdlock inside TabletManager::get_cooldown_tablets --- be/src/olap/tablet_manager.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp index 6696dcf2e68df2..3bac96370333d3 100644 --- a/be/src/olap/tablet_manager.cpp +++ b/be/src/olap/tablet_manager.cpp @@ -1620,7 +1620,6 @@ void TabletManager::get_cooldown_tablets(std::vector* tablets, if (UNLIKELY(nullptr == tablet)) { return; } - std::shared_lock rdlock(tablet->get_header_lock()); int64_t cooldown_timestamp = -1; size_t file_size = -1; if (!skip_tablet(tablet) && From 08a1affde4520dd8ac711363c44ffb2524d3739a Mon Sep 17 00:00:00 2001 From: hui lai <1353307710@qq.com> Date: Fri, 30 Aug 2024 12:23:35 +0800 Subject: [PATCH 41/60] [fix](multi table) restrict the multi tables load memory under high concurrency with a large number of tables (#39992) BE node was killed by OOM-killer when use multi table load under high concurrency with a large number of tables(128 concurrency and every concurrency load 200 tables). This pr restricts the multi tables load memory under this issue. If memory reaches hard limit, new task will be rejected and return directly. --- be/src/runtime/exec_env_init.cpp | 4 ++-- .../routine_load_task_executor.cpp | 18 ++++++++++++++++-- .../routine_load/routine_load_task_executor.h | 5 ++++- .../routine_load_task_executor_test.cpp | 2 +- 4 files changed, 23 insertions(+), 6 deletions(-) diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 53fe1993139cb0..674d5ee5115d76 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -298,7 +298,7 @@ Status ExecEnv::_init(const std::vector& store_paths, _stream_load_executor = StreamLoadExecutor::create_shared(this); } _routine_load_task_executor = new RoutineLoadTaskExecutor(this); - RETURN_IF_ERROR(_routine_load_task_executor->init()); + RETURN_IF_ERROR(_routine_load_task_executor->init(MemInfo::mem_limit())); _small_file_mgr = new SmallFileMgr(this, config::small_file_dir); _group_commit_mgr = new GroupCommitMgr(this); _memtable_memory_limiter = std::make_unique(); @@ -607,7 +607,7 @@ void ExecEnv::init_mem_tracker() { _s3_file_buffer_tracker = MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::GLOBAL, "S3FileBuffer"); _stream_load_pipe_tracker = - MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::GLOBAL, "StreamLoadPipe"); + MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::LOAD, "StreamLoadPipe"); } void ExecEnv::_register_metrics() { diff --git a/be/src/runtime/routine_load/routine_load_task_executor.cpp b/be/src/runtime/routine_load/routine_load_task_executor.cpp index 958ad37f8d2d09..b63495df837d1a 100644 --- a/be/src/runtime/routine_load/routine_load_task_executor.cpp +++ b/be/src/runtime/routine_load/routine_load_task_executor.cpp @@ -75,7 +75,8 @@ RoutineLoadTaskExecutor::~RoutineLoadTaskExecutor() { _task_map.clear(); } -Status RoutineLoadTaskExecutor::init() { +Status RoutineLoadTaskExecutor::init(int64_t process_mem_limit) { + _load_mem_limit = process_mem_limit * config::load_process_max_memory_limit_percent / 100; return ThreadPoolBuilder("routine_load") .set_min_threads(0) .set_max_threads(config::max_routine_load_thread_pool_size) @@ -210,7 +211,7 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) { return Status::OK(); } - if (_task_map.size() >= config::max_routine_load_thread_pool_size) { + if (_task_map.size() >= config::max_routine_load_thread_pool_size || _reach_memory_limit()) { LOG(INFO) << "too many tasks in thread pool. reject task: " << UniqueId(task.id) << ", job id: " << task.job_id << ", queue size: " << _thread_pool->get_queue_size() @@ -311,6 +312,19 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) { } } +bool RoutineLoadTaskExecutor::_reach_memory_limit() { + bool is_exceed_soft_mem_limit = GlobalMemoryArbitrator::is_exceed_soft_mem_limit(); + auto current_load_mem_value = + MemTrackerLimiter::TypeMemSum[MemTrackerLimiter::Type::LOAD]->current_value(); + if (is_exceed_soft_mem_limit || current_load_mem_value > _load_mem_limit) { + LOG(INFO) << "is_exceed_soft_mem_limit: " << is_exceed_soft_mem_limit + << " current_load_mem_value: " << current_load_mem_value + << " _load_mem_limit: " << _load_mem_limit; + return true; + } + return false; +} + void RoutineLoadTaskExecutor::exec_task(std::shared_ptr ctx, DataConsumerPool* consumer_pool, ExecFinishCallback cb) { #define HANDLE_ERROR(stmt, err_msg) \ diff --git a/be/src/runtime/routine_load/routine_load_task_executor.h b/be/src/runtime/routine_load/routine_load_task_executor.h index f16ef80ef76f8e..0e597d796c9f77 100644 --- a/be/src/runtime/routine_load/routine_load_task_executor.h +++ b/be/src/runtime/routine_load/routine_load_task_executor.h @@ -51,7 +51,7 @@ class RoutineLoadTaskExecutor { ~RoutineLoadTaskExecutor(); - Status init(); + Status init(int64_t process_mem_limit); void stop(); @@ -86,6 +86,7 @@ class RoutineLoadTaskExecutor { // create a dummy StreamLoadContext for PKafkaMetaProxyRequest Status _prepare_ctx(const PKafkaMetaProxyRequest& request, std::shared_ptr ctx); + bool _reach_memory_limit(); private: ExecEnv* _exec_env = nullptr; @@ -95,6 +96,8 @@ class RoutineLoadTaskExecutor { std::mutex _lock; // task id -> load context std::unordered_map> _task_map; + + int64_t _load_mem_limit = -1; }; } // namespace doris diff --git a/be/test/runtime/routine_load_task_executor_test.cpp b/be/test/runtime/routine_load_task_executor_test.cpp index f95fdcfdadfb8a..338b82c6eba3a2 100644 --- a/be/test/runtime/routine_load_task_executor_test.cpp +++ b/be/test/runtime/routine_load_task_executor_test.cpp @@ -94,7 +94,7 @@ TEST_F(RoutineLoadTaskExecutorTest, exec_task) { RoutineLoadTaskExecutor executor(&_env); Status st; - st = executor.init(); + st = executor.init(1024 * 1024); EXPECT_TRUE(st.ok()); // submit task st = executor.submit_task(task); From b3cc795b2cbe3ee3a524795b43dc61f2dcb1f5ff Mon Sep 17 00:00:00 2001 From: Pxl Date: Fri, 30 Aug 2024 12:43:52 +0800 Subject: [PATCH 42/60] [Bug](runtime-filter) support ip rf and use exception to replace dcheck when PrimitiveType to PColumnType (#39985) ## Proposed changes use exception to replace dcheck when PrimitiveType to PColumnType ```cpp *** SIGABRT unknown detail explain (@0x11d3f) received by PID 73023 (TID 74292 OR 0x7fd758225640) from PID 73023; stack trace: *** 0# doris::signal::(anonymous namespace)::FailureSignalHandler(int, siginfo_t*, void*) at /home/zcp/repo_center/doris_master/doris/be/src/common/signal_handler.h:421 1# 0x00007FDDBE6B9520 in /lib/x86_64-linux-gnu/libc.so.6 2# pthread_kill at ./nptl/pthread_kill.c:89 3# raise at ../sysdeps/posix/raise.c:27 4# abort at ./stdlib/abort.c:81 5# 0x000056123F81A94D in /root/output/be/lib/doris_be 6# 0x000056123F80CF8A in /root/output/be/lib/doris_be 7# google::LogMessage::SendToLog() in /root/output/be/lib/doris_be 8# google::LogMessage::Flush() in /root/output/be/lib/doris_be 9# google::LogMessageFatal::~LogMessageFatal() in /root/output/be/lib/doris_be 10# doris::to_proto(doris::PrimitiveType) at /home/zcp/repo_center/doris_master/doris/be/src/exprs/runtime_filter.cpp:114 11# doris::IRuntimeFilter::push_to_remote(doris::TNetworkAddress const*) at /home/zcp/repo_center/doris_master/doris/be/src/exprs/runtime_filter.cpp:1143 12# doris::IRuntimeFilter::publish(bool)::$_0::operator()(doris::IRuntimeFilter*) const at /home/zcp/repo_center/doris_master/doris/be/src/exprs/runtime_filter.cpp:959 13# doris::IRuntimeFilter::publish(bool)::$_2::operator()() const at /home/zcp/repo_center/doris_master/doris/be/src/exprs/runtime_filter.cpp:983 14# doris::IRuntimeFilter::publish(bool) at /home/zcp/repo_center/doris_master/doris/be/src/exprs/runtime_filter.cpp:997 ``` --- be/src/exprs/runtime_filter.cpp | 108 ++++++++++++++---- be/src/olap/predicate_creator.h | 4 +- be/src/runtime/large_int_value.h | 5 + be/src/runtime/primitive_type.h | 4 +- be/src/vec/core/types.h | 6 +- be/src/vec/data_types/data_type_ipv6.cpp | 2 +- be/src/vec/data_types/data_type_ipv6.h | 2 +- be/src/vec/exprs/vexpr.cpp | 13 ++- be/src/vec/exprs/vexpr.h | 16 +++ be/src/vec/olap/olap_data_convertor.cpp | 4 +- be/src/vec/runtime/ipv4_value.h | 18 +-- be/src/vec/runtime/ipv6_value.h | 18 +-- .../serde/data_type_serde_mysql_test.cpp | 4 +- be/test/vec/runtime/ip_value_test.cpp | 16 +-- gensrc/proto/internal_service.proto | 6 +- .../data/datatype_p0/ip/ip_rf/test_ip_rf.out | 31 +++++ .../datatype_p0/ip/ip_rf/test_ip_rf.groovy | 73 ++++++++++++ 17 files changed, 264 insertions(+), 66 deletions(-) create mode 100644 regression-test/data/datatype_p0/ip/ip_rf/test_ip_rf.out create mode 100644 regression-test/suites/datatype_p0/ip/ip_rf/test_ip_rf.groovy diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index 93d9c159759b0e..1e93dcc9167a99 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -110,11 +110,14 @@ PColumnType to_proto(PrimitiveType type) { return PColumnType::COLUMN_TYPE_VARCHAR; case TYPE_STRING: return PColumnType::COLUMN_TYPE_STRING; + case TYPE_IPV4: + return PColumnType::COLUMN_TYPE_IPV4; + case TYPE_IPV6: + return PColumnType::COLUMN_TYPE_IPV6; default: - DCHECK(false) << "Invalid type."; + throw Exception(ErrorCode::INTERNAL_ERROR, + "runtime filter meet invalid PrimitiveType type {}", int(type)); } - DCHECK(false); - return PColumnType::COLUMN_TYPE_INT; } // PColumnType->PrimitiveType @@ -160,10 +163,14 @@ PrimitiveType to_primitive_type(PColumnType type) { return TYPE_CHAR; case PColumnType::COLUMN_TYPE_STRING: return TYPE_STRING; + case PColumnType::COLUMN_TYPE_IPV4: + return TYPE_IPV4; + case PColumnType::COLUMN_TYPE_IPV6: + return TYPE_IPV6; default: - DCHECK(false); + throw Exception(ErrorCode::INTERNAL_ERROR, + "runtime filter meet invalid PColumnType type {}", int(type)); } - return TYPE_INT; } // PFilterType -> RuntimeFilterType @@ -554,14 +561,13 @@ class RuntimePredicateWrapper { } Status assign(const PInFilter* in_filter, bool contain_null) { - PrimitiveType type = to_primitive_type(in_filter->column_type()); - _context->hybrid_set.reset(create_set(type)); + _context->hybrid_set.reset(create_set(_column_return_type)); if (contain_null) { _context->hybrid_set->set_null_aware(true); _context->hybrid_set->insert((const void*)nullptr); } - switch (type) { + switch (_column_return_type) { case TYPE_BOOLEAN: { batch_assign(in_filter, [](std::shared_ptr& set, PColumnValue& column) { bool bool_val = column.boolval(); @@ -701,9 +707,27 @@ class RuntimePredicateWrapper { }); break; } + case TYPE_IPV4: { + batch_assign(in_filter, [](std::shared_ptr& set, PColumnValue& column) { + int32_t tmp = column.intval(); + set->insert(&tmp); + }); + break; + } + case TYPE_IPV6: { + batch_assign(in_filter, [](std::shared_ptr& set, PColumnValue& column) { + auto string_val = column.stringval(); + StringParser::ParseResult result; + auto int128_val = StringParser::string_to_int( + string_val.c_str(), string_val.length(), &result); + DCHECK(result == StringParser::PARSE_SUCCESS); + set->insert(&int128_val); + }); + break; + } default: { return Status::InternalError("not support assign to in filter, type: " + - type_to_string(type)); + type_to_string(_column_return_type)); } } return Status::OK(); @@ -726,15 +750,14 @@ class RuntimePredicateWrapper { // used by shuffle runtime filter // assign this filter by protobuf Status assign(const PMinMaxFilter* minmax_filter, bool contain_null) { - PrimitiveType type = to_primitive_type(minmax_filter->column_type()); - _context->minmax_func.reset(create_minmax_filter(type)); + _context->minmax_func.reset(create_minmax_filter(_column_return_type)); if (contain_null) { _context->minmax_func->set_null_aware(true); _context->minmax_func->set_contain_null(); } - switch (type) { + switch (_column_return_type) { case TYPE_BOOLEAN: { bool min_val = minmax_filter->min_val().boolval(); bool max_val = minmax_filter->max_val().boolval(); @@ -850,6 +873,23 @@ class RuntimePredicateWrapper { auto max_val_ref = minmax_filter->max_val().stringval(); return _context->minmax_func->assign(&min_val_ref, &max_val_ref); } + case TYPE_IPV4: { + int tmp_min = minmax_filter->min_val().intval(); + int tmp_max = minmax_filter->max_val().intval(); + return _context->minmax_func->assign(&tmp_min, &tmp_max); + } + case TYPE_IPV6: { + auto min_string_val = minmax_filter->min_val().stringval(); + auto max_string_val = minmax_filter->max_val().stringval(); + StringParser::ParseResult result; + auto min_val = StringParser::string_to_int(min_string_val.c_str(), + min_string_val.length(), &result); + DCHECK(result == StringParser::PARSE_SUCCESS); + auto max_val = StringParser::string_to_int(max_string_val.c_str(), + max_string_val.length(), &result); + DCHECK(result == StringParser::PARSE_SUCCESS); + return _context->minmax_func->assign(&min_val, &max_val); + } default: break; } @@ -1140,7 +1180,7 @@ Status IRuntimeFilter::push_to_remote(const TNetworkAddress* addr) { merge_filter_request->set_filter_id(_filter_id); merge_filter_request->set_is_pipeline(true); auto column_type = _wrapper->column_type(); - merge_filter_request->set_column_type(to_proto(column_type)); + RETURN_IF_CATCH_EXCEPTION(merge_filter_request->set_column_type(to_proto(column_type))); merge_filter_callback->cntl_->set_timeout_ms(wait_time_ms()); if (get_ignored()) { @@ -1413,13 +1453,10 @@ template Status IRuntimeFilter::_create_wrapper(const T* param, std::unique_ptr* wrapper) { int filter_type = param->request->filter_type(); - PrimitiveType column_type = PrimitiveType::INVALID_TYPE; - if (param->request->has_in_filter()) { - column_type = to_primitive_type(param->request->in_filter().column_type()); - } - if (param->request->has_column_type()) { - column_type = to_primitive_type(param->request->column_type()); + if (!param->request->has_column_type()) { + return Status::InternalError("unknown filter column type"); } + PrimitiveType column_type = to_primitive_type(param->request->column_type()); *wrapper = std::make_unique(column_type, get_type(filter_type), param->request->filter_id()); @@ -1639,9 +1676,21 @@ void IRuntimeFilter::to_protobuf(PInFilter* filter) { }); return; } + case TYPE_IPV4: { + batch_copy(filter, it, [](PColumnValue* column, const IPv4* value) { + column->set_intval(*reinterpret_cast(value)); + }); + return; + } + case TYPE_IPV6: { + batch_copy(filter, it, [](PColumnValue* column, const IPv6* value) { + column->set_stringval(LargeIntValue::to_string(*value)); + }); + return; + } default: { - DCHECK(false) << "unknown type"; - break; + throw Exception(ErrorCode::INTERNAL_ERROR, + "runtime filter meet invalid PrimitiveType type {}", int(column_type)); } } } @@ -1755,9 +1804,22 @@ void IRuntimeFilter::to_protobuf(PMinMaxFilter* filter) { filter->mutable_max_val()->set_stringval(*max_string_value); break; } + case TYPE_IPV4: { + filter->mutable_min_val()->set_intval(*reinterpret_cast(min_data)); + filter->mutable_max_val()->set_intval(*reinterpret_cast(max_data)); + return; + } + case TYPE_IPV6: { + filter->mutable_min_val()->set_stringval( + LargeIntValue::to_string(*reinterpret_cast(min_data))); + filter->mutable_max_val()->set_stringval( + LargeIntValue::to_string(*reinterpret_cast(max_data))); + return; + } default: { - DCHECK(false) << "unknown type"; - break; + throw Exception(ErrorCode::INTERNAL_ERROR, + "runtime filter meet invalid PrimitiveType type {}", + int(_wrapper->column_type())); } } } diff --git a/be/src/olap/predicate_creator.h b/be/src/olap/predicate_creator.h index c47a1694f17a1d..1020e985248d72 100644 --- a/be/src/olap/predicate_creator.h +++ b/be/src/olap/predicate_creator.h @@ -242,7 +242,7 @@ std::unique_ptr> get_creator(const FieldType& ty case FieldType::OLAP_FIELD_TYPE_IPV4: { return std::make_unique>( [](const std::string& condition) { - vectorized::IPv4 value; + IPv4 value; bool res = IPv4Value::from_string(value, condition); DCHECK(res); return value; @@ -251,7 +251,7 @@ std::unique_ptr> get_creator(const FieldType& ty case FieldType::OLAP_FIELD_TYPE_IPV6: { return std::make_unique>( [](const std::string& condition) { - vectorized::IPv6 value; + IPv6 value; bool res = IPv6Value::from_string(value, condition); DCHECK(res); return value; diff --git a/be/src/runtime/large_int_value.h b/be/src/runtime/large_int_value.h index 7bc89ea765c150..fbe37ac1f409ab 100644 --- a/be/src/runtime/large_int_value.h +++ b/be/src/runtime/large_int_value.h @@ -24,6 +24,8 @@ #include #include +#include "olap/olap_common.h" + namespace doris { inline const __int128 MAX_INT128 = ~((__int128)0x01 << 127); @@ -36,6 +38,9 @@ class LargeIntValue { } static std::string to_string(__int128 value) { return fmt::format(FMT_COMPILE("{}"), value); } + static std::string to_string(__uint128_t value) { + return fmt::format(FMT_COMPILE("{}"), value); + } }; std::ostream& operator<<(std::ostream& os, __int128 const& value); diff --git a/be/src/runtime/primitive_type.h b/be/src/runtime/primitive_type.h index 59c0d91e432391..fb7c6e317f10a1 100644 --- a/be/src/runtime/primitive_type.h +++ b/be/src/runtime/primitive_type.h @@ -243,13 +243,13 @@ struct PrimitiveTypeTraits { }; template <> struct PrimitiveTypeTraits { - using CppType = vectorized::IPv4; + using CppType = IPv4; using StorageFieldType = CppType; using ColumnType = vectorized::ColumnIPv4; }; template <> struct PrimitiveTypeTraits { - using CppType = vectorized::IPv6; + using CppType = IPv6; using StorageFieldType = CppType; using ColumnType = vectorized::ColumnIPv6; }; diff --git a/be/src/vec/core/types.h b/be/src/vec/core/types.h index 73025b166e5308..f5943fa6e1d31f 100644 --- a/be/src/vec/core/types.h +++ b/be/src/vec/core/types.h @@ -42,6 +42,9 @@ struct decimal12_t; struct uint24_t; struct StringRef; +using IPv4 = uint32_t; +using IPv6 = uint128_t; + namespace vectorized { /// Data types for representing elementary values from a database in RAM. @@ -296,9 +299,6 @@ struct TypeId { /// Not a data type in database, defined just for convenience. using Strings = std::vector; -using IPv4 = uint32_t; -using IPv6 = uint128_t; - template <> inline constexpr bool IsNumber = true; template <> diff --git a/be/src/vec/data_types/data_type_ipv6.cpp b/be/src/vec/data_types/data_type_ipv6.cpp index 4a81727870e4cf..2d524a71728a0b 100755 --- a/be/src/vec/data_types/data_type_ipv6.cpp +++ b/be/src/vec/data_types/data_type_ipv6.cpp @@ -50,7 +50,7 @@ std::string DataTypeIPv6::to_string(const IColumn& column, size_t row_num) const return value.to_string(); } -std::string DataTypeIPv6::to_string(const IPv6& ipv6_val) const { +std::string DataTypeIPv6::to_string(const IPv6& ipv6_val) { auto value = IPv6Value(ipv6_val); return value.to_string(); } diff --git a/be/src/vec/data_types/data_type_ipv6.h b/be/src/vec/data_types/data_type_ipv6.h index 89a42aca862c11..9196e9c936a19f 100755 --- a/be/src/vec/data_types/data_type_ipv6.h +++ b/be/src/vec/data_types/data_type_ipv6.h @@ -63,7 +63,7 @@ class DataTypeIPv6 final : public DataTypeNumberBase { void push_number(ColumnString::Chars& chars, const IPv6& num) const; std::string to_string(const IColumn& column, size_t row_num) const override; void to_string(const IColumn& column, size_t row_num, BufferWritable& ostr) const override; - std::string to_string(const IPv6& value) const; + static std::string to_string(const IPv6& value); Status from_string(ReadBuffer& rb, IColumn* column) const override; Field get_field(const TExprNode& node) const override { diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp index 5cb0607411d469..8dc0fb49a1abeb 100644 --- a/be/src/vec/exprs/vexpr.cpp +++ b/be/src/vec/exprs/vexpr.cpp @@ -31,6 +31,7 @@ #include "common/config.h" #include "common/exception.h" #include "common/status.h" +#include "runtime/define_primitive_type.h" #include "vec/columns/column_vector.h" #include "vec/columns/columns_number.h" #include "vec/data_types/data_type_factory.hpp" @@ -147,9 +148,17 @@ TExprNode create_texpr_node_from(const void* data, const PrimitiveType& type, in THROW_IF_ERROR(create_texpr_literal_node(data, &node)); break; } + case TYPE_IPV4: { + THROW_IF_ERROR(create_texpr_literal_node(data, &node)); + break; + } + case TYPE_IPV6: { + THROW_IF_ERROR(create_texpr_literal_node(data, &node)); + break; + } default: - DCHECK(false); - throw std::invalid_argument("Invalid type!"); + throw Exception(ErrorCode::INTERNAL_ERROR, "runtime filter meet invalid type {}", + int(type)); } return node; } diff --git a/be/src/vec/exprs/vexpr.h b/be/src/vec/exprs/vexpr.h index 110748efab3f9e..4720068a3aaf58 100644 --- a/be/src/vec/exprs/vexpr.h +++ b/be/src/vec/exprs/vexpr.h @@ -39,8 +39,10 @@ #include "vec/columns/column.h" #include "vec/core/block.h" #include "vec/core/column_with_type_and_name.h" +#include "vec/core/types.h" #include "vec/core/wide_integer.h" #include "vec/data_types/data_type.h" +#include "vec/data_types/data_type_ipv6.h" #include "vec/exprs/vexpr_fwd.h" #include "vec/functions/function.h" @@ -476,6 +478,20 @@ Status create_texpr_literal_node(const void* data, TExprNode* node, int precisio string_literal.__set_value(*origin_value); (*node).__set_string_literal(string_literal); (*node).__set_type(create_type_desc(PrimitiveType::TYPE_STRING)); + } else if constexpr (T == TYPE_IPV4) { + const auto* origin_value = reinterpret_cast(data); + (*node).__set_node_type(TExprNodeType::IPV4_LITERAL); + TIPv4Literal literal; + literal.__set_value(*origin_value); + (*node).__set_ipv4_literal(literal); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_IPV4)); + } else if constexpr (T == TYPE_IPV6) { + const auto* origin_value = reinterpret_cast(data); + (*node).__set_node_type(TExprNodeType::IPV6_LITERAL); + TIPv6Literal literal; + literal.__set_value(vectorized::DataTypeIPv6::to_string(*origin_value)); + (*node).__set_ipv6_literal(literal); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_IPV6)); } else { return Status::InvalidArgument("Invalid argument type!"); } diff --git a/be/src/vec/olap/olap_data_convertor.cpp b/be/src/vec/olap/olap_data_convertor.cpp index 5c8e09766e635f..8dcdd977d9a57f 100644 --- a/be/src/vec/olap/olap_data_convertor.cpp +++ b/be/src/vec/olap/olap_data_convertor.cpp @@ -184,10 +184,10 @@ OlapBlockDataConvertor::create_olap_column_data_convertor(const TabletColumn& co return std::make_unique>(); } case FieldType::OLAP_FIELD_TYPE_IPV4: { - return std::make_unique>(); + return std::make_unique>(); } case FieldType::OLAP_FIELD_TYPE_IPV6: { - return std::make_unique>(); + return std::make_unique>(); } case FieldType::OLAP_FIELD_TYPE_FLOAT: { return std::make_unique>(); diff --git a/be/src/vec/runtime/ipv4_value.h b/be/src/vec/runtime/ipv4_value.h index ffce9e2bb67641..76ae288e5be407 100644 --- a/be/src/vec/runtime/ipv4_value.h +++ b/be/src/vec/runtime/ipv4_value.h @@ -32,19 +32,19 @@ class IPv4Value { public: IPv4Value() = default; - explicit IPv4Value(vectorized::IPv4 ipv4) { _value = ipv4; } + explicit IPv4Value(IPv4 ipv4) { _value = ipv4; } - const vectorized::IPv4& value() const { return _value; } + const IPv4& value() const { return _value; } - vectorized::IPv4& value() { return _value; } + IPv4& value() { return _value; } - void set_value(vectorized::IPv4 ipv4) { _value = ipv4; } + void set_value(IPv4 ipv4) { _value = ipv4; } bool from_string(const std::string& ipv4_str) { return from_string(_value, ipv4_str); } std::string to_string() const { return to_string(_value); } - static bool from_string(vectorized::IPv4& value, const char* ipv4_str, size_t len) { + static bool from_string(IPv4& value, const char* ipv4_str, size_t len) { if (len == 0) { return false; } @@ -61,15 +61,15 @@ class IPv4Value { reinterpret_cast(&parse_value))) { return false; } - value = static_cast(parse_value); + value = static_cast(parse_value); return true; } - static bool from_string(vectorized::IPv4& value, const std::string& ipv4_str) { + static bool from_string(IPv4& value, const std::string& ipv4_str) { return from_string(value, ipv4_str.c_str(), ipv4_str.size()); } - static std::string to_string(vectorized::IPv4 value) { + static std::string to_string(IPv4 value) { char buf[IPV4_MAX_TEXT_LENGTH + 1]; char* start = buf; char* end = buf; @@ -97,7 +97,7 @@ class IPv4Value { } private: - vectorized::IPv4 _value; + IPv4 _value; }; } // namespace doris diff --git a/be/src/vec/runtime/ipv6_value.h b/be/src/vec/runtime/ipv6_value.h index 953b71baf84ef2..5a26af25243a0f 100644 --- a/be/src/vec/runtime/ipv6_value.h +++ b/be/src/vec/runtime/ipv6_value.h @@ -32,17 +32,17 @@ class IPv6Value { public: IPv6Value() { _value = 0; } - explicit IPv6Value(vectorized::IPv6 ipv6) { _value = ipv6; } + explicit IPv6Value(IPv6 ipv6) { _value = ipv6; } - const vectorized::IPv6& value() const { return _value; } + const IPv6& value() const { return _value; } - vectorized::IPv6& value() { return _value; } + IPv6& value() { return _value; } - void set_value(vectorized::IPv6 ipv6) { _value = ipv6; } + void set_value(IPv6 ipv6) { _value = ipv6; } bool from_string(const std::string& ipv6_str) { return from_string(_value, ipv6_str); } - static bool from_string(vectorized::IPv6& value, const char* ipv6_str, size_t len) { + static bool from_string(IPv6& value, const char* ipv6_str, size_t len) { if (len == 0) { return false; } @@ -59,13 +59,13 @@ class IPv6Value { reinterpret_cast(&value)); } - static bool from_string(vectorized::IPv6& value, const std::string& ipv6_str) { + static bool from_string(IPv6& value, const std::string& ipv6_str) { return from_string(value, ipv6_str.c_str(), ipv6_str.size()); } std::string to_string() const { return to_string(_value); } - static std::string to_string(vectorized::IPv6 value) { + static std::string to_string(IPv6 value) { char buf[IPV6_MAX_TEXT_LENGTH + 1]; char* start = buf; char* end = buf; @@ -80,7 +80,7 @@ class IPv6Value { if (len == 0 || len > IPV6_MAX_TEXT_LENGTH) { return false; } - vectorized::IPv6 value; + IPv6 value; size_t begin = 0; size_t end = len - 1; while (begin < len && std::isspace(ipv6_str[begin])) { @@ -94,7 +94,7 @@ class IPv6Value { } private: - vectorized::IPv6 _value; + IPv6 _value; }; } // namespace doris diff --git a/be/test/vec/data_types/serde/data_type_serde_mysql_test.cpp b/be/test/vec/data_types/serde/data_type_serde_mysql_test.cpp index 97e78f05c549a8..f05919e4a8f477 100644 --- a/be/test/vec/data_types/serde/data_type_serde_mysql_test.cpp +++ b/be/test/vec/data_types/serde/data_type_serde_mysql_test.cpp @@ -250,7 +250,7 @@ void serialize_and_deserialize_mysql_test() { case TYPE_IPV4: tslot.__set_slotType(type_desc.to_thrift()); { - auto column_vector_ipv4 = vectorized::ColumnVector::create(); + auto column_vector_ipv4 = vectorized::ColumnVector::create(); auto& ipv4_data = column_vector_ipv4->get_data(); for (int i = 0; i < row_num; ++i) { IPv4Value ipv4_value; @@ -267,7 +267,7 @@ void serialize_and_deserialize_mysql_test() { case TYPE_IPV6: tslot.__set_slotType(type_desc.to_thrift()); { - auto column_vector_ipv6 = vectorized::ColumnVector::create(); + auto column_vector_ipv6 = vectorized::ColumnVector::create(); auto& ipv6_data = column_vector_ipv6->get_data(); for (int i = 0; i < row_num; ++i) { IPv6Value ipv6_value; diff --git a/be/test/vec/runtime/ip_value_test.cpp b/be/test/vec/runtime/ip_value_test.cpp index cbeebcf24b67d9..f3b3b9ac9cd39f 100644 --- a/be/test/vec/runtime/ip_value_test.cpp +++ b/be/test/vec/runtime/ip_value_test.cpp @@ -49,8 +49,8 @@ static void print_bytes(T num) { TEST(IPValueTest, IPv4ValueTest) { const std::string ipv4_str1 = "192.168.103.254"; const std::string ipv4_str2 = "193.168.103.255"; - vectorized::IPv4 ipv4_val1; - vectorized::IPv4 ipv4_val2; + IPv4 ipv4_val1; + IPv4 ipv4_val2; ASSERT_TRUE(IPv4Value::from_string(ipv4_val1, ipv4_str1.c_str(), ipv4_str1.size())); ASSERT_TRUE(IPv4Value::from_string(ipv4_val2, ipv4_str2.c_str(), ipv4_str2.size())); ASSERT_TRUE(ipv4_val1 < ipv4_val2); @@ -65,8 +65,8 @@ TEST(IPValueTest, IPv4ValueTest) { TEST(IPValueTest, IPv6ValueTest) { const std::string ipv6_str1 = "2001:418:0:5000::c2d"; const std::string ipv6_str2 = "2001:428::205:171:200:230"; - vectorized::IPv6 ipv6_val1; - vectorized::IPv6 ipv6_val2; + IPv6 ipv6_val1; + IPv6 ipv6_val2; ASSERT_TRUE(IPv6Value::from_string(ipv6_val1, ipv6_str1.c_str(), ipv6_str1.size())); ASSERT_TRUE(IPv6Value::from_string(ipv6_val2, ipv6_str2.c_str(), ipv6_str2.size())); ASSERT_TRUE(ipv6_val1 < ipv6_val2); @@ -91,12 +91,12 @@ static void apply_cidr_mask(const char* __restrict src, char* __restrict dst_low TEST(IPValueTest, IPv6CIDRTest) { const std::string ipv6_str1 = "2001:0db8:0000:85a3:0000:0000:ac1f:8001"; const std::string ipv6_str2 = "2001:0db8:0000:85a3:ffff:ffff:ffff:ffff"; - vectorized::IPv6 ipv6_val1; // little-endian - vectorized::IPv6 ipv6_val2; // little-endian + IPv6 ipv6_val1; // little-endian + IPv6 ipv6_val2; // little-endian ASSERT_TRUE(IPv6Value::from_string(ipv6_val1, ipv6_str1.c_str(), ipv6_str1.size())); ASSERT_TRUE(IPv6Value::from_string(ipv6_val2, ipv6_str2.c_str(), ipv6_str2.size())); - vectorized::IPv6 min_range1, max_range1; - vectorized::IPv6 min_range2, max_range2; + IPv6 min_range1, max_range1; + IPv6 min_range2, max_range2; apply_cidr_mask(reinterpret_cast(&ipv6_val1), reinterpret_cast(&min_range1), reinterpret_cast(&max_range1), 0); apply_cidr_mask(reinterpret_cast(&ipv6_val2), reinterpret_cast(&min_range2), diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto index e188001e49103f..4ac4fd24f3b7e5 100644 --- a/gensrc/proto/internal_service.proto +++ b/gensrc/proto/internal_service.proto @@ -517,16 +517,18 @@ enum PColumnType { COLUMN_TYPE_DECIMAL64 = 18; COLUMN_TYPE_DECIMAL128I = 19; COLUMN_TYPE_DECIMAL256 = 20; + COLUMN_TYPE_IPV4 = 21; + COLUMN_TYPE_IPV6 = 22; } message PMinMaxFilter { - required PColumnType column_type = 1; + required PColumnType column_type = 1; // Deprecated required PColumnValue min_val = 2; required PColumnValue max_val = 3; }; message PInFilter { - required PColumnType column_type = 1; + required PColumnType column_type = 1; // Deprecated repeated PColumnValue values = 2; optional string ignored_msg = 3; } diff --git a/regression-test/data/datatype_p0/ip/ip_rf/test_ip_rf.out b/regression-test/data/datatype_p0/ip/ip_rf/test_ip_rf.out new file mode 100644 index 00000000000000..7e8a59791f636e --- /dev/null +++ b/regression-test/data/datatype_p0/ip/ip_rf/test_ip_rf.out @@ -0,0 +1,31 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +2 + +-- !sql -- +2 + +-- !sql -- +2 + +-- !sql -- +2 + +-- !sql -- +2 + +-- !sql -- +2 + +-- !sql -- +2 + +-- !sql -- +2 + +-- !sql -- +2 + +-- !sql -- +2 + diff --git a/regression-test/suites/datatype_p0/ip/ip_rf/test_ip_rf.groovy b/regression-test/suites/datatype_p0/ip/ip_rf/test_ip_rf.groovy new file mode 100644 index 00000000000000..c7f6c30481eb0b --- /dev/null +++ b/regression-test/suites/datatype_p0/ip/ip_rf/test_ip_rf.groovy @@ -0,0 +1,73 @@ + +// 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_ip_rf") { + sql """ DROP TABLE IF EXISTS ip_test """ + sql """ DROP TABLE IF EXISTS ip_test2 """ + sql """ + CREATE TABLE ip_test ( + `id` int, + `ip_v4` ipv4, + `ip_v6` ipv6 + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`id`) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql """ + CREATE TABLE ip_test2 ( + `id` int, + `ip_v4` ipv4, + `ip_v6` ipv6 + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`id`) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql """ + insert into ip_test values(1, '0.0.0.0','2001:16a0:2:200a::2'),(2, '127.0.0.1','2a02:e980:83:5b09:ecb8:c669:b336:650e'); + """ + + sql """ + insert into ip_test2 values(1, '0.0.0.0','2001:16a0:2:200a::2'),(2, '127.0.0.1','2a02:e980:83:5b09:ecb8:c669:b336:650e'),(3, '59.50.185.152','2001:4888:1f:e891:161:26::'),(4, '255.255.255.255','2001:4888:1f:e891:161:26::'); + + """ + + sql "ANALYZE TABLE ip_test WITH sync;" + sql "ANALYZE TABLE ip_test2 WITH sync;" + + sql "set runtime_filter_type=0;" + qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v4=b.ip_v4;" + qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v6=b.ip_v6;" + sql "set runtime_filter_type=1;" + qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v4=b.ip_v4;" + qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v6=b.ip_v6;" + sql "set runtime_filter_type=2;" + qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v4=b.ip_v4;" + qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v6=b.ip_v6;" + sql "set runtime_filter_type=4;" + qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v4=b.ip_v4;" + qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v6=b.ip_v6;" + sql "set runtime_filter_type=8;" + qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v4=b.ip_v4;" + qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v6=b.ip_v6;" + +} From 901f9706fcd58d8436658739d32da2a312a6253d Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Fri, 30 Aug 2024 13:56:05 +0800 Subject: [PATCH 43/60] [fix](Nereids) datetimev2 literal equals should compare microsecond (#40121) --- .../trees/expressions/literal/DateTimeV2Literal.java | 7 ++++++- .../trees/expressions/literal/DateTimeLiteralTest.java | 10 ++++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) 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 a3457f2463d65c..0ca19bf2a92715 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 @@ -104,6 +104,11 @@ public LiteralExpr toLegacyLiteral() { getDataType().toCatalogDataType()); } + @Override + public double getDouble() { + return super.getDouble() + microSecond / 1000000.0; + } + @Override public String toString() { return getStringValue(); @@ -291,6 +296,6 @@ public boolean equals(Object o) { return false; } DateTimeV2Literal literal = (DateTimeV2Literal) o; - return Objects.equals(dataType, literal.dataType); + return Objects.equals(dataType, literal.dataType) && Objects.equals(microSecond, literal.microSecond); } } 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 2eeee30a38a239..914246f08b6e47 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 @@ -490,4 +490,14 @@ void testRoundCeiling() { Assertions.assertEquals(1, literal.roundCeiling(0).month); Assertions.assertEquals(2001, literal.roundCeiling(0).year); } + + @Test + void testEquals() { + DateTimeV2Literal l1 = new DateTimeV2Literal(1, 1, 1, 1, 1, 1, 1); + DateTimeV2Literal l2 = new DateTimeV2Literal(1, 1, 1, 1, 1, 1, 1); + DateTimeV2Literal l3 = new DateTimeV2Literal(1, 1, 1, 1, 1, 1, 2); + + Assertions.assertEquals(l1, l2); + Assertions.assertNotEquals(l1, l3); + } } From d413b0be24766ea3913a1513e6838eaa6732941f Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Fri, 30 Aug 2024 14:16:09 +0800 Subject: [PATCH 44/60] [opt](Nereids) support some statements syntax (#40151) - copy into - backup - help - install plugin - uninstall plugin - lock tables - unlock tables - recover - start transaction --- .../org/apache/doris/nereids/DorisParser.g4 | 46 ++++++++++++++++++- .../org/apache/doris/analysis/BackupStmt.java | 2 +- .../org/apache/doris/analysis/CopyStmt.java | 2 +- .../org/apache/doris/analysis/HelpStmt.java | 2 +- .../doris/analysis/InstallPluginStmt.java | 2 +- .../apache/doris/analysis/LockTablesStmt.java | 2 +- .../apache/doris/analysis/RecoverDbStmt.java | 2 +- .../doris/analysis/RecoverPartitionStmt.java | 2 +- .../doris/analysis/RecoverTableStmt.java | 2 +- .../apache/doris/analysis/RestoreStmt.java | 2 +- .../doris/analysis/UninstallPluginStmt.java | 2 +- .../doris/analysis/UnlockTablesStmt.java | 2 +- .../doris/analysis/UnsupportedStmt.java | 2 +- .../doris/analysis/WarmUpClusterStmt.java | 2 +- 14 files changed, 58 insertions(+), 14 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 f79088b73aacd3..32f5293c00e745 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 @@ -69,6 +69,8 @@ unsupportedStatement | unsupportedGrantRevokeStatement | unsupportedAdminStatement | unsupportedTransactionStatement + | unsupportedRecoverStatement + | unsupportedOtherStatement ; materailizedViewStatement @@ -172,6 +174,40 @@ supportedDropStatement : DROP CATALOG RECYCLE BIN WHERE idType=STRING_LITERAL EQ id=INTEGER_VALUE #dropCatalogRecycleBin ; +unsupportedOtherStatement + : HELP mark=identifierOrText #help + | INSTALL PLUGIN FROM source=identifierOrText properties=propertyClause? #installPlugin + | UNINSTALL PLUGIN name=identifierOrText #uninstallPlugin + | LOCK TABLES (lockTable (COMMA lockTable)*)? #lockTables + | UNLOCK TABLES #unlockTables + | WARM UP CLUSTER destination=identifier WITH + (CLUSTER source=identifier | (warmUpItem (COMMA warmUpItem)*)) FORCE? #warmUpCluster + | BACKUP SNAPSHOT label=multipartIdentifier TO repo=identifier + ((ON | EXCLUDE) LEFT_PAREN baseTableRef (COMMA baseTableRef)* RIGHT_PAREN)? + properties=propertyClause? #backup + | RESTORE SNAPSHOT label=multipartIdentifier FROM repo=identifier + ((ON | EXCLUDE) LEFT_PAREN baseTableRef (COMMA baseTableRef)* RIGHT_PAREN)? + properties=propertyClause? #restore + | START TRANSACTION (WITH CONSISTENT SNAPSHOT)? #unsupportedStartTransaction + ; + +warmUpItem + : TABLE tableName=multipartIdentifier (PARTITION partitionName=identifier)? + ; + +lockTable + : name=multipartIdentifier (AS alias=identifierOrText)? + (READ (LOCAL)? | (LOW_PRIORITY)? WRITE) + ; + +unsupportedRecoverStatement + : RECOVER DATABASE name=identifier id=INTEGER_VALUE? (AS alias=identifier)? #recoverDatabase + | RECOVER TABLE name=multipartIdentifier + id=INTEGER_VALUE? (AS alias=identifier)? #recoverTable + | RECOVER PARTITION name=identifier id=INTEGER_VALUE? (AS alias=identifier)? + FROM tableName=multipartIdentifier #recoverPartition + ; + unsupportedAdminStatement : ADMIN SHOW REPLICA STATUS FROM baseTableRef wildWhere? #adminShowReplicaStatus | ADMIN SHOW REPLICA DISTRIBUTION FROM baseTableRef #adminShowReplicaDistribution @@ -551,7 +587,15 @@ unsupportedUseStatement ; unsupportedDmlStatement - : TRUNCATE TABLE multipartIdentifier specifiedPartition? # truncateTable + : TRUNCATE TABLE multipartIdentifier specifiedPartition? #truncateTable + | COPY INTO selectHint? name=multipartIdentifier columns=identifierList FROM + (stageAndPattern | (LEFT_PAREN SELECT selectColumnClause + FROM stageAndPattern whereClause RIGHT_PAREN)) + properties=propertyClause? #copyInto + ; + +stageAndPattern + : AT (stage=identifier | TILDE) (pattern=STRING_LITERAL)? ; unsupportedKillStatement diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BackupStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BackupStmt.java index 84b3171c2c134e..445db61157061d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BackupStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BackupStmt.java @@ -27,7 +27,7 @@ import java.util.Map; -public class BackupStmt extends AbstractBackupStmt { +public class BackupStmt extends AbstractBackupStmt implements NotFallbackInParser { private static final String PROP_TYPE = "type"; public static final String PROP_CONTENT = "content"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CopyStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CopyStmt.java index 80ba68ac575507..db94c106072f50 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CopyStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CopyStmt.java @@ -51,7 +51,7 @@ /** * Copy statement */ -public class CopyStmt extends DdlStmt { +public class CopyStmt extends DdlStmt implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(CopyStmt.class); private static final ShowResultSetMetaData COPY_INTO_META_DATA = diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/HelpStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/HelpStmt.java index c44687ac615ce3..f4c75ab48fd2b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/HelpStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/HelpStmt.java @@ -24,7 +24,7 @@ import com.google.common.base.Strings; -public class HelpStmt extends ShowStmt { +public class HelpStmt extends ShowStmt implements NotFallbackInParser { private static final ShowResultSetMetaData TOPIC_META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("name", ScalarType.createVarchar(64))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/InstallPluginStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/InstallPluginStmt.java index 3d5db0c884097d..ef625052107895 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/InstallPluginStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/InstallPluginStmt.java @@ -29,7 +29,7 @@ import java.util.Map; -public class InstallPluginStmt extends DdlStmt { +public class InstallPluginStmt extends DdlStmt implements NotFallbackInParser { private String pluginPath; private Map properties; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/LockTablesStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/LockTablesStmt.java index 803d52bf3fb39d..0f618667ebc275 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/LockTablesStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/LockTablesStmt.java @@ -31,7 +31,7 @@ import java.util.ArrayList; import java.util.List; -public class LockTablesStmt extends StatementBase { +public class LockTablesStmt extends StatementBase implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(LockTablesStmt.class); private List lockTables; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/RecoverDbStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/RecoverDbStmt.java index cfb4ecf4087e46..ef2805b0d9aa9c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/RecoverDbStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/RecoverDbStmt.java @@ -28,7 +28,7 @@ import com.google.common.base.Strings; -public class RecoverDbStmt extends DdlStmt { +public class RecoverDbStmt extends DdlStmt implements NotFallbackInParser { private String dbName; private long dbId = -1; private String newDbName = ""; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/RecoverPartitionStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/RecoverPartitionStmt.java index 05db7a81e83d75..def1b57f1b8ee4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/RecoverPartitionStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/RecoverPartitionStmt.java @@ -28,7 +28,7 @@ import com.google.common.base.Strings; -public class RecoverPartitionStmt extends DdlStmt { +public class RecoverPartitionStmt extends DdlStmt implements NotFallbackInParser { private TableName dbTblName; private String partitionName; private long partitionId = -1; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/RecoverTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/RecoverTableStmt.java index 56dfbbfc2c7c06..978a17dcd062f0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/RecoverTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/RecoverTableStmt.java @@ -28,7 +28,7 @@ import com.google.common.base.Strings; -public class RecoverTableStmt extends DdlStmt { +public class RecoverTableStmt extends DdlStmt implements NotFallbackInParser { private TableName dbTblName; private long tableId = -1; private String newTableName = ""; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java index 456916470013e0..35120b5fa11bcc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java @@ -33,7 +33,7 @@ import java.util.Map; import java.util.Set; -public class RestoreStmt extends AbstractBackupStmt { +public class RestoreStmt extends AbstractBackupStmt implements NotFallbackInParser { private static final String PROP_ALLOW_LOAD = "allow_load"; private static final String PROP_BACKUP_TIMESTAMP = "backup_timestamp"; private static final String PROP_META_VERSION = "meta_version"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/UninstallPluginStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/UninstallPluginStmt.java index c87456dcb37475..0ee0c28906bc3e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/UninstallPluginStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/UninstallPluginStmt.java @@ -26,7 +26,7 @@ import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; -public class UninstallPluginStmt extends DdlStmt { +public class UninstallPluginStmt extends DdlStmt implements NotFallbackInParser { private String pluginName; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/UnlockTablesStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/UnlockTablesStmt.java index 4c66198c596b62..0c37240ff3c01b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/UnlockTablesStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/UnlockTablesStmt.java @@ -17,7 +17,7 @@ package org.apache.doris.analysis; -public class UnlockTablesStmt extends StatementBase { +public class UnlockTablesStmt extends StatementBase implements NotFallbackInParser { @Override public String toSql() { return "UNLOCK TABLES"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/UnsupportedStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/UnsupportedStmt.java index fa2d46782e8200..8316b293f01a25 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/UnsupportedStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/UnsupportedStmt.java @@ -20,7 +20,7 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; -public class UnsupportedStmt extends StatementBase { +public class UnsupportedStmt extends StatementBase implements NotFallbackInParser { public UnsupportedStmt() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/WarmUpClusterStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/WarmUpClusterStmt.java index 46aca93c9ee325..9f386a686a298a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/WarmUpClusterStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/WarmUpClusterStmt.java @@ -39,7 +39,7 @@ import java.util.Map; import java.util.Objects; -public class WarmUpClusterStmt extends StatementBase { +public class WarmUpClusterStmt extends StatementBase implements NotFallbackInParser { private static final Logger LOG = LogManager.getLogger(WarmUpClusterStmt.class); private List> tableList; private List> tables = new ArrayList<>(); From 37c5848e7dd7718167e1e62e4cab06561d370282 Mon Sep 17 00:00:00 2001 From: Gabriel Date: Fri, 30 Aug 2024 14:20:11 +0800 Subject: [PATCH 45/60] [fix](join) Fix partitioned hash join strategy (#40163) ## Proposed changes Issue Number: close #xxx --- be/src/pipeline/pipeline_fragment_context.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index 19d94997531497..51cfd73e549d79 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -1333,6 +1333,8 @@ Status PipelineFragmentContext::_create_operator(ObjectPool* pool, const TPlanNo _pipeline_parent_map.push(op->node_id(), cur_pipe); _pipeline_parent_map.push(op->node_id(), build_side_pipe); + sink->set_followed_by_shuffled_join(sink->is_shuffled_hash_join()); + op->set_followed_by_shuffled_join(op->is_shuffled_hash_join()); } else { op.reset(new HashJoinProbeOperatorX(pool, tnode, next_operator_id(), descs)); RETURN_IF_ERROR(cur_pipe->add_operator(op)); @@ -1353,6 +1355,8 @@ Status PipelineFragmentContext::_create_operator(ObjectPool* pool, const TPlanNo _pipeline_parent_map.push(op->node_id(), cur_pipe); _pipeline_parent_map.push(op->node_id(), build_side_pipe); + sink->set_followed_by_shuffled_join(sink->is_shuffled_hash_join()); + op->set_followed_by_shuffled_join(op->is_shuffled_hash_join()); } _require_bucket_distribution = _require_bucket_distribution || op->require_data_distribution(); From 212b82f00bc6fc1f453dbc702e970c780d4554f2 Mon Sep 17 00:00:00 2001 From: slothever <18522955+wsjz@users.noreply.github.com> Date: Fri, 30 Aug 2024 14:48:27 +0800 Subject: [PATCH 46/60] [fix](multi-catalog)fix max compute null parts table read (#40046) ## Proposed changes fix max compute null parts table read if no any data in a part, we should make mc jni scanner return 0 rows --- .../java/org/apache/doris/maxcompute/MaxComputeJniScanner.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeJniScanner.java b/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeJniScanner.java index 89c351f7343e80..9c76a330a644e4 100644 --- a/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeJniScanner.java +++ b/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeJniScanner.java @@ -170,6 +170,9 @@ public void open() throws IOException { // query columns required non-null, when query partition table pushDownColumns.add(session.getSchema().getColumn(0)); } + if (totalRows == 0) { + return; + } arrowAllocator = new RootAllocator(Integer.MAX_VALUE); curReader = session.openArrowRecordReader(start, totalRows, pushDownColumns, arrowAllocator); remainBatchRows = totalRows; From 55de454a0c9bac16bd01ec7ea2b0a09a0d9b3ec2 Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Fri, 30 Aug 2024 15:03:24 +0800 Subject: [PATCH 47/60] [Feat](Job)After a job is paused, it can be manually triggered to execute. (#39565) ## Proposed changes - After a job is paused, it can be manually triggered to execute. - Update the return fields of the Insert task query to include a new start time field. --- .../apache/doris/job/base/AbstractJob.java | 26 ++++++++++++++----- .../job/extensions/insert/InsertTask.java | 8 +++++- .../doris/job/scheduler/JobScheduler.java | 5 ++-- 3 files changed, 29 insertions(+), 10 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java index 3f595d6daf5362..94a0b0146cd514 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java @@ -212,8 +212,9 @@ public List queryAllTasks() { } public List commonCreateTasks(TaskType taskType, C taskContext) { - if (!getJobStatus().equals(JobStatus.RUNNING)) { - log.warn("job is not running, job id is {}", jobId); + if (!canCreateTask(taskType)) { + log.info("job is not ready for scheduling, job id is {},job status is {}, taskType is {}", jobId, + jobStatus, taskType); return new ArrayList<>(); } if (!isReadyForScheduling(taskContext)) { @@ -235,6 +236,19 @@ public List commonCreateTasks(TaskType taskType, C taskContext) { } } + private boolean canCreateTask(TaskType taskType) { + JobStatus currentJobStatus = getJobStatus(); + + switch (taskType) { + case SCHEDULED: + return currentJobStatus.equals(JobStatus.RUNNING); + case MANUAL: + return currentJobStatus.equals(JobStatus.RUNNING) || currentJobStatus.equals(JobStatus.PAUSED); + default: + throw new IllegalArgumentException("Unsupported TaskType: " + taskType); + } + } + public void initTasks(Collection tasks, TaskType taskType) { tasks.forEach(task -> { task.setTaskType(taskType); @@ -307,7 +321,7 @@ public void logUpdateOperation() { @Override public void onTaskFail(T task) throws JobException { failedTaskCount.incrementAndGet(); - updateJobStatusIfEnd(false); + updateJobStatusIfEnd(false, task.getTaskType()); runningTasks.remove(task); logUpdateOperation(); } @@ -315,16 +329,16 @@ public void onTaskFail(T task) throws JobException { @Override public void onTaskSuccess(T task) throws JobException { succeedTaskCount.incrementAndGet(); - updateJobStatusIfEnd(true); + updateJobStatusIfEnd(true, task.getTaskType()); runningTasks.remove(task); logUpdateOperation(); } - private void updateJobStatusIfEnd(boolean taskSuccess) throws JobException { + private void updateJobStatusIfEnd(boolean taskSuccess, TaskType taskType) throws JobException { JobExecuteType executeType = getJobConfig().getExecuteType(); - if (executeType.equals(JobExecuteType.MANUAL)) { + if (executeType.equals(JobExecuteType.MANUAL) || taskType.equals(TaskType.MANUAL)) { return; } switch (executeType) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java index a8317ada51e705..266f3f8476f715 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java @@ -58,6 +58,7 @@ public class InsertTask extends AbstractTask { new Column("Status", ScalarType.createStringType()), new Column("ErrorMsg", ScalarType.createStringType()), new Column("CreateTime", ScalarType.createStringType()), + new Column("StartTime", ScalarType.createStringType()), new Column("FinishTime", ScalarType.createStringType()), new Column("TrackingUrl", ScalarType.createStringType()), new Column("LoadStatistic", ScalarType.createStringType()), @@ -246,6 +247,8 @@ public TRow getTvfInfo(String jobName) { trow.addToColumnValue(new TCell().setStringVal(errorMsg)); // create time trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(getCreateTimeMs()))); + trow.addToColumnValue(new TCell().setStringVal(null == getStartTimeMs() ? "" + : TimeUtils.longToTimeString(getStartTimeMs()))); // load end time trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(getFinishTimeMs()))); // tracking url @@ -273,7 +276,10 @@ private TRow getPendingTaskTVFInfo(String jobName) { trow.addToColumnValue(new TCell().setStringVal(getStatus().name())); trow.addToColumnValue(new TCell().setStringVal("")); trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(getCreateTimeMs()))); - trow.addToColumnValue(new TCell().setStringVal("")); + trow.addToColumnValue(new TCell().setStringVal(null == getStartTimeMs() ? "" + : TimeUtils.longToTimeString(getStartTimeMs()))); + trow.addToColumnValue(new TCell().setStringVal(null == getFinishTimeMs() ? "" + : TimeUtils.longToTimeString(getFinishTimeMs()))); trow.addToColumnValue(new TCell().setStringVal("")); trow.addToColumnValue(new TCell().setStringVal("")); trow.addToColumnValue(new TCell().setStringVal(userIdentity.getQualifiedUser())); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java index 7f0133bf957e77..33d12c30a4b76f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java @@ -192,10 +192,9 @@ private void executeTimerJobIdsWithinLastTenMinutesWindow() { clearEndJob(job); continue; } - if (!job.getJobStatus().equals(JobStatus.RUNNING) && !job.getJobConfig().checkIsTimerJob()) { - continue; + if (job.getJobStatus().equals(JobStatus.RUNNING) && job.getJobConfig().checkIsTimerJob()) { + cycleTimerJobScheduler(job, lastTimeWindowMs); } - cycleTimerJobScheduler(job, lastTimeWindowMs); } } From f852385ba3a6ae72f2094663bf719ca2f43d8418 Mon Sep 17 00:00:00 2001 From: zhangdong <493738387@qq.com> Date: Fri, 30 Aug 2024 15:14:52 +0800 Subject: [PATCH 48/60] [fix](auth)fix case should grant after create view (#40108) cause by:https://github.com/apache/doris/pull/39597 --- .../external_table_p0/tvf/test_s3_tvf_with_resource.groovy | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/regression-test/suites/external_table_p0/tvf/test_s3_tvf_with_resource.groovy b/regression-test/suites/external_table_p0/tvf/test_s3_tvf_with_resource.groovy index d436fa6551ae5d..695b3506b5c7fe 100644 --- a/regression-test/suites/external_table_p0/tvf/test_s3_tvf_with_resource.groovy +++ b/regression-test/suites/external_table_p0/tvf/test_s3_tvf_with_resource.groovy @@ -188,7 +188,6 @@ suite("test_s3_tvf_with_resource", "p0") { String viewName = "test_s3_tvf_with_resource_view" try_sql("DROP USER ${user}") sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" - sql """grant select_priv on ${db}.${viewName} to ${user}""" sql "drop view if exists ${viewName}" sql """ create view ${viewName} as @@ -199,7 +198,7 @@ suite("test_s3_tvf_with_resource", "p0") { "resource" = "${resource_name}" ) where k1 > 100 order by k3,k2,k1; """ - + sql """grant select_priv on ${db}.${viewName} to ${user}""" //cloud-mode if (isCloudMode()) { def clusters = sql " SHOW CLUSTERS; " From ce25a19c472ffa6977bf976d27d2c482d5ddf937 Mon Sep 17 00:00:00 2001 From: Mryange <59914473+Mryange@users.noreply.github.com> Date: Fri, 30 Aug 2024 17:21:04 +0800 Subject: [PATCH 49/60] [feature](profile)Enable merging of incomplete profiles. (#39560) ## Proposed changes ``` java.lang.IndexOutOfBoundsException: Index 0 out of bounds for length 0 at jdk.internal.util.Preconditions.outOfBounds(Preconditions.java:64) ~[?:?] at jdk.internal.util.Preconditions.outOfBoundsCheckIndex(Preconditions.java:70) ~[?:?] at jdk.internal.util.Preconditions.checkIndex(Preconditions.java:266) ~[?:?] at java.util.Objects.checkIndex(Objects.java:359) ~[?:?] at java.util.ArrayList.get(ArrayList.java:427) ~[?:?] at org.apache.doris.common.profile.ExecutionProfile.getPipelineAggregatedProfile(ExecutionProfile.java:142) ~[doris-fe.jar:1.2-SNAPSHOT] ``` In the past, we needed to ensure that profiles were complete before merging. Now, this allows incomplete profiles to be merged, with missing profiles being marked in the merged profile. ``` - ProjectionTime: avg 0ns, max 0ns, min 0ns - RowsProduced: sum 0, avg 0, max 0, min 0 - WaitForDependency[SORT_OPERATOR_DEPENDENCY]Time: avg 15min2sec, max 15min2sec, min 15min2sec Pipeline : 3(miss profile): Pipeline : 4(instance_num=48): LOCAL_EXCHANGE_SINK_OPERATOR (PASSTHROUGH) (id=-14): - CloseTime: avg 0ns, max 0ns, min 0ns - ExecTime: avg 29.410us, max 43.336us, min ``` --- .../common/profile/ExecutionProfile.java | 102 +++++++++++++----- 1 file changed, 74 insertions(+), 28 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/profile/ExecutionProfile.java b/fe/fe-core/src/main/java/org/apache/doris/common/profile/ExecutionProfile.java index d4a00939fe7b4f..d3d6826174f91d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/profile/ExecutionProfile.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/profile/ExecutionProfile.java @@ -41,7 +41,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.ReentrantReadWriteLock; /** * root is used to collect profile of a complete query plan(including query or load). @@ -74,6 +74,7 @@ public class ExecutionProfile { // use to merge profile from multi be private Map>> multiBeProfile = null; + private ReentrantReadWriteLock multiBeProfileLock = new ReentrantReadWriteLock(); // Not serialize this property, it is only used to get profile id. private SummaryProfile summaryProfile; @@ -97,8 +98,7 @@ public ExecutionProfile(TUniqueId queryId, List fragmentIds) { RuntimeProfile runtimeProfile = new RuntimeProfile("Fragment " + i); fragmentProfiles.put(fragmentId, runtimeProfile); fragmentsProfile.addChild(runtimeProfile); - multiBeProfile.put(fragmentId, - new ConcurrentHashMap>()); + multiBeProfile.put(fragmentId, Maps.newHashMap()); fragmentIdBeNum.put(fragmentId, 0); seqNoToFragmentId.put(i, fragmentId); ++i; @@ -108,24 +108,54 @@ public ExecutionProfile(TUniqueId queryId, List fragmentIds) { } private List> getMultiBeProfile(int fragmentId) { - Map> multiPipeline = multiBeProfile.get(fragmentId); - List> allPipelines = Lists.newArrayList(); - int pipelineSize = 0; - for (List profiles : multiPipeline.values()) { - pipelineSize = profiles.size(); - break; - } - for (int pipelineIdx = 0; pipelineIdx < pipelineSize; pipelineIdx++) { - List allPipelineTask = new ArrayList(); - for (List pipelines : multiPipeline.values()) { - RuntimeProfile pipeline = pipelines.get(pipelineIdx); - for (Pair runtimeProfile : pipeline.getChildList()) { - allPipelineTask.add(runtimeProfile.first); + multiBeProfileLock.readLock().lock(); + try { + // A fragment in the BE contains multiple pipelines, and each pipeline contains + // multiple pipeline tasks. + Map> multiPipeline = multiBeProfile.get(fragmentId); + List> allPipelines = Lists.newArrayList(); + int pipelineSize = -1; + for (TNetworkAddress beAddress : multiPipeline.keySet()) { + List profileSingleBE = multiPipeline.get(beAddress); + // Check that within the same fragment across all BEs, there should be the same + // number of pipelines. + if (pipelineSize == -1) { + pipelineSize = profileSingleBE.size(); + } else { + if (pipelineSize != profileSingleBE.size()) { + LOG.warn("The profile sizes of the two BE are different, {} vs {}", pipelineSize, + profileSingleBE.size()); + pipelineSize = Math.max(pipelineSize, profileSingleBE.size()); + } + } + } + for (int pipelineIdx = 0; pipelineIdx < pipelineSize; pipelineIdx++) { + List allPipelineTask = new ArrayList(); + for (List profileSingleBE : multiPipeline.values()) { + RuntimeProfile pipeline = profileSingleBE.get(pipelineIdx); + for (Pair pipelineTaskProfile : pipeline.getChildList()) { + allPipelineTask.add(pipelineTaskProfile.first); + } } + if (allPipelineTask.isEmpty()) { + LOG.warn("None of the BEs have pipeline task profiles in fragmentId:{} , pipelineIdx:{}", + fragmentId, pipelineIdx); + } + allPipelines.add(allPipelineTask); } - allPipelines.add(allPipelineTask); + return allPipelines; + } finally { + multiBeProfileLock.readLock().unlock(); + } + } + + void setMultiBeProfile(int fragmentId, TNetworkAddress backendHBAddress, List taskProfile) { + multiBeProfileLock.writeLock().lock(); + try { + multiBeProfile.get(fragmentId).put(backendHBAddress, taskProfile); + } finally { + multiBeProfileLock.writeLock().unlock(); } - return allPipelines; } private RuntimeProfile getPipelineAggregatedProfile(Map planNodeMap) { @@ -136,11 +166,20 @@ private RuntimeProfile getPipelineAggregatedProfile(Map planNod List> allPipelines = getMultiBeProfile(seqNoToFragmentId.get(i)); int pipelineIdx = 0; for (List allPipelineTask : allPipelines) { - RuntimeProfile mergedpipelineProfile = new RuntimeProfile( - "Pipeline : " + pipelineIdx + "(instance_num=" - + allPipelineTask.size() + ")", - allPipelineTask.get(0).nodeId()); - RuntimeProfile.mergeProfiles(allPipelineTask, mergedpipelineProfile, planNodeMap); + RuntimeProfile mergedpipelineProfile = null; + if (allPipelineTask.isEmpty()) { + // It is possible that the profile collection may be incomplete, so only part of + // the profile will be merged here. + mergedpipelineProfile = new RuntimeProfile( + "Pipeline : " + pipelineIdx + "(miss profile)", + -pipelineIdx); + } else { + mergedpipelineProfile = new RuntimeProfile( + "Pipeline : " + pipelineIdx + "(instance_num=" + + allPipelineTask.size() + ")", + allPipelineTask.get(0).nodeId()); + RuntimeProfile.mergeProfiles(allPipelineTask, mergedpipelineProfile, planNodeMap); + } newFragmentProfile.addChild(mergedpipelineProfile); pipelineIdx++; } @@ -207,7 +246,11 @@ public Status updateProfile(TQueryProfile profile, TNetworkAddress backendHBAddr pipelineIdx++; } RuntimeProfile profileNode = new RuntimeProfile(name); - taskProfile.add(profileNode); + // The taskprofile is used to save the profile of the pipeline, without + // considering the FragmentLevel. + if (!(pipelineProfile.isSetIsFragmentLevel() && pipelineProfile.is_fragment_level)) { + taskProfile.add(profileNode); + } if (!pipelineProfile.isSetProfile()) { LOG.warn("Profile is not set, {}", DebugUtil.printId(profile.getQueryId())); return new Status(TStatusCode.INVALID_ARGUMENT, "Profile is not set"); @@ -217,7 +260,7 @@ public Status updateProfile(TQueryProfile profile, TNetworkAddress backendHBAddr profileNode.setIsDone(isDone); fragmentProfiles.get(fragmentId).addChild(profileNode); } - multiBeProfile.get(fragmentId).put(backendHBAddress, taskProfile); + setMultiBeProfile(fragmentId, backendHBAddress, taskProfile); } if (profile.isSetLoadChannelProfiles()) { @@ -255,7 +298,11 @@ public void updateProfile(TReportExecStatusParams params) { pipelineIdx++; } RuntimeProfile profile = new RuntimeProfile(name); - taskProfile.add(profile); + // The taskprofile is used to save the profile of the pipeline, without + // considering the FragmentLevel. + if (!(param.isSetIsFragmentLevel() && param.is_fragment_level)) { + taskProfile.add(profile); + } if (param.isSetProfile()) { profile.update(param.profile); } @@ -270,8 +317,7 @@ public void updateProfile(TReportExecStatusParams params) { if (params.isSetLoadChannelProfile()) { loadChannelProfile.update(params.loadChannelProfile); } - - multiBeProfile.get(params.fragment_id).put(backend.getHeartbeatAddress(), taskProfile); + setMultiBeProfile(params.fragment_id, backend.getHeartbeatAddress(), taskProfile); } public synchronized void addFragmentBackend(PlanFragmentId fragmentId, Long backendId) { From 722b60d548824f36ab09cd59285347cdc88f3e9b Mon Sep 17 00:00:00 2001 From: TengJianPing <18241664+jacktengg@users.noreply.github.com> Date: Fri, 30 Aug 2024 17:29:45 +0800 Subject: [PATCH 50/60] [fix](be_ut) fix repeat function BE UT failure (#40191) ## Proposed changes Delete stale BE UT case of function `repeat`, since it's deprecated by #37907 --- be/test/vec/function/function_string_test.cpp | 8 -------- 1 file changed, 8 deletions(-) diff --git a/be/test/vec/function/function_string_test.cpp b/be/test/vec/function/function_string_test.cpp index ba5e7b9c19bd98..20727c6067176e 100644 --- a/be/test/vec/function/function_string_test.cpp +++ b/be/test/vec/function/function_string_test.cpp @@ -742,14 +742,6 @@ TEST(function_string_test, function_string_repeat_test) { check_function_all_arg_comb(func_name, input_types, data_set); } - - { - InputTypeSet input_types = {TypeIndex::String, TypeIndex::Int32}; - DataSet data_set = {{{std::string("a"), 1073741825}, - std::string("aaaaaaaaaa")}}; // ut repeat max num 10 - Status st = check_function(func_name, input_types, data_set, true); - EXPECT_NE(Status::OK(), st); - } } TEST(function_string_test, function_string_reverse_test) { From 22f2dd0b64e6e1bba18691d75a8063bafc606e19 Mon Sep 17 00:00:00 2001 From: Xinyi Zou Date: Fri, 30 Aug 2024 17:36:18 +0800 Subject: [PATCH 51/60] [fix](ut) Fix ThreadMemTrackerMgrTest TEST_F (#40175) --- be/test/runtime/memory/thread_mem_tracker_mgr_test.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/be/test/runtime/memory/thread_mem_tracker_mgr_test.cpp b/be/test/runtime/memory/thread_mem_tracker_mgr_test.cpp index bab94ace4708f8..c1feb43fe91955 100644 --- a/be/test/runtime/memory/thread_mem_tracker_mgr_test.cpp +++ b/be/test/runtime/memory/thread_mem_tracker_mgr_test.cpp @@ -87,7 +87,7 @@ TEST_F(ThreadMemTrackerMgrTest, ConsumeMemory) { EXPECT_EQ(t->consumption(), 0); // detach automatic call flush_untracked_mem. } -TEST(ThreadMemTrackerMgrTest, Boundary) { +TEST_F(ThreadMemTrackerMgrTest, Boundary) { // TODO, Boundary check may not be necessary, add some `IF` maybe increase cost time. } From c12833b84f0e89d6c2fdf8dd247d19d51b129fce Mon Sep 17 00:00:00 2001 From: lihangyu <15605149486@163.com> Date: Fri, 30 Aug 2024 17:57:10 +0800 Subject: [PATCH 52/60] [regression-test](cases) optimize some cases (#40174) 1. modify some variant cases to concurrent and p1 2. rowsstore cases to concurrent --- be/src/common/config.cpp | 2 +- .../test_variant_index_format_v1.out | 0 .../compaction}/compaction_sparse_column.out | 0 .../test_compaction_extract_root.out | 0 .../test_compaction_uniq_keys_row_store.groovy | 2 +- ...est_compaction_uniq_keys_row_store_ck.groovy | 2 +- .../test_variant_index_format_v1.groovy | 2 +- regression-test/suites/variant_p0/nested.groovy | 17 +++-------------- .../suites/variant_p0/with_index/load.groovy | 3 +-- .../compaction}/compaction_sparse_column.groovy | 2 +- .../test_compaction_extract_root.groovy | 2 +- 11 files changed, 10 insertions(+), 22 deletions(-) rename regression-test/data/{inverted_index_p0 => inverted_index_p2}/test_variant_index_format_v1.out (100%) rename regression-test/data/{variant_p0 => variant_p1/compaction}/compaction_sparse_column.out (100%) rename regression-test/data/{variant_p0 => variant_p1/compaction}/test_compaction_extract_root.out (100%) rename regression-test/suites/{inverted_index_p0 => inverted_index_p2}/test_variant_index_format_v1.groovy (99%) rename regression-test/suites/{variant_p0 => variant_p1/compaction}/compaction_sparse_column.groovy (99%) rename regression-test/suites/{variant_p0 => variant_p1/compaction}/test_compaction_extract_root.groovy (99%) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index d6aaeaf6accf1c..6c38a95bdc91c6 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -971,7 +971,7 @@ DEFINE_mInt64(workload_group_scan_task_wait_timeout_ms, "10000"); DEFINE_Bool(enable_index_apply_preds_except_leafnode_of_andnode, "true"); DEFINE_mDouble(variant_ratio_of_defaults_as_sparse_column, "1"); -DEFINE_mInt64(variant_threshold_rows_to_estimate_sparse_column, "1000"); +DEFINE_mInt64(variant_threshold_rows_to_estimate_sparse_column, "2048"); DEFINE_mBool(variant_throw_exeception_on_invalid_json, "false"); // block file cache diff --git a/regression-test/data/inverted_index_p0/test_variant_index_format_v1.out b/regression-test/data/inverted_index_p2/test_variant_index_format_v1.out similarity index 100% rename from regression-test/data/inverted_index_p0/test_variant_index_format_v1.out rename to regression-test/data/inverted_index_p2/test_variant_index_format_v1.out diff --git a/regression-test/data/variant_p0/compaction_sparse_column.out b/regression-test/data/variant_p1/compaction/compaction_sparse_column.out similarity index 100% rename from regression-test/data/variant_p0/compaction_sparse_column.out rename to regression-test/data/variant_p1/compaction/compaction_sparse_column.out diff --git a/regression-test/data/variant_p0/test_compaction_extract_root.out b/regression-test/data/variant_p1/compaction/test_compaction_extract_root.out similarity index 100% rename from regression-test/data/variant_p0/test_compaction_extract_root.out rename to regression-test/data/variant_p1/compaction/test_compaction_extract_root.out diff --git a/regression-test/suites/compaction/test_compaction_uniq_keys_row_store.groovy b/regression-test/suites/compaction/test_compaction_uniq_keys_row_store.groovy index 15cdef1466c2be..86a338b301de22 100644 --- a/regression-test/suites/compaction/test_compaction_uniq_keys_row_store.groovy +++ b/regression-test/suites/compaction/test_compaction_uniq_keys_row_store.groovy @@ -18,7 +18,7 @@ import org.codehaus.groovy.runtime.IOGroovyMethods -suite("test_compaction_uniq_keys_row_store", "nonConcurrent") { +suite("test_compaction_uniq_keys_row_store", "p0") { def realDb = "regression_test_serving_p0" def tableName = realDb + ".compaction_uniq_keys_row_store_regression_test" sql "CREATE DATABASE IF NOT EXISTS ${realDb}" diff --git a/regression-test/suites/compaction/test_compaction_uniq_keys_row_store_ck.groovy b/regression-test/suites/compaction/test_compaction_uniq_keys_row_store_ck.groovy index 9d38d5e6c9af01..bf4e8dc1a51004 100644 --- a/regression-test/suites/compaction/test_compaction_uniq_keys_row_store_ck.groovy +++ b/regression-test/suites/compaction/test_compaction_uniq_keys_row_store_ck.groovy @@ -18,7 +18,7 @@ import org.codehaus.groovy.runtime.IOGroovyMethods -suite("test_compaction_uniq_keys_row_store_ck", "nonConcurrent") { +suite("test_compaction_uniq_keys_row_store_ck", "p0") { if (isCloudMode()) { logger.info("cloud does not support mow cluster key") return diff --git a/regression-test/suites/inverted_index_p0/test_variant_index_format_v1.groovy b/regression-test/suites/inverted_index_p2/test_variant_index_format_v1.groovy similarity index 99% rename from regression-test/suites/inverted_index_p0/test_variant_index_format_v1.groovy rename to regression-test/suites/inverted_index_p2/test_variant_index_format_v1.groovy index 153e8b82f56532..cc44ac2bb38234 100644 --- a/regression-test/suites/inverted_index_p0/test_variant_index_format_v1.groovy +++ b/regression-test/suites/inverted_index_p2/test_variant_index_format_v1.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_variant_index_format_v1", "p0") { +suite("test_variant_index_format_v1", "p2") { def calc_file_crc_on_tablet = { ip, port, tablet -> return curl("GET", String.format("http://%s:%s/api/calc_crc?tablet_id=%s", ip, port, tablet)) } diff --git a/regression-test/suites/variant_p0/nested.groovy b/regression-test/suites/variant_p0/nested.groovy index bf162250bd81d6..a2a3355854f7c6 100644 --- a/regression-test/suites/variant_p0/nested.groovy +++ b/regression-test/suites/variant_p0/nested.groovy @@ -15,20 +15,12 @@ // specific language governing permissions and limitations // under the License. -suite("regression_test_variant_nested", "p0,nonConcurrent"){ +suite("regression_test_variant_nested", "p0"){ def backendId_to_backendIP = [:] def backendId_to_backendHttpPort = [:] getBackendIpHttpPort(backendId_to_backendIP, backendId_to_backendHttpPort); - def set_be_config = { key, value -> - for (String backend_id: backendId_to_backendIP.keySet()) { - def (code, out, err) = update_be_config(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), key, value) - logger.info("update config: code=" + code + ", out=" + out + ", err=" + err) - } - } - try { - set_be_config.call("variant_ratio_of_defaults_as_sparse_column", "1") def table_name = "var_nested" sql "DROP TABLE IF EXISTS ${table_name}" @@ -135,9 +127,8 @@ suite("regression_test_variant_nested", "p0,nonConcurrent"){ qt_sql """ select * from var_nested order by k limit 101 """ - set_be_config.call("variant_ratio_of_defaults_as_sparse_column", "1") sql """INSERT INTO var_nested SELECT *, '{"k1":1, "k2": "some", "k3" : [1234], "k4" : 1.10000, "k5" : [[123]], "nested1" : {"nested2" : [{"a" : 10, "b" : 1.1, "c" : "1111"}]}}' FROM numbers("number" = "1000") where number > 200 limit 100;""" - sql """INSERT INTO var_nested SELECT *, '{"k2":1, "k3": "nice", "k4" : [1234], "k5" : 1.10000, "k6" : [[123]], "nested2" : {"nested1" : [{"a" : 10, "b" : 1.1, "c" : "1111"}]}}' FROM numbers("number" = "5013") where number >= 400;""" + sql """INSERT INTO var_nested SELECT *, '{"k2":1, "k3": "nice", "k4" : [1234], "k5" : 1.10000, "k6" : [[123]], "nested2" : {"nested1" : [{"a" : 10, "b" : 1.1, "c" : "1111"}]}}' FROM numbers("number" = "5013") where number >= 400 limit 1024;""" triger_compaction.call() qt_sql """select /*+SET_VAR(batch_size=1024,broker_load_batch_size=16352,disable_streaming_preaggregations=true,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_ @@ -145,10 +136,8 @@ parallel_pipeline_task_num=7,parallel_fragment_exec_instance_num=4,profile_level ,parallel_scan_min_rows_per_scanner=128,enable_fold_constant_by_be=false,enable_rewrite_element_at_to_slot=true,runtime_filter_type=2,enable_parallel_result_sink=true,sort_phase_num=0,enable_nereids_planner=true,rewrite_or_to_in_predicate_threshold=2,enable_function_pushdown=true,enable_common_expr_pushdown=false,enable_local_exchange=true,partitioned_hash_join_rows_threshold=8,partitioned_hash_agg_rows_threshold=8,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_common_expr_pushdown_for_inverted_index=true,enable_delete_sub_predicate_v2=true,min_revocable_mem=1,fetch_remote_schema_timeout_seconds=120,max_fetch_remote_schema_tablet_count=512,enable_join_spill=true,enable_sort_spill=true,enable_agg_spill=true,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,external_agg_partition_bits=5) */ * from var_nested where v['k2'] = 'some' order by k limit 10""" qt_sql """select * from var_nested where v['k2'] = 'some' and array_contains(cast(v['nested1']['nested2']['a'] as array), 10) order by k limit 1;""" - set_be_config.call("variant_ratio_of_defaults_as_sparse_column", "1") sql """INSERT INTO var_nested SELECT *, '{"k1":1, "k2": "some", "k3" : [1234], "k4" : 1.10000, "k5" : [[123]], "nested1" : {"nested2" : [{"a" : 10, "b" : 1.1, "c" : "1111"}]}}' FROM numbers("number" = "4096") where number > 1024 limit 1024;""" sql """INSERT INTO var_nested SELECT *, '{"k1":1, "k2": "what", "k3" : [1234], "k4" : 1.10000, "k5" : [[123]], "nested1" : {"nested2" : [{"a" : 10, "b" : 1.1, "c" : "1111"}]}}' FROM numbers("number" = "4096") where number > 2048 limit 1024;""" - set_be_config.call("variant_ratio_of_defaults_as_sparse_column", "1") sql """INSERT INTO var_nested SELECT *, '{"k1":1, "k2": "about", "k3" : [1234], "k4" : 1.10000, "k5" : [[123]], "nested1" : {"nested2" : [{"a" : 10, "b" : 1.1, "c" : "1111"}]}}' FROM numbers("number" = "4096") where number > 3072 limit 1024;""" sql """INSERT INTO var_nested SELECT *, '{"k1":1, "k2": "nested", "k3" : [1234], "k4" : 1.10000, "k5" : [[123]], "nested1" : {"nested2" : [{"a" : 10, "b" : 1.1, "c" : "1111"}]}}' FROM numbers("number" = "6000") where number > 4096 limit 1024;""" qt_sql """select * from var_nested where v['k2'] = 'what' order by k limit 10""" @@ -172,7 +161,7 @@ parallel_pipeline_task_num=7,parallel_fragment_exec_instance_num=4,profile_level DISTRIBUTED BY HASH(k) BUCKETS 1 properties("replication_num" = "1", "disable_auto_compaction" = "false", "enable_unique_key_merge_on_write" = "true", "variant_enable_flatten_nested" = "true"); """ - sql """insert into var_nested2 select * from var_nested""" + sql """insert into var_nested2 select * from var_nested order by k limit 1024""" qt_sql """select * from var_nested2 order by k limit 10;""" qt_sql """select v['nested'] from var_nested2 where k < 10 order by k limit 10;""" // explode variant array diff --git a/regression-test/suites/variant_p0/with_index/load.groovy b/regression-test/suites/variant_p0/with_index/load.groovy index 93737e8a5b9dc8..ba46e7a9eeed8b 100644 --- a/regression-test/suites/variant_p0/with_index/load.groovy +++ b/regression-test/suites/variant_p0/with_index/load.groovy @@ -67,8 +67,7 @@ suite("regression_test_variant_with_index", "nonConcurrent"){ qt_sql_inv_3 """select * from var_with_index where inv match 'hello' and cast(v["a"] as int) > 0 order by k""" sql "truncate table var_with_index" // set back configs - set_be_config.call("variant_ratio_of_defaults_as_sparse_column", "0.95") - set_be_config.call("variant_threshold_rows_to_estimate_sparse_column", "1000") + set_be_config.call("variant_threshold_rows_to_estimate_sparse_column", "2048") // sql "truncate table ${table_name}" sql """insert into var_with_index values(1, '{"a1" : 0, "b1": 3}', 'hello world'), (2, '{"a2" : 123}', 'world'),(3, '{"a3" : 123}', 'hello world')""" sql """insert into var_with_index values(4, '{"b1" : 0, "b2": 3}', 'hello world'), (5, '{"b2" : 123}', 'world'),(6, '{"b3" : 123}', 'hello world')""" diff --git a/regression-test/suites/variant_p0/compaction_sparse_column.groovy b/regression-test/suites/variant_p1/compaction/compaction_sparse_column.groovy similarity index 99% rename from regression-test/suites/variant_p0/compaction_sparse_column.groovy rename to regression-test/suites/variant_p1/compaction/compaction_sparse_column.groovy index 0a7ea100373452..91f64c19a02d22 100644 --- a/regression-test/suites/variant_p0/compaction_sparse_column.groovy +++ b/regression-test/suites/variant_p1/compaction/compaction_sparse_column.groovy @@ -18,7 +18,7 @@ import org.codehaus.groovy.runtime.IOGroovyMethods import org.awaitility.Awaitility -suite("test_compaction_sparse_column", "nonConcurrent") { +suite("test_compaction_sparse_column", "p1,nonConcurrent") { def tableName = "test_compaction" try { diff --git a/regression-test/suites/variant_p0/test_compaction_extract_root.groovy b/regression-test/suites/variant_p1/compaction/test_compaction_extract_root.groovy similarity index 99% rename from regression-test/suites/variant_p0/test_compaction_extract_root.groovy rename to regression-test/suites/variant_p1/compaction/test_compaction_extract_root.groovy index 43f1048f1510c3..83dc0a559e6fc5 100644 --- a/regression-test/suites/variant_p0/test_compaction_extract_root.groovy +++ b/regression-test/suites/variant_p1/compaction/test_compaction_extract_root.groovy @@ -17,7 +17,7 @@ import org.codehaus.groovy.runtime.IOGroovyMethods -suite("test_compaction_extract_root", "nonConcurrent") { +suite("test_compaction_extract_root", "p1,nonConcurrent") { def tableName = "test_t" def backendId_to_backendIP = [:] From 9dfd6b2d5cd2af8e26aa5a7fb492a709b3a8e544 Mon Sep 17 00:00:00 2001 From: hui lai <1353307710@qq.com> Date: Fri, 30 Aug 2024 17:57:47 +0800 Subject: [PATCH 53/60] [fix](log) remove some invalid log (#40177) introduced by https://github.com/apache/doris/pull/37903 --- .../doris/load/StreamLoadRecordMgr.java | 28 +++++++++---------- 1 file changed, 13 insertions(+), 15 deletions(-) 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 020b38158d1855..7f1d33bd649736 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 @@ -150,19 +150,16 @@ public void addStreamLoadRecord(long dbId, String label, StreamLoadRecord stream } public List getStreamLoadRecords() { - LOG.info("test log: {}", streamLoadRecordHeap); return new ArrayList<>(streamLoadRecordHeap); } public List> getStreamLoadRecordByDb( long dbId, String label, boolean accurateMatch, StreamLoadState state) { LinkedList> streamLoadRecords = new LinkedList>(); - LOG.info("test log: {}", dbId); readLock(); try { if (!dbIdToLabelToStreamLoadRecord.containsKey(dbId)) { - LOG.info("test log: {}", dbId); return streamLoadRecords; } @@ -205,7 +202,6 @@ public List> getStreamLoadRecordByDb( } } - LOG.info("test log: {}", streamLoadRecords); return streamLoadRecords; } finally { readUnlock(); @@ -273,17 +269,19 @@ protected void runAfterCatalogReady() { TimeUtils.getDatetimeMsFormatWithTimeZone()); String finishTime = TimeUtils.longToTimeString(streamLoadItem.getFinishTime(), TimeUtils.getDatetimeMsFormatWithTimeZone()); - LOG.info("receive stream load record info from backend: {}." - + " label: {}, db: {}, tbl: {}, user: {}, user_ip: {}," - + " status: {}, message: {}, error_url: {}," - + " total_rows: {}, loaded_rows: {}, filtered_rows: {}, unselected_rows: {}," - + " load_bytes: {}, start_time: {}, finish_time: {}.", - backend.getHost(), streamLoadItem.getLabel(), streamLoadItem.getDb(), - streamLoadItem.getTbl(), streamLoadItem.getUser(), streamLoadItem.getUserIp(), - streamLoadItem.getStatus(), streamLoadItem.getMessage(), streamLoadItem.getUrl(), - streamLoadItem.getTotalRows(), streamLoadItem.getLoadedRows(), - streamLoadItem.getFilteredRows(), streamLoadItem.getUnselectedRows(), - streamLoadItem.getLoadBytes(), startTime, finishTime); + if (LOG.isDebugEnabled()) { + LOG.debug("receive stream load record info from backend: {}." + + " label: {}, db: {}, tbl: {}, user: {}, user_ip: {}," + + " status: {}, message: {}, error_url: {}," + + " total_rows: {}, loaded_rows: {}, filtered_rows: {}, unselected_rows: {}," + + " load_bytes: {}, start_time: {}, finish_time: {}.", + backend.getHost(), streamLoadItem.getLabel(), streamLoadItem.getDb(), + streamLoadItem.getTbl(), streamLoadItem.getUser(), streamLoadItem.getUserIp(), + streamLoadItem.getStatus(), streamLoadItem.getMessage(), streamLoadItem.getUrl(), + streamLoadItem.getTotalRows(), streamLoadItem.getLoadedRows(), + streamLoadItem.getFilteredRows(), streamLoadItem.getUnselectedRows(), + streamLoadItem.getLoadBytes(), startTime, finishTime); + } AuditEvent auditEvent = new StreamLoadAuditEvent.AuditEventBuilder().setEventType(EventType.STREAM_LOAD_FINISH) From 8e44e50e592476cf78487ccda7ed20fd14272cef Mon Sep 17 00:00:00 2001 From: Jibing-Li <64681310+Jibing-Li@users.noreply.github.com> Date: Fri, 30 Aug 2024 19:24:09 +0800 Subject: [PATCH 54/60] [feature](statistics) Support get row count for JDBC external table. (#38889) Support get row count for JDBC external table by executing select count(1) from table. Return -1 when external table row count is unknown. --- .../datasource/jdbc/JdbcExternalTable.java | 58 +++++++++++++++++++ .../jdbc/test_mysql_jdbc_statistics.groovy | 47 +++++++++------ 2 files changed, 88 insertions(+), 17 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalTable.java index 21d73499f2e8cc..495311bc087d5b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalTable.java @@ -17,19 +17,28 @@ package org.apache.doris.datasource.jdbc; +import org.apache.doris.analysis.StatementBase; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.JdbcResource; import org.apache.doris.catalog.JdbcTable; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.doris.qe.AutoCloseConnectContext; +import org.apache.doris.qe.StmtExecutor; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ExternalAnalysisTask; +import org.apache.doris.statistics.ResultRow; +import org.apache.doris.statistics.util.StatisticsUtil; import org.apache.doris.thrift.TTableDescriptor; +import org.apache.commons.text.StringSubstitutor; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Optional; /** @@ -38,6 +47,9 @@ public class JdbcExternalTable extends ExternalTable { private static final Logger LOG = LogManager.getLogger(JdbcExternalTable.class); + public static final String MYSQL_ROW_COUNT_SQL = "SELECT * FROM QUERY" + + "(\"catalog\"=\"${ctlName}\", \"query\"=\"show table status from `${dbName}` like '${tblName}'\");"; + private JdbcTable jdbcTable; /** @@ -98,4 +110,50 @@ public BaseAnalysisTask createAnalysisTask(AnalysisInfo info) { makeSureInitialized(); return new ExternalAnalysisTask(info); } + + @Override + public long fetchRowCount() { + Map params = new HashMap<>(); + params.put("ctlName", catalog.getName()); + params.put("dbName", dbName); + params.put("tblName", name); + switch (((JdbcExternalCatalog) catalog).getDatabaseTypeName()) { + case JdbcResource.MYSQL: + try (AutoCloseConnectContext r = StatisticsUtil.buildConnectContext(false, false)) { + StringSubstitutor stringSubstitutor = new StringSubstitutor(params); + String sql = stringSubstitutor.replace(MYSQL_ROW_COUNT_SQL); + StmtExecutor stmtExecutor = new StmtExecutor(r.connectContext, sql); + List resultRows = stmtExecutor.executeInternalQuery(); + if (resultRows == null || resultRows.size() != 1) { + LOG.info("No mysql status found for table {}.{}.{}", catalog.getName(), dbName, name); + return -1; + } + StatementBase parsedStmt = stmtExecutor.getParsedStmt(); + if (parsedStmt == null || parsedStmt.getColLabels() == null) { + LOG.info("No column label found for table {}.{}.{}", catalog.getName(), dbName, name); + return -1; + } + ResultRow resultRow = resultRows.get(0); + List colLabels = parsedStmt.getColLabels(); + int index = colLabels.indexOf("TABLE_ROWS"); + if (index == -1) { + LOG.info("No TABLE_ROWS in status for table {}.{}.{}", catalog.getName(), dbName, name); + return -1; + } + long rows = Long.parseLong(resultRow.get(index)); + LOG.info("Get mysql table {}.{}.{} row count {}", catalog.getName(), dbName, name, rows); + return rows; + } catch (Exception e) { + LOG.warn("Failed to fetch mysql row count for table {}.{}.{}. Reason [{}]", + catalog.getName(), dbName, name, e.getMessage()); + return -1; + } + case JdbcResource.ORACLE: + case JdbcResource.POSTGRESQL: + case JdbcResource.SQLSERVER: + default: + break; + } + return -1; + } } diff --git a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_statistics.groovy b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_statistics.groovy index 66b04ebd5136ea..e9bd59d8cb29a8 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_statistics.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_statistics.groovy @@ -17,6 +17,7 @@ suite("test_mysql_jdbc_statistics", "p0,external,mysql,external_docker,external_docker_mysql") { String enabled = context.config.otherConfigs.get("enableJdbcTest") + logger.info("enabled " + enabled) String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") String mysql_port = context.config.otherConfigs.get("mysql_57_port"); String s3_endpoint = getS3Endpoint() @@ -35,28 +36,40 @@ suite("test_mysql_jdbc_statistics", "p0,external,mysql,external_docker,external_ );""" sql """use ${catalog_name}.doris_test""" + + def result = sql """show table stats ex_tb0""" + Thread.sleep(1000) + for (int i = 0; i < 20; i++) { + result = sql """show table stats ex_tb0"""; + if (result[0][2] != "-1") { + assertEquals("5", result[0][2]) + break; + } + logger.info("Table row count not ready yet. Wait 1 second.") + Thread.sleep(1000) + } sql """analyze table ex_tb0 with sync""" - def result = sql """show column stats ex_tb0 (name)""" - assertTrue(result.size() == 1) - assertTrue(result[0][0] == "name") - assertTrue(result[0][2] == "5.0") - assertTrue(result[0][3] == "5.0") - assertTrue(result[0][4] == "0.0") - assertTrue(result[0][5] == "15.0") - assertTrue(result[0][6] == "3.0") + result = sql """show column stats ex_tb0 (name)""" + assertEquals(result.size(), 1) + assertEquals(result[0][0], "name") + assertEquals(result[0][2], "5.0") + assertEquals(result[0][3], "5.0") + assertEquals(result[0][4], "0.0") + assertEquals(result[0][5], "15.0") + assertEquals(result[0][6], "3.0") assertEquals(result[0][7], "'abc'") assertEquals(result[0][8], "'abg'") result = sql """show column stats ex_tb0 (id)""" - assertTrue(result.size() == 1) - assertTrue(result[0][0] == "id") - assertTrue(result[0][2] == "5.0") - assertTrue(result[0][3] == "5.0") - assertTrue(result[0][4] == "0.0") - assertTrue(result[0][5] == "20.0") - assertTrue(result[0][6] == "4.0") - assertTrue(result[0][7] == "111") - assertTrue(result[0][8] == "115") + assertEquals(result.size(), 1) + assertEquals(result[0][0], "id") + assertEquals(result[0][2], "5.0") + assertEquals(result[0][3], "5.0") + assertEquals(result[0][4], "0.0") + assertEquals(result[0][5], "20.0") + assertEquals(result[0][6], "4.0") + assertEquals(result[0][7], "111") + assertEquals(result[0][8], "115") sql """drop catalog ${catalog_name}""" } From 3ff7ad3a576ae90a6838f1e71fe8d78c463f013f Mon Sep 17 00:00:00 2001 From: zclllhhjj Date: Sat, 31 Aug 2024 14:08:15 +0800 Subject: [PATCH 55/60] [Enhancement](DDL) check illegal partition exprs (#40158) before: ```sql mysql> CREATE TABLE not_auto_expr ( -> `TIME_STAMP` date NOT NULL -> ) -> partition by range (date_trunc(`TIME_STAMP`, 'day'))() -> DISTRIBUTED BY HASH(`TIME_STAMP`) BUCKETS 10 -> PROPERTIES ( -> "replication_allocation" = "tag.location.default: 1" -> ); Query OK, 0 rows affected (0.14 sec) ``` now: ```sql mysql> CREATE TABLE not_auto_expr ( -> `TIME_STAMP` date NOT NULL -> ) -> partition by range (date_trunc(`TIME_STAMP`, 'day'))() -> DISTRIBUTED BY HASH(`TIME_STAMP`) BUCKETS 10 -> PROPERTIES ( -> "replication_allocation" = "tag.location.default: 1" -> ); ERROR 1105 (HY000): errCode = 2, detailMessage = errCode = 2, detailMessage = Non-auto partition table not support partition expr! ``` --- .../doris/nereids/parser/PartitionTableInfo.java | 9 +++++++++ .../test_auto_partition_behavior.groovy | 14 ++++++++++++++ 2 files changed, 23 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/PartitionTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/PartitionTableInfo.java index a68ddcdf87ac24..e9f7fdcfee3e79 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/PartitionTableInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/PartitionTableInfo.java @@ -28,6 +28,7 @@ import org.apache.doris.analysis.StringLiteral; import org.apache.doris.catalog.AggregateType; import org.apache.doris.catalog.PartitionType; +import org.apache.doris.common.DdlException; import org.apache.doris.nereids.analyzer.UnboundFunction; import org.apache.doris.nereids.analyzer.UnboundSlot; import org.apache.doris.nereids.exceptions.AnalysisException; @@ -269,6 +270,14 @@ public PartitionDesc convertToPartitionDesc(boolean isExternal) { try { ArrayList exprs = convertToLegacyAutoPartitionExprs(partitionList); + + // only auto partition support partition expr + if (!isAutoPartition) { + if (exprs.stream().anyMatch(expr -> expr instanceof FunctionCallExpr)) { + throw new DdlException("Non-auto partition table not support partition expr!"); + } + } + // here we have already extracted identifierPartitionColumns if (partitionType.equals(PartitionType.RANGE.name())) { if (isAutoPartition) { diff --git a/regression-test/suites/partition_p0/auto_partition/test_auto_partition_behavior.groovy b/regression-test/suites/partition_p0/auto_partition/test_auto_partition_behavior.groovy index fb8be0b55101a7..e5ce52af31e395 100644 --- a/regression-test/suites/partition_p0/auto_partition/test_auto_partition_behavior.groovy +++ b/regression-test/suites/partition_p0/auto_partition/test_auto_partition_behavior.groovy @@ -407,4 +407,18 @@ suite("test_auto_partition_behavior") { sql """ insert into test_change values ("20001212"); """ part_result = sql " show tablets from test_change " assertEquals(part_result.size, 52 * replicaNum) + + test { + sql """ + CREATE TABLE not_auto_expr ( + `TIME_STAMP` date NOT NULL + ) + partition by range (date_trunc(`TIME_STAMP`, 'day'))() + DISTRIBUTED BY HASH(`TIME_STAMP`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + exception "Non-auto partition table not support partition expr!" + } } From 352be298ac048e96cb336ce917f03f1202579d6e Mon Sep 17 00:00:00 2001 From: yiguolei <676222867@qq.com> Date: Sat, 31 Aug 2024 15:10:41 +0800 Subject: [PATCH 56/60] [enhancement](memory) add exception logic in write page and schema scanner (#40168) ## Proposed changes Issue Number: close #xxx --------- Co-authored-by: yiguolei --- be/src/exec/schema_scanner.cpp | 6 +++++- be/src/olap/rowset/segment_v2/column_writer.cpp | 3 ++- be/src/olap/rowset/segment_v2/column_writer.h | 8 +++++++- 3 files changed, 14 insertions(+), 3 deletions(-) diff --git a/be/src/exec/schema_scanner.cpp b/be/src/exec/schema_scanner.cpp index be0bd8eff72c4c..b5cd484b650b3f 100644 --- a/be/src/exec/schema_scanner.cpp +++ b/be/src/exec/schema_scanner.cpp @@ -134,7 +134,11 @@ Status SchemaScanner::get_next_block_async(RuntimeState* state) { _opened = true; } bool eos = false; - _scanner_status.update(get_next_block_internal(_data_block.get(), &eos)); + auto call_next_block_internal = [&]() -> Status { + RETURN_IF_CATCH_EXCEPTION( + { return get_next_block_internal(_data_block.get(), &eos); }); + }; + _scanner_status.update(call_next_block_internal()); _eos = eos; _async_thread_running = false; _dependency->set_ready(); diff --git a/be/src/olap/rowset/segment_v2/column_writer.cpp b/be/src/olap/rowset/segment_v2/column_writer.cpp index 75acd9f1f956fe..2637017b78d02b 100644 --- a/be/src/olap/rowset/segment_v2/column_writer.cpp +++ b/be/src/olap/rowset/segment_v2/column_writer.cpp @@ -534,7 +534,8 @@ Status ScalarColumnWriter::append_data(const uint8_t** ptr, size_t num_rows) { return Status::OK(); } -Status ScalarColumnWriter::append_data_in_current_page(const uint8_t* data, size_t* num_written) { +Status ScalarColumnWriter::_internal_append_data_in_current_page(const uint8_t* data, + size_t* num_written) { RETURN_IF_ERROR(_page_builder->add(data, num_written)); if (_opts.need_zone_map) { _zone_map_index_builder->add_values(data, *num_written); diff --git a/be/src/olap/rowset/segment_v2/column_writer.h b/be/src/olap/rowset/segment_v2/column_writer.h index acda2b75fb2488..62f209db5ad4a5 100644 --- a/be/src/olap/rowset/segment_v2/column_writer.h +++ b/be/src/olap/rowset/segment_v2/column_writer.h @@ -217,10 +217,16 @@ class ScalarColumnWriter : public ColumnWriter { // used for append not null data. When page is full, will append data not reach num_rows. Status append_data_in_current_page(const uint8_t** ptr, size_t* num_written); - Status append_data_in_current_page(const uint8_t* ptr, size_t* num_written); + Status append_data_in_current_page(const uint8_t* ptr, size_t* num_written) { + RETURN_IF_CATCH_EXCEPTION( + { return _internal_append_data_in_current_page(ptr, num_written); }); + } friend class ArrayColumnWriter; friend class OffsetColumnWriter; +private: + Status _internal_append_data_in_current_page(const uint8_t* ptr, size_t* num_written); + private: std::unique_ptr _page_builder; From dac10b761caece33203cb45893b7b0872eaf167b Mon Sep 17 00:00:00 2001 From: Mryange <59914473+Mryange@users.noreply.github.com> Date: Sat, 31 Aug 2024 16:45:30 +0800 Subject: [PATCH 57/60] [case](array) fix unsort array case (#39941) --- .../data/nereids_p0/aggregate/agg_nullable_2.out | 2 +- .../suites/nereids_p0/aggregate/agg_nullable_2.groovy | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/regression-test/data/nereids_p0/aggregate/agg_nullable_2.out b/regression-test/data/nereids_p0/aggregate/agg_nullable_2.out index 37a9eba793ca7a..a038340000d550 100644 --- a/regression-test/data/nereids_p0/aggregate/agg_nullable_2.out +++ b/regression-test/data/nereids_p0/aggregate/agg_nullable_2.out @@ -141,7 +141,7 @@ [4, 5, 6] -- !select_group_array_intersect_n -- -[2, 1, 3] +[1, 2, 3] -- !select_group_bit_and -- 50 diff --git a/regression-test/suites/nereids_p0/aggregate/agg_nullable_2.groovy b/regression-test/suites/nereids_p0/aggregate/agg_nullable_2.groovy index 42c24815a9ace5..5337f59d015aac 100644 --- a/regression-test/suites/nereids_p0/aggregate/agg_nullable_2.groovy +++ b/regression-test/suites/nereids_p0/aggregate/agg_nullable_2.groovy @@ -331,19 +331,19 @@ suite("agg_nullable_2") { contains "colUniqueId=null, type=bigint, nullable=false" } - qt_select_group_array_intersect """select group_array_intersect(kaint) from agg_nullable_test_2;""" + qt_select_group_array_intersect """select array_sort(group_array_intersect(kaint)) from agg_nullable_test_2;""" explain { sql("verbose select group_array_intersect(kaint) from agg_nullable_test_2;") contains "colUniqueId=null, type=array, nullable=false" } - qt_select_group_array_intersect2 """select group_array_intersect(kaint) from agg_nullable_test_2 group by id;""" + qt_select_group_array_intersect2 """select array_sort(group_array_intersect(kaint)) from agg_nullable_test_2 group by id;""" explain { sql("verbose select group_array_intersect(kaint) from agg_nullable_test_2 group by id;") contains "colUniqueId=null, type=array, nullable=false" } - qt_select_group_array_intersect_n """select group_array_intersect(knaint) from agg_nullable_test_2;""" + qt_select_group_array_intersect_n """select array_sort(group_array_intersect(knaint)) from agg_nullable_test_2;""" explain { sql("verbose select group_array_intersect(knaint) from agg_nullable_test_2;") contains "colUniqueId=null, type=array, nullable=false" From f4e3ec0f4d233901c93c17bea919e4946f055c50 Mon Sep 17 00:00:00 2001 From: zhangstar333 <87313068+zhangstar333@users.noreply.github.com> Date: Sat, 31 Aug 2024 16:49:17 +0800 Subject: [PATCH 58/60] [Bug](expr) execute expr should use local states instead of operators (#40189) ## Proposed changes The expr of operator cannot be executed concurrently, should use local state's expr. --- .../pipeline/exec/aggregation_source_operator.cpp | 3 ++- be/src/pipeline/exec/assert_num_rows_operator.cpp | 3 ++- be/src/pipeline/exec/assert_num_rows_operator.h | 3 +++ .../distinct_streaming_aggregation_operator.cpp | 4 ++-- .../pipeline/exec/multi_cast_data_stream_source.h | 7 ++++--- be/src/pipeline/exec/operator.h | 12 +++++++----- be/src/pipeline/exec/repeat_operator.cpp | 2 +- .../exec/streaming_aggregation_operator.cpp | 4 ++-- .../data/javaudf_p0/test_javaudf_string.out | 3 +++ .../suites/javaudf_p0/test_javaudf_string.groovy | 13 +++++++++++++ 10 files changed, 39 insertions(+), 15 deletions(-) diff --git a/be/src/pipeline/exec/aggregation_source_operator.cpp b/be/src/pipeline/exec/aggregation_source_operator.cpp index 3264ad56f3c585..a5f40a431c5ee6 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/aggregation_source_operator.cpp @@ -443,7 +443,8 @@ Status AggSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* blo RETURN_IF_ERROR(local_state._executor.get_result(state, block, eos)); local_state.make_nullable_output_key(block); // dispose the having clause, should not be execute in prestreaming agg - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(_conjuncts, block, block->columns())); + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, + block->columns())); local_state.do_agg_limit(block, eos); return Status::OK(); } diff --git a/be/src/pipeline/exec/assert_num_rows_operator.cpp b/be/src/pipeline/exec/assert_num_rows_operator.cpp index 4a51002beff389..5aa27b51c45095 100644 --- a/be/src/pipeline/exec/assert_num_rows_operator.cpp +++ b/be/src/pipeline/exec/assert_num_rows_operator.cpp @@ -116,7 +116,8 @@ Status AssertNumRowsOperatorX::pull(doris::RuntimeState* state, vectorized::Bloc } COUNTER_SET(local_state.rows_returned_counter(), local_state.num_rows_returned()); COUNTER_UPDATE(local_state.blocks_returned_counter(), 1); - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(_conjuncts, block, block->columns())); + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, + block->columns())); return Status::OK(); } diff --git a/be/src/pipeline/exec/assert_num_rows_operator.h b/be/src/pipeline/exec/assert_num_rows_operator.h index 423bd69144e49d..dcc64f57878d38 100644 --- a/be/src/pipeline/exec/assert_num_rows_operator.h +++ b/be/src/pipeline/exec/assert_num_rows_operator.h @@ -28,6 +28,9 @@ class AssertNumRowsLocalState final : public PipelineXLocalState(state, parent) {} ~AssertNumRowsLocalState() = default; + +private: + friend class AssertNumRowsOperatorX; }; class AssertNumRowsOperatorX final : public StreamingOperatorX { diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp index e8efb51973e436..ab71b52ae015f0 100644 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp +++ b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp @@ -462,8 +462,8 @@ Status DistinctStreamingAggOperatorX::pull(RuntimeState* state, vectorized::Bloc local_state._make_nullable_output_key(block); if (!_is_streaming_preagg) { // dispose the having clause, should not be execute in prestreaming agg - RETURN_IF_ERROR( - vectorized::VExprContext::filter_block(_conjuncts, block, block->columns())); + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, + block->columns())); } local_state.add_num_rows_returned(block->rows()); COUNTER_UPDATE(local_state.blocks_returned_counter(), 1); 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 8ecbd23764dee6..c71310e3ee3327 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.h +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.h @@ -62,6 +62,7 @@ class MultiCastDataStreamSourceLocalState final : public PipelineXLocalState> _filter_dependencies; @@ -95,8 +96,8 @@ class MultiCastDataStreamerSourceOperatorX final if (_t_data_stream_sink.__isset.conjuncts) { RETURN_IF_ERROR(vectorized::VExpr::create_expr_trees(_t_data_stream_sink.conjuncts, - _conjuncts)); - RETURN_IF_ERROR(vectorized::VExpr::prepare(_conjuncts, state, _row_desc())); + conjuncts())); + RETURN_IF_ERROR(vectorized::VExpr::prepare(conjuncts(), state, _row_desc())); } return Status::OK(); } @@ -107,7 +108,7 @@ class MultiCastDataStreamerSourceOperatorX final RETURN_IF_ERROR(vectorized::VExpr::open(_output_expr_contexts, state)); } if (_t_data_stream_sink.__isset.conjuncts) { - RETURN_IF_ERROR(vectorized::VExpr::open(_conjuncts, state)); + RETURN_IF_ERROR(vectorized::VExpr::open(conjuncts(), state)); } return Status::OK(); } diff --git a/be/src/pipeline/exec/operator.h b/be/src/pipeline/exec/operator.h index abed7fb446a4ad..bde2291ec3a372 100644 --- a/be/src/pipeline/exec/operator.h +++ b/be/src/pipeline/exec/operator.h @@ -760,16 +760,18 @@ class OperatorXBase : public OperatorBase { ObjectPool* _pool = nullptr; std::vector _tuple_ids; +private: + // The expr of operator set to private permissions, as cannot be executed concurrently, + // should use local state's expr. vectorized::VExprContextSPtrs _conjuncts; + vectorized::VExprContextSPtrs _projections; + // Used in common subexpression elimination to compute intermediate results. + std::vector _intermediate_projections; +protected: RowDescriptor _row_descriptor; - std::unique_ptr _output_row_descriptor = nullptr; - vectorized::VExprContextSPtrs _projections; - std::vector _intermediate_output_row_descriptor; - // Used in common subexpression elimination to compute intermediate results. - std::vector _intermediate_projections; /// Resource information sent from the frontend. const TBackendResourceProfile _resource_profile; diff --git a/be/src/pipeline/exec/repeat_operator.cpp b/be/src/pipeline/exec/repeat_operator.cpp index 48cc427d85bf9a..fe45e85c52d485 100644 --- a/be/src/pipeline/exec/repeat_operator.cpp +++ b/be/src/pipeline/exec/repeat_operator.cpp @@ -234,7 +234,7 @@ Status RepeatOperatorX::pull(doris::RuntimeState* state, vectorized::Block* outp } _child_block.clear_column_data(_child_x->row_desc().num_materialized_slots()); } - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(_conjuncts, output_block, + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, output_block, output_block->columns())); *eos = _child_eos && _child_block.rows() == 0; local_state.reached_limit(output_block, eos); diff --git a/be/src/pipeline/exec/streaming_aggregation_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_operator.cpp index 689a361c371d16..8aa1eb8df97b22 100644 --- a/be/src/pipeline/exec/streaming_aggregation_operator.cpp +++ b/be/src/pipeline/exec/streaming_aggregation_operator.cpp @@ -1285,8 +1285,8 @@ Status StreamingAggOperatorX::pull(RuntimeState* state, vectorized::Block* block RETURN_IF_ERROR(local_state._executor->get_result(&local_state, state, block, eos)); local_state.make_nullable_output_key(block); // dispose the having clause, should not be execute in prestreaming agg - RETURN_IF_ERROR( - vectorized::VExprContext::filter_block(_conjuncts, block, block->columns())); + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, + block->columns())); } local_state.reached_limit(block, eos); diff --git a/regression-test/data/javaudf_p0/test_javaudf_string.out b/regression-test/data/javaudf_p0/test_javaudf_string.out index 59f2f7c776dd51..b42a368b02886e 100644 --- a/regression-test/data/javaudf_p0/test_javaudf_string.out +++ b/regression-test/data/javaudf_p0/test_javaudf_string.out @@ -65,3 +65,6 @@ ab***fg7 ab***fg7 ab***fg8 ab***fg8 ab***fg9 ab***fg9 +-- !select_5 -- +0 + diff --git a/regression-test/suites/javaudf_p0/test_javaudf_string.groovy b/regression-test/suites/javaudf_p0/test_javaudf_string.groovy index 07070cec8b03b7..48e98b0c5b6f4c 100644 --- a/regression-test/suites/javaudf_p0/test_javaudf_string.groovy +++ b/regression-test/suites/javaudf_p0/test_javaudf_string.groovy @@ -102,9 +102,22 @@ suite("test_javaudf_string") { ); """ exception "does not support type" } + sql """DROP TABLE IF EXISTS tbl1""" + sql """create table tbl1(k1 int, k2 string) distributed by hash(k1) buckets 1 properties("replication_num" = "1");""" + sql """ insert into tbl1 values(1, "5");""" + Integer count = 0; + Integer maxCount = 20; + while (count < maxCount) { + sql """ insert into tbl1 select * from tbl1;""" + count++ + sleep(100); + } + sql """ insert into tbl1 select random()%10000 * 10000, "5" from tbl1;""" + qt_select_5 """ select count(0) from (select k1, max(k2) as k2 from tbl1 group by k1)v where java_udf_string_test(k2, 0, 1) = "asd" """; } finally { try_sql("DROP FUNCTION IF EXISTS java_udf_string_test(string, int, int);") try_sql("DROP TABLE IF EXISTS ${tableName}") + try_sql("DROP TABLE IF EXISTS tbl1") try_sql("DROP TABLE IF EXISTS test_javaudf_string_2") } } From 83e12a84b485a6da92180ce706941bd6245c3d1f Mon Sep 17 00:00:00 2001 From: TengJianPing <18241664+jacktengg@users.noreply.github.com> Date: Sun, 1 Sep 2024 09:25:52 +0800 Subject: [PATCH 59/60] [fix](regression) fix regression failure of window_funnel caused by duplicated table name (#40217) ## Proposed changes Issue Number: close #xxx --- .../nereids_p0/aggregate/window_funnel.out | 3 + .../window_functions/window_funnel.out | 84 ---- .../data/query_p0/aggregate/window_funnel.out | 31 -- .../nereids_p0/aggregate/window_funnel.groovy | 67 +++- .../window_functions/window_funnel.sql | 92 ----- .../query_p0/aggregate/window_funnel.groovy | 360 ------------------ 6 files changed, 69 insertions(+), 568 deletions(-) delete mode 100644 regression-test/data/nereids_p0/sql_functions/window_functions/window_funnel.out delete mode 100644 regression-test/data/query_p0/aggregate/window_funnel.out delete mode 100644 regression-test/suites/nereids_p0/sql_functions/window_functions/window_funnel.sql delete mode 100644 regression-test/suites/query_p0/aggregate/window_funnel.groovy diff --git a/regression-test/data/nereids_p0/aggregate/window_funnel.out b/regression-test/data/nereids_p0/aggregate/window_funnel.out index f16f050c8e69d2..0bbe397fe3ecb4 100644 --- a/regression-test/data/nereids_p0/aggregate/window_funnel.out +++ b/regression-test/data/nereids_p0/aggregate/window_funnel.out @@ -11,6 +11,9 @@ -- !window_funnel -- 2 +-- !window_funnel_deduplication_compat -- +4 + -- !window_funnel_deduplication -- 2 diff --git a/regression-test/data/nereids_p0/sql_functions/window_functions/window_funnel.out b/regression-test/data/nereids_p0/sql_functions/window_functions/window_funnel.out deleted file mode 100644 index 43948dc795138e..00000000000000 --- a/regression-test/data/nereids_p0/sql_functions/window_functions/window_funnel.out +++ /dev/null @@ -1,84 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !window_funnel -- -0 - --- !window_funnel_2 -- -0 - --- !window_funnel_3 -- -4 - --- !window_funnel_4 -- -1 2022-03-12T19:05:04 4 -1 2022-03-12T16:15:01 3 -1 2022-03-12T13:28:02 2 -1 2022-03-12T10:41 1 - --- !window_funnel_5 -- -1 - --- !window_funnel_6 -- -2 - --- !window_funnel_7 -- -0 - --- !window_funnel_8 -- -1 - --- !window_funnel_9 -- -1 - --- !window_funnel_10 -- -1 - --- !window_funnel_11 -- -1 - --- !window_funnel_12 -- -1 - --- !window_funnel_13 -- -1 - --- !window_funnel_14 -- -1 - --- !window_funnel_15 -- -1 - --- !window_funnel_16 -- -1 - --- !window_funnel_17 -- -1 - --- !window_funnel_18 -- -1 - --- !window_funnel_19 -- -1 - --- !window_funnel_20 -- -1 - --- !window_funnel_21 -- -1 - --- !window_funnel_22 -- -1 - --- !window_funnel_23 -- -1 - --- !window_funnel_24 -- -1 - --- !window_funnel_25 -- -1 - --- !window_funnel_26 -- -users 13 -browser 10 -buy 1 - diff --git a/regression-test/data/query_p0/aggregate/window_funnel.out b/regression-test/data/query_p0/aggregate/window_funnel.out deleted file mode 100644 index fa239a9e719288..00000000000000 --- a/regression-test/data/query_p0/aggregate/window_funnel.out +++ /dev/null @@ -1,31 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !window_funnel -- -1 - --- !window_funnel -- -2 - --- !window_funnel -- -1 - --- !window_funnel -- -2 - --- !window_funnel_deduplication_compat -- -4 - --- !window_funnel_deduplication -- -2 - --- !window_funnel_fixed -- -2 - --- !window_funnel_fixed -- -1 - --- !window_funnel_increase -- -4 - --- !window_funnel_increase -- -2 - diff --git a/regression-test/suites/nereids_p0/aggregate/window_funnel.groovy b/regression-test/suites/nereids_p0/aggregate/window_funnel.groovy index 7b551c28ac2440..378311a575fd74 100644 --- a/regression-test/suites/nereids_p0/aggregate/window_funnel.groovy +++ b/regression-test/suites/nereids_p0/aggregate/window_funnel.groovy @@ -111,7 +111,7 @@ suite("window_funnel") { strBuilder.append("curl --location-trusted -u " + context.config.jdbcUser + ":" + context.config.jdbcPassword) strBuilder.append(" http://" + context.config.feHttpAddress + "/rest/v1/config/fe?conf_item=be_exec_version") - String command = strBuilder.toString() + def command = strBuilder.toString() def process = command.toString().execute() def code = process.waitFor() def err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); @@ -130,8 +130,73 @@ suite("window_funnel") { } } if (beExecVersion < 3) { + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ + CREATE TABLE IF NOT EXISTS ${tableName} ( + xwho varchar(50) NULL COMMENT 'xwho', + xwhen datetimev2(3) COMMENT 'xwhen', + xwhat int NULL COMMENT 'xwhat' + ) + DUPLICATE KEY(xwho) + DISTRIBUTED BY HASH(xwho) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql "INSERT into ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 10:41:00.111111', 1)" + sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:02.111111', 2)" + sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:03.111111', 2)" + sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 14:15:01.111111', 3)" + sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 15:05:04.111111', 4)" + qt_window_funnel_deduplication_compat """ + select + window_funnel( + 20000, + 'deduplication', + t.xwhen, + t.xwhat = 1, + t.xwhat = 2, + t.xwhat = 3, + t.xwhat = 4 + ) AS level + from ${tableName} t; + """ + sql """ DROP TABLE IF EXISTS ${tableName} """ logger.warn("Be exec version(${beExecVersion}) is less than 3, skip window_funnel mode test") return + } else { + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ + CREATE TABLE IF NOT EXISTS ${tableName} ( + xwho varchar(50) NULL COMMENT 'xwho', + xwhen datetimev2(3) COMMENT 'xwhen', + xwhat int NULL COMMENT 'xwhat' + ) + DUPLICATE KEY(xwho) + DISTRIBUTED BY HASH(xwho) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql "INSERT into ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 10:41:00.111111', 1)" + sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:02.111111', 2)" + sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:03.111111', 2)" + sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 14:15:01.111111', 3)" + sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 15:05:04.111111', 4)" + qt_window_funnel_deduplication_compat """ + select + window_funnel( + 20000, + 'default', + t.xwhen, + t.xwhat = 1, + t.xwhat = 2, + t.xwhat = 3, + t.xwhat = 4 + ) AS level + from ${tableName} t; + """ + sql """ DROP TABLE IF EXISTS ${tableName} """ } sql """ DROP TABLE IF EXISTS ${tableName} """ diff --git a/regression-test/suites/nereids_p0/sql_functions/window_functions/window_funnel.sql b/regression-test/suites/nereids_p0/sql_functions/window_functions/window_funnel.sql deleted file mode 100644 index 00fc65c1cc848c..00000000000000 --- a/regression-test/suites/nereids_p0/sql_functions/window_functions/window_funnel.sql +++ /dev/null @@ -1,92 +0,0 @@ -DROP TABLE IF EXISTS windowfunnel_test; - -CREATE TABLE IF NOT EXISTS windowfunnel_test ( - `xwho` varchar(50) NULL COMMENT 'xwho', - `xwhen` datetime COMMENT 'xwhen', - `xwhat` int NULL COMMENT 'xwhat' - ) -DUPLICATE KEY(xwho) -DISTRIBUTED BY HASH(xwho) BUCKETS 3 -PROPERTIES ( - "replication_num" = "1" -); - -INSERT into windowfunnel_test (xwho, xwhen, xwhat) values ('1', '2022-03-12 10:41:00', 1), - ('1', '2022-03-12 13:28:02', 2), - ('1', '2022-03-12 16:15:01', 3), - ('1', '2022-03-12 19:05:04', 4); - -select * from windowfunnel_test; - -select window_funnel(1, 'default', t.xwhen, t.xwhat = 1, t.xwhat = 2 ) AS level from windowfunnel_test t; -select window_funnel(3600 * 3, 'default', t.xwhen, t.xwhat = 1, t.xwhat = 2 ) AS level from windowfunnel_test t; - - -CREATE TABLE IF NOT EXISTS user_analysis -( - user_id INT NOT NULL , - event_type varchar(20) , - event_time datetime NOT NULL -) -DUPLICATE KEY(`user_id`, `event_type`) -DISTRIBUTED BY HASH(`event_type`) BUCKETS 2 -PROPERTIES ( - "replication_num" = "1" -); - - -insert into user_analysis values (1000001,'browse', '2022-07-17 00:00:00'); -insert into user_analysis values (1000002,'browse','2022-07-18 00:00:00'); -insert into user_analysis values (1000003,'shopping cart','2022-07-19 00:00:00'); -insert into user_analysis values (1000004,'browse','2022-07-20 00:00:00'); -insert into user_analysis values (1000005,'browse','2022-07-21 00:00:00'); -insert into user_analysis values (1000006,'favorite','2022-07-22 00:00:00'); -insert into user_analysis values (1000007,'browse','2022-07-23 00:00:00'); -insert into user_analysis values (1000008,'browse','2022-07-23 23:31:00'); -insert into user_analysis values (1000008,'favorite','2022-07-23 23:50:00'); -insert into user_analysis values (1000008,'shopping cart','2022-07-23 23:58:00'); -insert into user_analysis values (1000008,'buy','2022-07-24 00:00:00'); -insert into user_analysis values (1000009,'browse','2022-07-25 00:00:00'); -insert into user_analysis values (1000010,'favorite','2022-07-26 00:00:00'); -insert into user_analysis values (1000007,'browse','2022-07-27 00:00:00'); -insert into user_analysis values (1000012,'browse','2022-07-28 00:00:00'); -insert into user_analysis values (1000013,'browse','2022-07-29 00:00:00'); -insert into user_analysis values (1000014,'browse','2022-07-30 00:00:00'); -insert into user_analysis values (1000015,'browse','2022-07-31 00:00:00'); - -WITH - level_detail AS ( - SELECT - level - ,COUNT(1) AS count_user - FROM ( - SELECT - user_id - ,window_funnel( - 1800 - ,'default' - ,event_time - ,event_type = 'browse' - ,event_type = 'favorite' - ,event_type = 'shopping cart' - ,event_type = 'buy' - ) AS level - FROM user_analysis - WHERE event_time >= TIMESTAMP '2022-07-17 00:00:00' - AND event_time < TIMESTAMP '2022-07-31 00:00:00' - GROUP BY user_id - ) AS basic_table - GROUP BY level - ORDER BY level ASC ) -SELECT CASE level WHEN 0 THEN 'users' - WHEN 1 THEN 'browser' - WHEN 2 THEN 'favorite' - WHEN 3 THEN 'shopping cart' - WHEN 4 THEN 'buy' - END - ,SUM(count_user) over ( ORDER BY level DESC ) -FROM level_detail -GROUP BY level - ,count_user -ORDER BY level ASC; - diff --git a/regression-test/suites/query_p0/aggregate/window_funnel.groovy b/regression-test/suites/query_p0/aggregate/window_funnel.groovy deleted file mode 100644 index 63d21a9663bbb0..00000000000000 --- a/regression-test/suites/query_p0/aggregate/window_funnel.groovy +++ /dev/null @@ -1,360 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -// The cases is copied from https://github.com/trinodb/trino/tree/master -// /testing/trino-product-tests/src/main/resources/sql-tests/testcases/aggregate -// and modified by Doris. - -import org.codehaus.groovy.runtime.IOGroovyMethods - -suite("window_funnel") { - def tableName = "windowfunnel_test" - - sql """ DROP TABLE IF EXISTS ${tableName} """ - sql """ - CREATE TABLE IF NOT EXISTS ${tableName} ( - xwho varchar(50) NULL COMMENT 'xwho', - xwhen datetime COMMENT 'xwhen', - xwhat int NULL COMMENT 'xwhat' - ) - DUPLICATE KEY(xwho) - DISTRIBUTED BY HASH(xwho) BUCKETS 3 - PROPERTIES ( - "replication_num" = "1" - ); - """ - sql "INSERT into ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 10:41:00', 1)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:02', 2)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 16:15:01', 3)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 19:05:04', 4)" - - qt_window_funnel """ select - window_funnel( - 1, - 'default', - t.xwhen, - t.xwhat = 1, - t.xwhat = 2 - ) AS level - from ${tableName} t; - """ - qt_window_funnel """ select - window_funnel( - 20000, - 'default', - t.xwhen, - t.xwhat = 1, - t.xwhat = 2 - ) AS level - from ${tableName} t; - """ - - sql """ DROP TABLE IF EXISTS ${tableName} """ - sql """ - CREATE TABLE IF NOT EXISTS ${tableName} ( - xwho varchar(50) NULL COMMENT 'xwho', - xwhen datetimev2(3) COMMENT 'xwhen', - xwhat int NULL COMMENT 'xwhat' - ) - DUPLICATE KEY(xwho) - DISTRIBUTED BY HASH(xwho) BUCKETS 3 - PROPERTIES ( - "replication_num" = "1" - ); - """ - sql "INSERT into ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 10:41:00.111111', 1)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:02.111111', 2)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 16:15:01.111111', 3)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 19:05:04.111111', 4)" - - qt_window_funnel """ - select - window_funnel( - 1, - 'default', - t.xwhen, - t.xwhat = 1, - t.xwhat = 2 - ) AS level - from ${tableName} t; - """ - qt_window_funnel """ - select - window_funnel( - 20000, - 'default', - t.xwhen, - t.xwhat = 1, - t.xwhat = 2 - ) AS level - from ${tableName} t; - """ - sql """ DROP TABLE IF EXISTS ${tableName} """ - - StringBuilder strBuilder = new StringBuilder() - strBuilder.append("curl --location-trusted -u " + context.config.jdbcUser + ":" + context.config.jdbcPassword) - strBuilder.append(" http://" + context.config.feHttpAddress + "/rest/v1/config/fe?conf_item=be_exec_version") - - def command = strBuilder.toString() - def process = command.toString().execute() - def code = process.waitFor() - def err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); - def out = process.getText() - logger.info("Request FE Config: code=" + code + ", out=" + out + ", err=" + err) - assertEquals(code, 0) - def response = parseJson(out.trim()) - assertEquals(response.code, 0) - assertEquals(response.msg, "success") - def configJson = response.data.rows - def beExecVersion = 0 - for (Object conf: configJson) { - assert conf instanceof Map - if (((Map) conf).get("Name").toLowerCase() == "be_exec_version") { - beExecVersion = ((Map) conf).get("Value").toInteger() - } - } - if (beExecVersion < 3) { - sql """ DROP TABLE IF EXISTS ${tableName} """ - sql """ - CREATE TABLE IF NOT EXISTS ${tableName} ( - xwho varchar(50) NULL COMMENT 'xwho', - xwhen datetimev2(3) COMMENT 'xwhen', - xwhat int NULL COMMENT 'xwhat' - ) - DUPLICATE KEY(xwho) - DISTRIBUTED BY HASH(xwho) BUCKETS 3 - PROPERTIES ( - "replication_num" = "1" - ); - """ - sql "INSERT into ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 10:41:00.111111', 1)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:02.111111', 2)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:03.111111', 2)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 14:15:01.111111', 3)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 15:05:04.111111', 4)" - qt_window_funnel_deduplication_compat """ - select - window_funnel( - 20000, - 'deduplication', - t.xwhen, - t.xwhat = 1, - t.xwhat = 2, - t.xwhat = 3, - t.xwhat = 4 - ) AS level - from ${tableName} t; - """ - sql """ DROP TABLE IF EXISTS ${tableName} """ - logger.warn("Be exec version(${beExecVersion}) is less than 3, skip window_funnel mode test") - return - } else { - sql """ DROP TABLE IF EXISTS ${tableName} """ - sql """ - CREATE TABLE IF NOT EXISTS ${tableName} ( - xwho varchar(50) NULL COMMENT 'xwho', - xwhen datetimev2(3) COMMENT 'xwhen', - xwhat int NULL COMMENT 'xwhat' - ) - DUPLICATE KEY(xwho) - DISTRIBUTED BY HASH(xwho) BUCKETS 3 - PROPERTIES ( - "replication_num" = "1" - ); - """ - sql "INSERT into ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 10:41:00.111111', 1)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:02.111111', 2)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:03.111111', 2)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 14:15:01.111111', 3)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 15:05:04.111111', 4)" - qt_window_funnel_deduplication_compat """ - select - window_funnel( - 20000, - 'default', - t.xwhen, - t.xwhat = 1, - t.xwhat = 2, - t.xwhat = 3, - t.xwhat = 4 - ) AS level - from ${tableName} t; - """ - sql """ DROP TABLE IF EXISTS ${tableName} """ - } - - sql """ DROP TABLE IF EXISTS ${tableName} """ - sql """ - CREATE TABLE IF NOT EXISTS ${tableName} ( - xwho varchar(50) NULL COMMENT 'xwho', - xwhen datetimev2(3) COMMENT 'xwhen', - xwhat int NULL COMMENT 'xwhat' - ) - DUPLICATE KEY(xwho) - DISTRIBUTED BY HASH(xwho) BUCKETS 3 - PROPERTIES ( - "replication_num" = "1" - ); - """ - sql "INSERT into ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 10:41:00.111111', 1)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:02.111111', 2)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:03.111111', 2)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 14:15:01.111111', 3)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 15:05:04.111111', 4)" - qt_window_funnel_deduplication """ - select - window_funnel( - 20000, - 'deduplication', - t.xwhen, - t.xwhat = 1, - t.xwhat = 2, - t.xwhat = 3, - t.xwhat = 4 - ) AS level - from ${tableName} t; - """ - sql """ DROP TABLE IF EXISTS ${tableName} """ - - sql """ DROP TABLE IF EXISTS ${tableName} """ - sql """ - CREATE TABLE IF NOT EXISTS ${tableName} ( - xwho varchar(50) NULL COMMENT 'xwho', - xwhen datetimev2(3) COMMENT 'xwhen', - xwhat int NULL COMMENT 'xwhat' - ) - DUPLICATE KEY(xwho) - DISTRIBUTED BY HASH(xwho) BUCKETS 3 - PROPERTIES ( - "replication_num" = "1" - ); - """ - sql "INSERT into ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 10:41:00.111111', 1)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:02.111111', 2)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 14:15:01.111111', 3)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 15:05:04.111111', 4)" - qt_window_funnel_fixed """ - select - window_funnel( - 20000, - 'fixed', - t.xwhen, - t.xwhat = 1, - t.xwhat = 2, - t.xwhat = 4, - t.xwhat = 3 - ) AS level - from ${tableName} t; - """ - sql """ DROP TABLE IF EXISTS ${tableName} """ - - sql """ DROP TABLE IF EXISTS ${tableName} """ - sql """ - CREATE TABLE IF NOT EXISTS ${tableName} ( - xwho varchar(50) NULL COMMENT 'xwho', - xwhen datetimev2(3) COMMENT 'xwhen', - xwhat int NULL COMMENT 'xwhat' - ) - DUPLICATE KEY(xwho) - DISTRIBUTED BY HASH(xwho) BUCKETS 3 - PROPERTIES ( - "replication_num" = "1" - ); - """ - sql "INSERT into ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 10:41:00.111111', 1)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:02.111111', 2)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 14:15:01.111111', 3)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 15:05:04.111111', 4)" - qt_window_funnel_fixed """ - select - window_funnel( - 20000, - 'fixed', - t.xwhen, - t.xwhat = 4, - t.xwhat = 3, - t.xwhat = 2, - t.xwhat = 1 - ) AS level - from ${tableName} t; - """ - sql """ DROP TABLE IF EXISTS ${tableName} """ - - sql """ DROP TABLE IF EXISTS ${tableName} """ - sql """ - CREATE TABLE IF NOT EXISTS ${tableName} ( - xwho varchar(50) NULL COMMENT 'xwho', - xwhen datetimev2(3) COMMENT 'xwhen', - xwhat int NULL COMMENT 'xwhat' - ) - DUPLICATE KEY(xwho) - DISTRIBUTED BY HASH(xwho) BUCKETS 3 - PROPERTIES ( - "replication_num" = "1" - ); - """ - sql "INSERT into ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 10:41:00.111111', 1)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:02.111111', 2)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:03.111111', 3)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 15:05:04.111111', 4)" - qt_window_funnel_increase """ - select - window_funnel( - 20000, - 'increase', - t.xwhen, - t.xwhat = 1, - t.xwhat = 2, - t.xwhat = 3, - t.xwhat = 4 - ) AS level - from ${tableName} t; - """ - sql """ DROP TABLE IF EXISTS ${tableName} """ - - sql """ DROP TABLE IF EXISTS ${tableName} """ - sql """ - CREATE TABLE IF NOT EXISTS ${tableName} ( - xwho varchar(50) NULL COMMENT 'xwho', - xwhen datetimev2(3) COMMENT 'xwhen', - xwhat int NULL COMMENT 'xwhat' - ) - DUPLICATE KEY(xwho) - DISTRIBUTED BY HASH(xwho) BUCKETS 3 - PROPERTIES ( - "replication_num" = "1" - ); - """ - sql "INSERT into ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 10:41:00.111111', 1)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:02.111111', 2)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 13:28:02.111111', 3)" - sql "INSERT INTO ${tableName} (xwho, xwhen, xwhat) VALUES('1', '2022-03-12 15:05:04.111111', 4)" - qt_window_funnel_increase """ - select - window_funnel( - 20000, - 'increase', - t.xwhen, - t.xwhat = 1, - t.xwhat = 2, - t.xwhat = 3, - t.xwhat = 4 - ) AS level - from ${tableName} t; - """ - sql """ DROP TABLE IF EXISTS ${tableName} """ -} From f87e51ca089a2e8a94b9ae3af0025c083154a021 Mon Sep 17 00:00:00 2001 From: qiye Date: Sun, 1 Sep 2024 12:16:07 +0800 Subject: [PATCH 60/60] [fix](ES Catalog)Do not push down limit to ES when predicates can not be processed by ES. (#40111) --- be/src/pipeline/exec/es_scan_operator.cpp | 5 +- .../external_table_p0/es/test_es_query.out | 104 ++++++++++++++++++ .../external_table_p0/es/test_es_query.groovy | 12 ++ 3 files changed, 119 insertions(+), 2 deletions(-) diff --git a/be/src/pipeline/exec/es_scan_operator.cpp b/be/src/pipeline/exec/es_scan_operator.cpp index 7dfa73a8cdfee2..5c5f1b3eb8021e 100644 --- a/be/src/pipeline/exec/es_scan_operator.cpp +++ b/be/src/pipeline/exec/es_scan_operator.cpp @@ -84,8 +84,9 @@ Status EsScanLocalState::_init_scanners(std::list* sca std::to_string(RuntimeFilterConsumer::_state->batch_size()); properties[ESScanReader::KEY_HOST_PORT] = get_host_and_port(es_scan_range->es_hosts); // push down limit to Elasticsearch - // if predicate in _conjunct_ctxs can not be processed by Elasticsearch, we can not push down limit operator to Elasticsearch - if (p.limit() != -1 && p.limit() <= RuntimeFilterConsumer::_state->batch_size()) { + // if predicate in _conjuncts can not be processed by Elasticsearch, we can not push down limit operator to Elasticsearch + if (p.limit() != -1 && p.limit() <= RuntimeFilterConsumer::_state->batch_size() && + p.conjuncts().empty()) { properties[ESScanReader::KEY_TERMINATE_AFTER] = std::to_string(p.limit()); } diff --git a/regression-test/data/external_table_p0/es/test_es_query.out b/regression-test/data/external_table_p0/es/test_es_query.out index a2289bec1b4927..1e103748dc44c2 100644 --- a/regression-test/data/external_table_p0/es/test_es_query.out +++ b/regression-test/data/external_table_p0/es/test_es_query.out @@ -193,6 +193,16 @@ I'm not null or empty [{"name":"Andy","age":18},{"name":"Tim","age":28}] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] "Andy" "White" [{"name":"Andy","age":18},{"name":"Tim","age":28}] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] "Andy" "White" +-- !sql_5_21 -- +2022-08-08T20:10:10 2022-08-08 20 +2022-08-08T20:10:10 2022-08-08 20 +2022-08-08T20:10:10 2022-08-08 20 +2022-08-08T20:10:10 2022-08-08 20 + +-- !sql_5_22 -- +2022-08-08T12:10:10 2022-08-08 12 +2022-08-08T12:10:10 2022-08-08 12 + -- !sql_6_02 -- [1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] [{"name":"Andy","age":18},{"name":"Tim","age":28}] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] \N string1 text#1 3.14 2022-08-08T00:00 12345 2022-08-08T20:10:10 @@ -265,6 +275,16 @@ I'm not null or empty [{"name":"Andy","age":18},{"name":"Tim","age":28}] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] "Andy" "White" [{"name":"Andy","age":18},{"name":"Tim","age":28}] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] "Andy" "White" +-- !sql_6_21 -- +2022-08-08T20:10:10 2022-08-08 20 +2022-08-08T20:10:10 2022-08-08 20 +2022-08-08T20:10:10 2022-08-08 20 +2022-08-08T20:10:10 2022-08-08 20 + +-- !sql_6_22 -- +2022-08-08T12:10:10 2022-08-08 12 +2022-08-08T12:10:10 2022-08-08 12 + -- !sql_7_02 -- [1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] [{"name":"Andy","age":18},{"name":"Tim","age":28}] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] debug \N This string can be quite lengthy string1 2022-08-08T20:10:10 text#1 3.14 2022-08-08T00:00 2022-08-08T12:10:10 1659931810000 2022-08-08T12:10:10 2022-08-08T20:10:10 12345 @@ -370,6 +390,22 @@ I'm not null or empty [{"name":"Andy","age":18},{"name":"Tim","age":28}] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] "Andy" "White" [{"name":"Andy","age":18},{"name":"Tim","age":28}] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] "Andy" "White" +-- !sql_7_26 -- +2022-08-08T04:10:10 2022-08-08 +2022-08-08T04:10:10 2022-08-08 + +-- !sql_7_27 -- +2022-08-09T12:10:10 2022-08-09 +2022-08-09T12:10:10 2022-08-09 + +-- !sql_7_28 -- +2022-08-10T04:10:10 2022-08-10 +2022-08-10T04:10:10 2022-08-10 + +-- !sql_7_29 -- +2022-08-11T12:10:10 2022-08-11 +2022-08-11T12:10:10 2022-08-11 + -- !sql_7_26 -- value1 value2 @@ -475,6 +511,22 @@ I'm not null or empty [{"name":"Andy","age":18},{"name":"Tim","age":28}] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] "Andy" "White" [{"name":"Andy","age":18},{"name":"Tim","age":28}] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] "Andy" "White" +-- !sql_8_24 -- +2022-08-08T04:10:10 2022-08-08 +2022-08-08T04:10:10 2022-08-08 + +-- !sql_8_25 -- +2022-08-09T12:10:10 2022-08-09 +2022-08-09T12:10:10 2022-08-09 + +-- !sql_8_26 -- +2022-08-10T04:10:10 2022-08-10 +2022-08-10T04:10:10 2022-08-10 + +-- !sql_8_27 -- +2022-08-11T12:10:10 2022-08-11 +2022-08-11T12:10:10 2022-08-11 + -- !sql01 -- ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [-1, 0, 1, 2] [0, 1, 2, 3] ["d", "e", "f"] [128, 129, -129, -130] ["192.168.0.1", "127.0.0.1"] string1 [1, 2, 3, 4] 2022-08-08 2022-08-08T12:10:10 text#1 ["2020-01-01", "2020-01-02"] 3.14 [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] ["a", "b", "c"] [{"name":"Andy","age":18},{"name":"Tim","age":28}] 2022-08-08T12:10:10 2022-08-08T12:10:10 2022-08-08T20:10:10 [1, -2, -3, 4] [1, 0, 1, 1] [32768, 32769, -32769, -32770] \N [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] @@ -608,6 +660,16 @@ I'm not null or empty [{"name":"Andy","age":18},{"name":"Tim","age":28}] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] "Andy" "White" [{"name":"Andy","age":18},{"name":"Tim","age":28}] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] "Andy" "White" +-- !sql_5_21 -- +2022-08-08T20:10:10 2022-08-08 20 +2022-08-08T20:10:10 2022-08-08 20 +2022-08-08T20:10:10 2022-08-08 20 +2022-08-08T20:10:10 2022-08-08 20 + +-- !sql_5_22 -- +2022-08-08T12:10:10 2022-08-08 12 +2022-08-08T12:10:10 2022-08-08 12 + -- !sql_6_02 -- [1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] [{"name":"Andy","age":18},{"name":"Tim","age":28}] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] \N string1 text#1 3.14 2022-08-08T00:00 12345 2022-08-08T20:10:10 @@ -680,6 +742,16 @@ I'm not null or empty [{"name":"Andy","age":18},{"name":"Tim","age":28}] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] "Andy" "White" [{"name":"Andy","age":18},{"name":"Tim","age":28}] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] "Andy" "White" +-- !sql_6_21 -- +2022-08-08T20:10:10 2022-08-08 20 +2022-08-08T20:10:10 2022-08-08 20 +2022-08-08T20:10:10 2022-08-08 20 +2022-08-08T20:10:10 2022-08-08 20 + +-- !sql_6_22 -- +2022-08-08T12:10:10 2022-08-08 12 +2022-08-08T12:10:10 2022-08-08 12 + -- !sql_7_02 -- [1, 0, 1, 1] [1, -2, -3, 4] ["2020-01-01", "2020-01-02"] ["2020-01-01 12:00:00", "2020-01-02 13:01:01"] [1, 2, 3, 4] [1, 1.1, 1.2, 1.3] [1, 2, 3, 4] [32768, 32769, -32769, -32770] ["192.168.0.1", "127.0.0.1"] ["a", "b", "c"] [-1, 0, 1, 2] [{"name":"Andy","age":18},{"name":"Tim","age":28}] [1, 2, 3, 4] [128, 129, -129, -130] ["d", "e", "f"] [0, 1, 2, 3] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] debug \N This string can be quite lengthy string1 2022-08-08T20:10:10 text#1 3.14 2022-08-08T00:00 2022-08-08T12:10:10 1659931810000 2022-08-08T12:10:10 2022-08-08T20:10:10 12345 @@ -785,6 +857,22 @@ I'm not null or empty [{"name":"Andy","age":18},{"name":"Tim","age":28}] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] "Andy" "White" [{"name":"Andy","age":18},{"name":"Tim","age":28}] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] "Andy" "White" +-- !sql_7_26 -- +2022-08-08T04:10:10 2022-08-08 +2022-08-08T04:10:10 2022-08-08 + +-- !sql_7_27 -- +2022-08-09T12:10:10 2022-08-09 +2022-08-09T12:10:10 2022-08-09 + +-- !sql_7_28 -- +2022-08-10T04:10:10 2022-08-10 +2022-08-10T04:10:10 2022-08-10 + +-- !sql_7_29 -- +2022-08-11T12:10:10 2022-08-11 +2022-08-11T12:10:10 2022-08-11 + -- !sql_7_26 -- value1 value2 @@ -890,3 +978,19 @@ I'm not null or empty [{"name":"Andy","age":18},{"name":"Tim","age":28}] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] "Andy" "White" [{"name":"Andy","age":18},{"name":"Tim","age":28}] [{"last":"Smith","first":"John"},{"last":"White","first":"Alice"}] "Andy" "White" +-- !sql_8_24 -- +2022-08-08T04:10:10 2022-08-08 +2022-08-08T04:10:10 2022-08-08 + +-- !sql_8_25 -- +2022-08-09T12:10:10 2022-08-09 +2022-08-09T12:10:10 2022-08-09 + +-- !sql_8_26 -- +2022-08-10T04:10:10 2022-08-10 +2022-08-10T04:10:10 2022-08-10 + +-- !sql_8_27 -- +2022-08-11T12:10:10 2022-08-11 +2022-08-11T12:10:10 2022-08-11 + diff --git a/regression-test/suites/external_table_p0/es/test_es_query.groovy b/regression-test/suites/external_table_p0/es/test_es_query.groovy index 54b1c331de58d9..9d3d2cb80adaa2 100644 --- a/regression-test/suites/external_table_p0/es/test_es_query.groovy +++ b/regression-test/suites/external_table_p0/es/test_es_query.groovy @@ -214,6 +214,8 @@ suite("test_es_query", "p0,external,es,external_docker,external_docker_es") { order_qt_sql_5_18 """select message from test1 where not_null_or_empty(message)""" order_qt_sql_5_19 """select * from test1 where esquery(c_unsigned_long, '{"match":{"c_unsigned_long":0}}')""" order_qt_sql_5_20 """select c_person, c_user, json_extract(c_person, '\$.[0].name'), json_extract(c_user, '\$.[1].last') from test1;""" + order_qt_sql_5_21 """select test6, substring(test6, 1, 13) from test2 where substring(test6, 1, 13) = '2022-08-08 20' limit 4;""" + order_qt_sql_5_22 """select test6, substring(test6, 1, 13) from test2 where substring(test6, 1, 13) = '2022-08-08 12' limit 4;""" try { sql """select * from composite_type_array;""" fail("Should not reach here") @@ -243,6 +245,8 @@ suite("test_es_query", "p0,external,es,external_docker,external_docker_es") { order_qt_sql_6_18 """select message from test1 where not_null_or_empty(message)""" order_qt_sql_6_19 """select * from test1 where esquery(c_person, '{"match":{"c_person.name":"Andy"}}')""" order_qt_sql_6_20 """select c_person, c_user, json_extract(c_person, '\$.[0].name'), json_extract(c_user, '\$.[1].last') from test1;""" + order_qt_sql_6_21 """select test6, substring(test6, 1, 13) from test2 where substring(test6, 1, 13) = '2022-08-08 20' limit 4;""" + order_qt_sql_6_22 """select test6, substring(test6, 1, 13) from test2 where substring(test6, 1, 13) = '2022-08-08 12' limit 4;""" try { sql """select * from composite_type_array;""" fail("Should not reach here") @@ -295,6 +299,10 @@ suite("test_es_query", "p0,external,es,external_docker,external_docker_es") { order_qt_sql_7_23 """select * from test1 where level = 'debug'""" order_qt_sql_7_24 """select * from test1 where esquery(c_float, '{"match":{"c_float":1.1}}')""" order_qt_sql_7_25 """select c_person, c_user, json_extract(c_person, '\$.[0].name'), json_extract(c_user, '\$.[1].last') from test1;""" + order_qt_sql_7_26 """select test7,substring(test7, 1, 10) from test2 where substring(test7, 1, 10)='2022-08-08' limit 2;""" + order_qt_sql_7_27 """select test7,substring(test7, 1, 10) from test2 where substring(test7, 1, 10)='2022-08-09' limit 2;""" + order_qt_sql_7_28 """select test7,substring(test7, 1, 10) from test2 where substring(test7, 1, 10)='2022-08-10' limit 2;""" + order_qt_sql_7_29 """select test7,substring(test7, 1, 10) from test2 where substring(test7, 1, 10)='2022-08-11' limit 2;""" try { sql """select * from composite_type_array;""" fail("Should not reach here") @@ -347,6 +355,10 @@ suite("test_es_query", "p0,external,es,external_docker,external_docker_es") { order_qt_sql_8_21 """select * from test1 where level = 'debug'""" order_qt_sql_8_22 """select * from test1 where esquery(c_ip, '{"match":{"c_ip":"192.168.0.1"}}')""" order_qt_sql_8_23 """select c_person, c_user, json_extract(c_person, '\$.[0].name'), json_extract(c_user, '\$.[1].last') from test1;""" + order_qt_sql_8_24 """select test7,substring(test7, 1, 10) from test2 where substring(test7, 1, 10)='2022-08-08' limit 2;""" + order_qt_sql_8_25 """select test7,substring(test7, 1, 10) from test2 where substring(test7, 1, 10)='2022-08-09' limit 2;""" + order_qt_sql_8_26 """select test7,substring(test7, 1, 10) from test2 where substring(test7, 1, 10)='2022-08-10' limit 2;""" + order_qt_sql_8_27 """select test7,substring(test7, 1, 10) from test2 where substring(test7, 1, 10)='2022-08-11' limit 2;""" try { sql """select * from composite_type_array;""" fail("Should not reach here")