From 9ff269b94a5e5bfdfc8ab49c8716be3b2ccede4d Mon Sep 17 00:00:00 2001 From: jianhao <1367919489@qq.com> Date: Fri, 31 Oct 2025 14:31:16 +0800 Subject: [PATCH 1/7] jh-v1 --- be/src/vec/functions/function_interval.cpp | 143 ++++++++++++++ .../vec/functions/simple_function_factory.h | 2 + .../doris/catalog/BuiltinScalarFunctions.java | 6 +- .../functions/scalar/Interval.java | 62 +++++++ .../visitor/ScalarFunctionVisitor.java | 5 + .../function_p0/test_interval_function.out | 175 ++++++++++++++++++ .../function_p0/test_interval_function.groovy | 165 +++++++++++++++++ 7 files changed, 556 insertions(+), 2 deletions(-) create mode 100644 be/src/vec/functions/function_interval.cpp create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Interval.java create mode 100644 regression-test/data/function_p0/test_interval_function.out create mode 100644 regression-test/suites/function_p0/test_interval_function.groovy diff --git a/be/src/vec/functions/function_interval.cpp b/be/src/vec/functions/function_interval.cpp new file mode 100644 index 00000000000000..c14bcd9894fa94 --- /dev/null +++ b/be/src/vec/functions/function_interval.cpp @@ -0,0 +1,143 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include +#include +#include +#include + +#include "vec/columns/column.h" +#include "vec/columns/column_nullable.h" +#include "vec/columns/column_vector.h" +#include "vec/common/assert_cast.h" +#include "vec/core/block.h" +#include "vec/core/column_numbers.h" +#include "vec/core/column_with_type_and_name.h" +#include "vec/core/types.h" +#include "vec/data_types/data_type.h" +#include "vec/data_types/data_type_number.h" +#include "vec/functions/function.h" +#include "vec/functions/simple_function_factory.h" + +namespace doris::vectorized { + +class FunctionInterval : public IFunction { +public: + static constexpr auto name = "interval"; + static FunctionPtr create() { return std::make_shared(); } + + String get_name() const override { return name; } + bool is_variadic() const override { return true; } + size_t get_number_of_arguments() const override { return 0; } + bool use_default_implementation_for_nulls() const override { return true; } + + DataTypePtr get_return_type_impl(const DataTypes& /*arguments*/) const override { + return std::make_shared(); + } + + Status execute_impl(FunctionContext* /*context*/, Block& block, const ColumnNumbers& arguments, + uint32_t result, size_t input_rows_count) const override { + if (arguments.size() < 2) { + return Status::InvalidArgument("interval requires at least 2 arguments"); + } + + auto res_col = ColumnInt32::create(); + auto& res_data = res_col->get_data(); + res_data.resize(input_rows_count); + + auto compare_cwn = block.get_by_position(arguments[0]); + auto compare_col_ptr = compare_cwn.column; + bool compare_is_const = false; + std::tie(compare_col_ptr, compare_is_const) = unpack_if_const(compare_col_ptr); + + switch (compare_cwn.type->get_primitive_type()) { + case PrimitiveType::TYPE_TINYINT: + compute_interval(block, arguments, *compare_col_ptr, compare_is_const, + res_data); + break; + case PrimitiveType::TYPE_SMALLINT: + compute_interval(block, arguments, *compare_col_ptr, compare_is_const, + res_data); + break; + case PrimitiveType::TYPE_INT: + compute_interval(block, arguments, *compare_col_ptr, compare_is_const, + res_data); + break; + case PrimitiveType::TYPE_BIGINT: + compute_interval(block, arguments, *compare_col_ptr, compare_is_const, + res_data); + break; + case PrimitiveType::TYPE_LARGEINT: + compute_interval(block, arguments, *compare_col_ptr, compare_is_const, + res_data); + break; + default: + return Status::InvalidArgument( + "interval only supports integer numeric types for the first argument"); + } + + block.replace_by_position(result, std::move(res_col)); + return Status::OK(); + } + +private: + template + static void compute_interval(Block& block, const ColumnNumbers& arguments, const IColumn& compare_col, + bool compare_is_const, PaddedPODArray& res) { + const auto& compare_data = assert_cast(compare_col).get_data(); + const size_t rows = res.size(); + const size_t num_thresholds = arguments.size() - 1; + + for (size_t row = 0; row < rows; ++row) { + auto compare_val = compare_data[index_check_const(row, compare_is_const)]; + + std::vector thresholds; + thresholds.reserve(num_thresholds); + + for (size_t i = 1; i < arguments.size(); ++i) { + const auto& col_cwn = block.get_by_position(arguments[i]); + ColumnPtr col_ptr = col_cwn.column; + bool is_const = false; + std::tie(col_ptr, is_const) = unpack_if_const(col_ptr); + const auto& th_col = assert_cast(*col_ptr); + thresholds.push_back(th_col.get_data()[index_check_const(row, is_const)]); + } + + size_t left = 0; + size_t right = num_thresholds; + size_t result_idx = num_thresholds; + + while (left < right) { + size_t mid = left + (right - left) / 2; + if (thresholds[mid] > compare_val) { + result_idx = mid; + right = mid; + } else { + left = mid + 1; + } + } + + res[row] = static_cast(result_idx); + } + } +}; + +void register_function_interval(SimpleFunctionFactory& factory) { + factory.register_function(); +} + +} // namespace doris::vectorized diff --git a/be/src/vec/functions/simple_function_factory.h b/be/src/vec/functions/simple_function_factory.h index 54b635ecc1bb38..0ae03effcc6d78 100644 --- a/be/src/vec/functions/simple_function_factory.h +++ b/be/src/vec/functions/simple_function_factory.h @@ -98,6 +98,7 @@ void register_function_geo(SimpleFunctionFactory& factory); void register_function_multi_string_position(SimpleFunctionFactory& factory); void register_function_multi_string_search(SimpleFunctionFactory& factory); void register_function_width_bucket(SimpleFunctionFactory& factory); +void register_function_interval(SimpleFunctionFactory& factory); void register_function_ignore(SimpleFunctionFactory& factory); void register_function_encryption(SimpleFunctionFactory& factory); void register_function_regexp_extract(SimpleFunctionFactory& factory); @@ -337,6 +338,7 @@ class SimpleFunctionFactory { register_function_multi_string_position(instance); register_function_multi_string_search(instance); register_function_width_bucket(instance); + register_function_interval(instance); register_function_match(instance); register_function_ip(instance); register_function_tokenize(instance); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java index 7e62f190c9c579..24d837ffc841ed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java @@ -247,6 +247,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.InnerProduct; import org.apache.doris.nereids.trees.expressions.functions.scalar.InnerProductApproximate; import org.apache.doris.nereids.trees.expressions.functions.scalar.Instr; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Interval; import org.apache.doris.nereids.trees.expressions.functions.scalar.InttoUuid; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4CIDRToRange; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4NumToString; @@ -785,6 +786,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(InnerProductApproximate.class, "inner_product_approximate"), scalar(InnerProduct.class, "inner_product"), scalar(Instr.class, "instr"), + scalar(Interval.class, "interval"), scalar(InttoUuid.class, "int_to_uuid"), scalar(Ipv4NumToString.class, "ipv4_num_to_string", "inet_ntoa"), scalar(Ipv4StringToNum.class, "ipv4_string_to_num"), @@ -1066,10 +1068,10 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(TrimIn.class, "trim_in"), scalar(Truncate.class, "truncate"), scalar(Unhex.class, "unhex"), - scalar(UnhexNull.class, "unhex_null"), + scalar(Uncompress.class, "uncompress"), + scalar(UnhexNull.class, "unhex_null"), scalar(UnixTimestamp.class, "unix_timestamp"), scalar(Upper.class, "ucase", "upper"), - scalar(Uncompress.class, "uncompress"), scalar(UnicodeNormalize.class, "unicode_normalize"), scalar(Uniform.class, "uniform"), scalar(UrlDecode.class, "url_decode"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Interval.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Interval.java new file mode 100644 index 00000000000000..0c8f604b1edf1a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Interval.java @@ -0,0 +1,62 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.types.BigIntType; +import org.apache.doris.nereids.types.IntegerType; +import org.apache.doris.nereids.types.LargeIntType; +import org.apache.doris.nereids.types.SmallIntType; +import org.apache.doris.nereids.types.TinyIntType; +import org.apache.doris.nereids.util.ExpressionUtils; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'interval'. + */ +public class Interval extends ScalarFunction implements ExplicitlyCastableSignature, PropagateNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(IntegerType.INSTANCE).varArgs(TinyIntType.INSTANCE, TinyIntType.INSTANCE), + FunctionSignature.ret(IntegerType.INSTANCE).varArgs(SmallIntType.INSTANCE, SmallIntType.INSTANCE), + FunctionSignature.ret(IntegerType.INSTANCE).varArgs(IntegerType.INSTANCE, IntegerType.INSTANCE), + FunctionSignature.ret(IntegerType.INSTANCE).varArgs(BigIntType.INSTANCE, BigIntType.INSTANCE), + FunctionSignature.ret(IntegerType.INSTANCE).varArgs(LargeIntType.INSTANCE, LargeIntType.INSTANCE)); + + public Interval(Expression arg, Expression... varArgs) { + super("interval", ExpressionUtils.mergeArguments(arg, varArgs)); + } + + @Override + public Interval withChildren(List children) { + Preconditions.checkArgument(children.size() >= 2); + return new Interval(children.get(0), children.subList(1, children.size()).toArray(new Expression[0])); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java index 5a556add8376b3..8818cc9ec2b7e6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java @@ -255,6 +255,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.InnerProduct; import org.apache.doris.nereids.trees.expressions.functions.scalar.InnerProductApproximate; import org.apache.doris.nereids.trees.expressions.functions.scalar.Instr; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Interval; import org.apache.doris.nereids.trees.expressions.functions.scalar.InttoUuid; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4CIDRToRange; import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4NumToString; @@ -1469,6 +1470,10 @@ default R visitInstr(Instr instr, C context) { return visitScalarFunction(instr, context); } + default R visitInterval(Interval interval, C context) { + return visitScalarFunction(interval, context); + } + default R visitIpv4NumToString(Ipv4NumToString ipv4NumToString, C context) { return visitScalarFunction(ipv4NumToString, context); } diff --git a/regression-test/data/function_p0/test_interval_function.out b/regression-test/data/function_p0/test_interval_function.out new file mode 100644 index 00000000000000..f2f0783ed8f048 --- /dev/null +++ b/regression-test/data/function_p0/test_interval_function.out @@ -0,0 +1,175 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !interval_basic_1 -- +3 + +-- !interval_basic_2 -- +2 + +-- !interval_basic_3 -- +0 + +-- !interval_basic_4 -- +4 + +-- !interval_basic_5 -- +4 + +-- !interval_boundary_min -- +0 + +-- !interval_boundary_first -- +1 + +-- !interval_boundary_last -- +3 + +-- !interval_boundary_max -- +3 + +-- !interval_boundary_between_1 -- +1 + +-- !interval_boundary_between_2 -- +2 + +-- !interval_negative_1 -- +3 + +-- !interval_negative_2 -- +3 + +-- !interval_negative_3 -- +4 + +-- !interval_negative_4 -- +4 + +-- !interval_duplicate_thresholds_1 -- +3 + +-- !interval_duplicate_thresholds_2 -- +3 + +-- !interval_duplicate_thresholds_3 -- +5 + +-- !interval_single_threshold_1 -- +0 + +-- !interval_single_threshold_2 -- +1 + +-- !interval_single_threshold_3 -- +1 + +-- !interval_two_thresholds_1 -- +0 + +-- !interval_two_thresholds_2 -- +1 + +-- !interval_two_thresholds_3 -- +1 + +-- !interval_two_thresholds_4 -- +2 + +-- !interval_two_thresholds_5 -- +2 + +-- !interval_tinyint -- +1 + +-- !interval_smallint -- +2 + +-- !interval_int -- +2 + +-- !interval_bigint -- +2 + +-- !interval_largeint -- +2 + +-- !interval_null_first_arg -- +\N + +-- !interval_null_threshold -- +\N + +-- !interval_from_table_int -- +1 2 +2 3 +3 4 +4 0 +5 5 +6 3 +7 3 + +-- !interval_from_table_tinyint -- +1 0 +2 1 +3 2 +4 0 +5 5 +6 1 +7 2 + +-- !interval_from_table_smallint -- +1 1 +2 2 +3 3 +4 0 +5 5 +6 2 +7 2 + +-- !interval_from_table_bigint -- +1 3 +2 4 +3 5 +4 0 +5 5 +6 4 +7 3 + +-- !interval_from_table_largeint -- +1 4 +2 5 +3 5 +4 0 +5 5 +6 5 +7 4 + +-- !interval_with_const_thresholds -- +1 2 +2 3 +3 4 +4 0 +5 5 +6 3 +7 0 + +-- !interval_with_const_value -- +1 2 +2 2 +3 2 +4 2 +5 2 +6 2 +7 5 + +-- !interval_complex_1 -- +4 + +-- !interval_complex_2 -- +0 + +-- !interval_complex_3 -- +7 + +-- !interval_complex_4 -- +7 + diff --git a/regression-test/suites/function_p0/test_interval_function.groovy b/regression-test/suites/function_p0/test_interval_function.groovy new file mode 100644 index 00000000000000..9d8b1e52ee0af7 --- /dev/null +++ b/regression-test/suites/function_p0/test_interval_function.groovy @@ -0,0 +1,165 @@ +// 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_interval_function") { + + qt_interval_basic_1 """ SELECT `INTERVAL`(23, 1, 15, 17, 30, 44, 200); """ + qt_interval_basic_2 """ SELECT `INTERVAL`(10, 1, 10, 100, 1000); """ + qt_interval_basic_3 """ SELECT `INTERVAL`(22, 23, 30, 44, 200); """ + qt_interval_basic_4 """ SELECT `INTERVAL`(33, 1, 10, 32, 32, 102, 200); """ + qt_interval_basic_5 """ SELECT `INTERVAL`(33, 1, 10, 32, 33, 102, 200); """ + + qt_interval_boundary_min """ SELECT `INTERVAL`(0, 1, 10, 100); """ + qt_interval_boundary_first """ SELECT `INTERVAL`(1, 1, 10, 100); """ + qt_interval_boundary_last """ SELECT `INTERVAL`(100, 1, 10, 100); """ + qt_interval_boundary_max """ SELECT `INTERVAL`(200, 1, 10, 100); """ + qt_interval_boundary_between_1 """ SELECT `INTERVAL`(5, 1, 10, 100); """ + qt_interval_boundary_between_2 """ SELECT `INTERVAL`(50, 1, 10, 100); """ + + qt_interval_negative_1 """ SELECT `INTERVAL`(-10, -100, -50, -10, 0, 50, 100); """ + qt_interval_negative_2 """ SELECT `INTERVAL`(-5, -100, -50, -10, 0, 50, 100); """ + qt_interval_negative_3 """ SELECT `INTERVAL`(5, -100, -50, -10, 0, 50, 100); """ + qt_interval_negative_4 """ SELECT `INTERVAL`(0, -100, -50, -10, 0, 50, 100); """ + + qt_interval_duplicate_thresholds_1 """ SELECT `INTERVAL`(10, 1, 10, 10, 20, 20, 30); """ + qt_interval_duplicate_thresholds_2 """ SELECT `INTERVAL`(15, 1, 10, 10, 20, 20, 30); """ + qt_interval_duplicate_thresholds_3 """ SELECT `INTERVAL`(25, 1, 10, 10, 20, 20, 30); """ + + qt_interval_single_threshold_1 """ SELECT `INTERVAL`(0, 10); """ + qt_interval_single_threshold_2 """ SELECT `INTERVAL`(10, 10); """ + qt_interval_single_threshold_3 """ SELECT `INTERVAL`(20, 10); """ + + qt_interval_two_thresholds_1 """ SELECT `INTERVAL`(0, 10, 20); """ + qt_interval_two_thresholds_2 """ SELECT `INTERVAL`(10, 10, 20); """ + qt_interval_two_thresholds_3 """ SELECT `INTERVAL`(15, 10, 20); """ + qt_interval_two_thresholds_4 """ SELECT `INTERVAL`(20, 10, 20); """ + qt_interval_two_thresholds_5 """ SELECT `INTERVAL`(30, 10, 20); """ + + qt_interval_tinyint """ SELECT `INTERVAL`(CAST(5 AS TINYINT), CAST(1 AS TINYINT), CAST(10 AS TINYINT), CAST(20 AS TINYINT)); """ + qt_interval_smallint """ SELECT `INTERVAL`(CAST(15 AS SMALLINT), CAST(1 AS SMALLINT), CAST(10 AS SMALLINT), CAST(20 AS SMALLINT)); """ + qt_interval_int """ SELECT `INTERVAL`(CAST(15 AS INT), CAST(1 AS INT), CAST(10 AS INT), CAST(20 AS INT)); """ + qt_interval_bigint """ SELECT `INTERVAL`(CAST(15 AS BIGINT), CAST(1 AS BIGINT), CAST(10 AS BIGINT), CAST(20 AS BIGINT)); """ + qt_interval_largeint """ SELECT `INTERVAL`(CAST(15 AS LARGEINT), CAST(1 AS LARGEINT), CAST(10 AS LARGEINT), CAST(20 AS LARGEINT)); """ + + qt_interval_null_first_arg """ SELECT `INTERVAL`(NULL, 1, 10, 100); """ + qt_interval_null_threshold """ SELECT `INTERVAL`(50, NULL, 10, 100); """ + + def intervalTestTable = "interval_function_test_table" + + sql """ DROP TABLE IF EXISTS ${intervalTestTable}; """ + + sql """ + CREATE TABLE IF NOT EXISTS ${intervalTestTable} ( + id INT, + val_tinyint TINYINT, + val_smallint SMALLINT, + val_int INT, + val_bigint BIGINT, + val_largeint LARGEINT, + thresh1 INT, + thresh2 INT, + thresh3 INT, + thresh4 INT, + thresh5 INT + ) + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ) + """ + + sql """ + INSERT INTO ${intervalTestTable} VALUES + (1, 5, 15, 25, 35, 45, 10, 20, 30, 40, 50), + (2, 15, 25, 35, 45, 55, 10, 20, 30, 40, 50), + (3, 25, 35, 45, 55, 65, 10, 20, 30, 40, 50), + (4, 0, 5, 5, 5, 5, 10, 20, 30, 40, 50), + (5, 60, 60, 60, 60, 60, 10, 20, 30, 40, 50), + (6, 10, 20, 30, 40, 50, 10, 20, 30, 40, 50), + (7, -10, -5, 0, 5, 10, -20, -10, 0, 10, 20), + (8, NULL, NULL, NULL, NULL, NULL, 10, 20, 30, 40, 50) + """ + + qt_interval_from_table_int """ + SELECT id, `INTERVAL`(val_int, thresh1, thresh2, thresh3, thresh4, thresh5) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_from_table_tinyint """ + SELECT id, `INTERVAL`(val_tinyint, thresh1, thresh2, thresh3, thresh4, thresh5) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_from_table_smallint """ + SELECT id, `INTERVAL`(val_smallint, thresh1, thresh2, thresh3, thresh4, thresh5) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_from_table_bigint """ + SELECT id, `INTERVAL`(val_bigint, thresh1, thresh2, thresh3, thresh4, thresh5) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_from_table_largeint """ + SELECT id, `INTERVAL`(val_largeint, thresh1, thresh2, thresh3, thresh4, thresh5) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_with_const_thresholds """ + SELECT id, `INTERVAL`(val_int, 10, 20, 30, 40, 50) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_with_const_value """ + SELECT id, `INTERVAL`(25, thresh1, thresh2, thresh3, thresh4, thresh5) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_complex_1 """ + SELECT `INTERVAL`(100, 1, 10, 50, 100, 200, 500, 1000); + """ + + qt_interval_complex_2 """ + SELECT `INTERVAL`(0, 1, 10, 50, 100, 200, 500, 1000); + """ + + qt_interval_complex_3 """ + SELECT `INTERVAL`(1000, 1, 10, 50, 100, 200, 500, 1000); + """ + + qt_interval_complex_4 """ + SELECT `INTERVAL`(1001, 1, 10, 50, 100, 200, 500, 1000); + """ + + sql """ DROP TABLE IF EXISTS ${intervalTestTable}; """ +} + From 7eb09d92db135276ecfd112ddcb77dbe7c34a5cc Mon Sep 17 00:00:00 2001 From: jianhao <1367919489@qq.com> Date: Fri, 31 Oct 2025 14:53:15 +0800 Subject: [PATCH 2/7] style --- be/src/vec/functions/function_interval.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/be/src/vec/functions/function_interval.cpp b/be/src/vec/functions/function_interval.cpp index c14bcd9894fa94..f87ec57890e74a 100644 --- a/be/src/vec/functions/function_interval.cpp +++ b/be/src/vec/functions/function_interval.cpp @@ -63,7 +63,7 @@ class FunctionInterval : public IFunction { auto compare_col_ptr = compare_cwn.column; bool compare_is_const = false; std::tie(compare_col_ptr, compare_is_const) = unpack_if_const(compare_col_ptr); - + switch (compare_cwn.type->get_primitive_type()) { case PrimitiveType::TYPE_TINYINT: compute_interval(block, arguments, *compare_col_ptr, compare_is_const, @@ -96,8 +96,9 @@ class FunctionInterval : public IFunction { private: template - static void compute_interval(Block& block, const ColumnNumbers& arguments, const IColumn& compare_col, - bool compare_is_const, PaddedPODArray& res) { + static void compute_interval(Block& block, const ColumnNumbers& arguments, + const IColumn& compare_col, bool compare_is_const, + PaddedPODArray& res) { const auto& compare_data = assert_cast(compare_col).get_data(); const size_t rows = res.size(); const size_t num_thresholds = arguments.size() - 1; From 38a59a4ee791ea4ab8e23d434fcdf8e26c79969b Mon Sep 17 00:00:00 2001 From: jianhao <1367919489@qq.com> Date: Mon, 3 Nov 2025 10:49:39 +0800 Subject: [PATCH 3/7] fix_basic --- be/src/vec/functions/function_interval.cpp | 32 ++++++----------- .../function_p0/test_interval_function.out | 36 +++++++++++++++++++ .../function_p0/test_interval_function.groovy | 29 +++++++++++++++ 3 files changed, 76 insertions(+), 21 deletions(-) diff --git a/be/src/vec/functions/function_interval.cpp b/be/src/vec/functions/function_interval.cpp index f87ec57890e74a..2e0d4c1c20e2fb 100644 --- a/be/src/vec/functions/function_interval.cpp +++ b/be/src/vec/functions/function_interval.cpp @@ -15,12 +15,14 @@ // specific language governing permissions and limitations // under the License. +#include #include #include #include #include #include "vec/columns/column.h" +#include "vec/columns/column_const.h" #include "vec/columns/column_nullable.h" #include "vec/columns/column_vector.h" #include "vec/common/assert_cast.h" @@ -43,7 +45,6 @@ class FunctionInterval : public IFunction { String get_name() const override { return name; } bool is_variadic() const override { return true; } size_t get_number_of_arguments() const override { return 0; } - bool use_default_implementation_for_nulls() const override { return true; } DataTypePtr get_return_type_impl(const DataTypes& /*arguments*/) const override { return std::make_shared(); @@ -51,8 +52,8 @@ class FunctionInterval : public IFunction { Status execute_impl(FunctionContext* /*context*/, Block& block, const ColumnNumbers& arguments, uint32_t result, size_t input_rows_count) const override { - if (arguments.size() < 2) { - return Status::InvalidArgument("interval requires at least 2 arguments"); + if (arguments.size() < 2) [[unlikely]] { + return Status::InternalError("interval requires at least 2 arguments"); } auto res_col = ColumnInt32::create(); @@ -60,9 +61,11 @@ class FunctionInterval : public IFunction { res_data.resize(input_rows_count); auto compare_cwn = block.get_by_position(arguments[0]); - auto compare_col_ptr = compare_cwn.column; + ColumnPtr compare_col_ptr = ColumnPtr {}; bool compare_is_const = false; - std::tie(compare_col_ptr, compare_is_const) = unpack_if_const(compare_col_ptr); + compare_is_const = is_column_const(*block.get_by_position(arguments[0]).column); + default_preprocess_parameter_columns(&compare_col_ptr, &compare_is_const, {0}, block, + arguments); switch (compare_cwn.type->get_primitive_type()) { case PrimitiveType::TYPE_TINYINT: @@ -86,7 +89,7 @@ class FunctionInterval : public IFunction { res_data); break; default: - return Status::InvalidArgument( + [[unlikely]] return Status::InternalError( "interval only supports integer numeric types for the first argument"); } @@ -118,21 +121,8 @@ class FunctionInterval : public IFunction { thresholds.push_back(th_col.get_data()[index_check_const(row, is_const)]); } - size_t left = 0; - size_t right = num_thresholds; - size_t result_idx = num_thresholds; - - while (left < right) { - size_t mid = left + (right - left) / 2; - if (thresholds[mid] > compare_val) { - result_idx = mid; - right = mid; - } else { - left = mid + 1; - } - } - - res[row] = static_cast(result_idx); + auto it = std::upper_bound(thresholds.begin(), thresholds.end(), compare_val); + res[row] = static_cast(it - thresholds.begin()); } } }; diff --git a/regression-test/data/function_p0/test_interval_function.out b/regression-test/data/function_p0/test_interval_function.out index f2f0783ed8f048..1ad9d2863e359b 100644 --- a/regression-test/data/function_p0/test_interval_function.out +++ b/regression-test/data/function_p0/test_interval_function.out @@ -161,6 +161,42 @@ 6 2 7 5 +-- !interval_mixed_thresholds_1 -- +1 2 +2 3 +3 4 +4 0 +5 5 +6 3 +7 3 + +-- !interval_mixed_thresholds_2 -- +1 2 +2 3 +3 4 +4 0 +5 5 +6 3 +7 3 + +-- !interval_mixed_thresholds_3 -- +1 2 +2 3 +3 4 +4 0 +5 5 +6 3 +7 3 + +-- !interval_const_value_mixed_thresholds -- +1 2 +2 2 +3 2 +4 2 +5 2 +6 2 +7 3 + -- !interval_complex_1 -- 4 diff --git a/regression-test/suites/function_p0/test_interval_function.groovy b/regression-test/suites/function_p0/test_interval_function.groovy index 9d8b1e52ee0af7..d3ea194dd27877 100644 --- a/regression-test/suites/function_p0/test_interval_function.groovy +++ b/regression-test/suites/function_p0/test_interval_function.groovy @@ -144,6 +144,35 @@ suite("test_interval_function") { ORDER BY id; """ + // Mixed thresholds: some constants, some columns + qt_interval_mixed_thresholds_1 """ + SELECT id, `INTERVAL`(val_int, thresh1, 20, thresh3, 40, 50) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_mixed_thresholds_2 """ + SELECT id, `INTERVAL`(val_int, 10, thresh2, thresh3, 40, thresh5) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_mixed_thresholds_3 """ + SELECT id, `INTERVAL`(val_int, 10, 20, thresh3, thresh4, 50) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_const_value_mixed_thresholds """ + SELECT id, `INTERVAL`(25, thresh1, 20, thresh3, 40, 50) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + qt_interval_complex_1 """ SELECT `INTERVAL`(100, 1, 10, 50, 100, 200, 500, 1000); """ From 170bcf018bf6aadcfb8edbaac5c34a14de32434b Mon Sep 17 00:00:00 2001 From: jianhao <1367919489@qq.com> Date: Mon, 3 Nov 2025 10:55:52 +0800 Subject: [PATCH 4/7] fix_style --- .../java/org/apache/doris/catalog/BuiltinScalarFunctions.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java index 24d837ffc841ed..5b53ddb7e46922 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java @@ -1068,11 +1068,11 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(TrimIn.class, "trim_in"), scalar(Truncate.class, "truncate"), scalar(Unhex.class, "unhex"), - scalar(Uncompress.class, "uncompress"), - scalar(UnhexNull.class, "unhex_null"), + scalar(UnhexNull.class, "unhex_null"), scalar(UnixTimestamp.class, "unix_timestamp"), scalar(Upper.class, "ucase", "upper"), scalar(UnicodeNormalize.class, "unicode_normalize"), + scalar(Uncompress.class, "uncompress"), scalar(Uniform.class, "uniform"), scalar(UrlDecode.class, "url_decode"), scalar(UrlEncode.class, "url_encode"), From 0e179d60f1ec06353aa7c9e20a3f6ac0523ee7ff Mon Sep 17 00:00:00 2001 From: linzhenqi Date: Tue, 9 Dec 2025 12:03:39 +0800 Subject: [PATCH 5/7] lzq --- be/src/vec/columns/column_const.cpp | 8 + be/src/vec/columns/column_const.h | 4 + be/src/vec/functions/function_interval.cpp | 155 ++++++++------- .../functions/function_needs_to_handle_null.h | 27 ++- be/src/vec/functions/function_string.h | 2 +- .../org/apache/doris/nereids/DorisParser.g4 | 1 + .../executable/NumericArithmetic.java | 30 +++ .../functions/scalar/Interval.java | 19 +- .../function_p0/test_interval_function.out | 57 ++++-- .../string_functions/test_string_function.out | Bin 34086 -> 34261 bytes .../function_p0/test_interval_function.groovy | 183 ++++++++++++------ .../test_string_function.groovy | 1 + 12 files changed, 330 insertions(+), 157 deletions(-) diff --git a/be/src/vec/columns/column_const.cpp b/be/src/vec/columns/column_const.cpp index 6f7dd5172339a6..16f80f5baa2c7f 100644 --- a/be/src/vec/columns/column_const.cpp +++ b/be/src/vec/columns/column_const.cpp @@ -136,6 +136,14 @@ std::pair unpack_if_const(const ColumnPtr& ptr) noexcept void default_preprocess_parameter_columns(ColumnPtr* columns, const bool* col_const, const std::initializer_list& parameters, Block& block, const ColumnNumbers& arg_indexes) { + default_preprocess_parameter_columns( + columns, col_const, std::span(parameters.begin(), parameters.end()), + block, arg_indexes); +} + +void default_preprocess_parameter_columns(ColumnPtr* columns, const bool* col_const, + const std::span& parameters, Block& block, + const ColumnNumbers& arg_indexes) { if (std::all_of(parameters.begin(), parameters.end(), [&](size_t const_index) -> bool { return col_const[const_index]; })) { // only need to avoid expanding when all parameters are const diff --git a/be/src/vec/columns/column_const.h b/be/src/vec/columns/column_const.h index 8f21b8f8f63193..9849e84037e564 100644 --- a/be/src/vec/columns/column_const.h +++ b/be/src/vec/columns/column_const.h @@ -92,6 +92,10 @@ void default_preprocess_parameter_columns(ColumnPtr* columns, const bool* col_co const std::initializer_list& parameters, Block& block, const ColumnNumbers& arg_indexes); +void default_preprocess_parameter_columns(ColumnPtr* columns, const bool* col_const, + const std::span& parameters, Block& block, + const ColumnNumbers& arg_indexes); + /** ColumnConst contains another column with single element, * but looks like a column with arbitrary amount of same elements. */ diff --git a/be/src/vec/functions/function_interval.cpp b/be/src/vec/functions/function_interval.cpp index 2e0d4c1c20e2fb..909fa860241bc8 100644 --- a/be/src/vec/functions/function_interval.cpp +++ b/be/src/vec/functions/function_interval.cpp @@ -16,11 +16,15 @@ // under the License. #include +#include #include #include #include #include +#include "common/status.h" +#include "runtime/define_primitive_type.h" +#include "runtime/primitive_type.h" #include "vec/columns/column.h" #include "vec/columns/column_const.h" #include "vec/columns/column_nullable.h" @@ -33,102 +37,109 @@ #include "vec/data_types/data_type.h" #include "vec/data_types/data_type_number.h" #include "vec/functions/function.h" +#include "vec/functions/function_needs_to_handle_null.h" #include "vec/functions/simple_function_factory.h" namespace doris::vectorized { - -class FunctionInterval : public IFunction { +class FunctionIntervalImpl { public: static constexpr auto name = "interval"; - static FunctionPtr create() { return std::make_shared(); } - - String get_name() const override { return name; } - bool is_variadic() const override { return true; } - size_t get_number_of_arguments() const override { return 0; } - - DataTypePtr get_return_type_impl(const DataTypes& /*arguments*/) const override { + static size_t get_number_of_arguments() { return 0; } + static bool is_variadic() { return true; } + static DataTypePtr get_return_type_impl(const DataTypes& /*arguments*/) { return std::make_shared(); } - Status execute_impl(FunctionContext* /*context*/, Block& block, const ColumnNumbers& arguments, - uint32_t result, size_t input_rows_count) const override { - if (arguments.size() < 2) [[unlikely]] { - return Status::InternalError("interval requires at least 2 arguments"); - } - - auto res_col = ColumnInt32::create(); - auto& res_data = res_col->get_data(); - res_data.resize(input_rows_count); - - auto compare_cwn = block.get_by_position(arguments[0]); - ColumnPtr compare_col_ptr = ColumnPtr {}; - bool compare_is_const = false; - compare_is_const = is_column_const(*block.get_by_position(arguments[0]).column); - default_preprocess_parameter_columns(&compare_col_ptr, &compare_is_const, {0}, block, - arguments); + static bool is_return_nullable(bool has_nullable, + const std::vector& cols_info) { + return false; + } - switch (compare_cwn.type->get_primitive_type()) { - case PrimitiveType::TYPE_TINYINT: - compute_interval(block, arguments, *compare_col_ptr, compare_is_const, - res_data); - break; - case PrimitiveType::TYPE_SMALLINT: - compute_interval(block, arguments, *compare_col_ptr, compare_is_const, - res_data); - break; - case PrimitiveType::TYPE_INT: - compute_interval(block, arguments, *compare_col_ptr, compare_is_const, - res_data); - break; - case PrimitiveType::TYPE_BIGINT: - compute_interval(block, arguments, *compare_col_ptr, compare_is_const, - res_data); - break; - case PrimitiveType::TYPE_LARGEINT: - compute_interval(block, arguments, *compare_col_ptr, compare_is_const, - res_data); - break; - default: - [[unlikely]] return Status::InternalError( - "interval only supports integer numeric types for the first argument"); + static bool execute_const_null(ColumnInt32::MutablePtr& res_col, + PaddedPODArray& res_null_map_data, + size_t input_rows_count, size_t null_index) { + if (null_index == 0) { + res_col->insert_many_vals(-1, input_rows_count); + return true; } - - block.replace_by_position(result, std::move(res_col)); - return Status::OK(); + return false; } -private: - template - static void compute_interval(Block& block, const ColumnNumbers& arguments, - const IColumn& compare_col, bool compare_is_const, - PaddedPODArray& res) { - const auto& compare_data = assert_cast(compare_col).get_data(); - const size_t rows = res.size(); - const size_t num_thresholds = arguments.size() - 1; + static void collect_columns_info(std::vector& columns_info, + Block& block, const ColumnNumbers& arguments, + bool& has_nullable) { + const size_t arg_size = arguments.size(); + std::vector arg_col(arg_size); + std::vector is_const(arg_size); + std::vector param_idx(arg_size - 1); + bool all_const = true; + for (size_t i = 0; i < arg_size; ++i) { + is_const[i] = is_column_const(*block.get_by_position(arguments[i]).column); + if (i != 0) { + param_idx[i - 1] = i; + all_const = all_const & is_const[i]; + } + } + arg_col[0] = is_const[0] ? assert_cast( + *block.get_by_position(arguments[0]).column) + .get_data_column_ptr() + : block.get_by_position(arguments[0]).column; + default_preprocess_parameter_columns(arg_col.data(), + reinterpret_cast(is_const.data()), + param_idx, block, arguments); + + // The second parameter's `is_const` is used to indicate whether all the parameters are constants. + columns_info[0].is_const = is_const[0]; + columns_info[1].is_const = all_const; + for (size_t i = 0; i < arguments.size(); ++i) { + ColumnPtr col_ptr = arg_col[i]; + columns_info[i].type = block.get_by_position(arguments[i]).type; + columns_info[i].column = col_ptr; + + if (const auto* nullable = check_and_get_column(col_ptr.get())) { + has_nullable = true; + columns_info[i].nested_col = &nullable->get_nested_column(); + columns_info[i].null_map = &nullable->get_null_map_data(); + } else { + columns_info[i].nested_col = col_ptr.get(); + } + } + } - for (size_t row = 0; row < rows; ++row) { - auto compare_val = compare_data[index_check_const(row, compare_is_const)]; + static void execute(const std::vector& columns_info, + ColumnInt32::MutablePtr& res_col, PaddedPODArray& res_null_map_data, + size_t input_rows_count) { + res_col->resize(input_rows_count); + const auto& compare_data = + assert_cast(*columns_info[0].nested_col).get_data(); + const size_t num_thresholds = columns_info.size() - 1; + + for (size_t row = 0; row < input_rows_count; ++row) { + if (columns_info[0].is_null_at(row)) { + res_col->get_data()[row] = -1; + continue; + } - std::vector thresholds; + // Determine whether all the threshold columns are constant columns. + size_t row_idx = columns_info[1].is_const ? 0 : row; + auto compare_val = compare_data[index_check_const(row, columns_info[0].is_const)]; + std::vector thresholds; thresholds.reserve(num_thresholds); - for (size_t i = 1; i < arguments.size(); ++i) { - const auto& col_cwn = block.get_by_position(arguments[i]); - ColumnPtr col_ptr = col_cwn.column; - bool is_const = false; - std::tie(col_ptr, is_const) = unpack_if_const(col_ptr); - const auto& th_col = assert_cast(*col_ptr); - thresholds.push_back(th_col.get_data()[index_check_const(row, is_const)]); + for (size_t i = 1; i < columns_info.size(); ++i) { + const auto& th_col = assert_cast(*columns_info[i].nested_col); + thresholds.push_back( + columns_info[i].is_null_at(row_idx) ? 0 : th_col.get_data()[row_idx]); } auto it = std::upper_bound(thresholds.begin(), thresholds.end(), compare_val); - res[row] = static_cast(it - thresholds.begin()); + res_col->get_data()[row] = static_cast(it - thresholds.begin()); } } }; void register_function_interval(SimpleFunctionFactory& factory) { - factory.register_function(); + factory.register_function>(); } } // namespace doris::vectorized diff --git a/be/src/vec/functions/function_needs_to_handle_null.h b/be/src/vec/functions/function_needs_to_handle_null.h index 0e60da92e68981..6fc3f60528b548 100644 --- a/be/src/vec/functions/function_needs_to_handle_null.h +++ b/be/src/vec/functions/function_needs_to_handle_null.h @@ -17,6 +17,7 @@ #pragma once #include +#include "udf/udf.h" #include "vec/functions/function.h" namespace doris::vectorized { @@ -24,7 +25,9 @@ namespace doris::vectorized { // Helper struct to store information about const+nullable columns struct ColumnWithConstAndNullMap { + ColumnPtr column = nullptr; const IColumn* nested_col = nullptr; + DataTypePtr type = nullptr; const NullMap* null_map = nullptr; bool is_const = false; @@ -80,8 +83,12 @@ class FunctionNeedsToHandleNull : public IFunction { if (columns_info[i].is_const && columns_info[i].null_map && (*columns_info[i].null_map)[0] && execute_const_null(res_col, null_map_data, input_rows_count, i)) { - block.replace_by_position( - result, ColumnNullable::create(std::move(res_col), std::move(null_map))); + if (is_return_nullable(has_nullable, columns_info)) { + block.replace_by_position(result, ColumnNullable::create(std::move(res_col), + std::move(null_map))); + } else { + block.replace_by_position(result, std::move(res_col)); + } return Status::OK(); } } @@ -121,17 +128,23 @@ class FunctionNeedsToHandleNull : public IFunction { // Collect the required information for each column into columns_info // Including whether it is a constant column, nested column and null map(if exists). - void collect_columns_info(std::vector& columns_info, - const Block& block, const ColumnNumbers& arguments, - bool& has_nullable) const { + void collect_columns_info(std::vector& columns_info, Block& block, + const ColumnNumbers& arguments, bool& has_nullable) const { + if constexpr (requires { + Impl::collect_columns_info(columns_info, block, arguments, has_nullable); + }) { + Impl::collect_columns_info(columns_info, block, arguments, has_nullable); + return; + } for (size_t i = 0; i < arguments.size(); ++i) { ColumnPtr col_ptr; const auto& col_with_type = block.get_by_position(arguments[i]); std::tie(col_ptr, columns_info[i].is_const) = unpack_if_const(col_with_type.column); + columns_info[i].type = col_with_type.type; + columns_info[i].column = col_ptr; - if (is_column_nullable(*col_ptr)) { + if (const auto* nullable = check_and_get_column(col_ptr.get())) { has_nullable = true; - const auto* nullable = check_and_get_column(col_ptr.get()); columns_info[i].nested_col = &nullable->get_nested_column(); columns_info[i].null_map = &nullable->get_null_map_data(); } else { diff --git a/be/src/vec/functions/function_string.h b/be/src/vec/functions/function_string.h index 6a29bac2b658b1..8fe2be2276390d 100644 --- a/be/src/vec/functions/function_string.h +++ b/be/src/vec/functions/function_string.h @@ -5099,7 +5099,7 @@ class FunctionExportSet : public IFunction { bool col_const[5]; ColumnPtr arg_cols[5]; bool all_const = true; - for (int i = 0; i < arg_size; ++i) { + for (int i = 1; i < arg_size; ++i) { col_const[i] = is_column_const(*block.get_by_position(arguments[i]).column); all_const = all_const && col_const[i]; } 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 d916626420678f..f04ce589b5769a 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 @@ -1689,6 +1689,7 @@ functionNameIdentifier | CURRENT_USER | DATABASE | IF + | INTERVAL | LEFT | LIKE | PASSWORD diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/NumericArithmetic.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/NumericArithmetic.java index 3aeb33ff09470d..4169ed6532cc68 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/NumericArithmetic.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/NumericArithmetic.java @@ -1108,4 +1108,34 @@ public static Expression boolor(BooleanLiteral first) { public static Expression boolxor(BooleanLiteral first) { return first; } + + /** + * interval + */ + @ExecFunction(name = "interval") + public static Expression interval(NullLiteral compareValue, Literal... thresholds) { + return new IntegerLiteral(-1); + } + + /** + * interval + */ + @ExecFunction(name = "interval") + public static Expression interval(BigIntLiteral compareValue, BigIntLiteral... thresholds) { + long value = compareValue.getValue(); + if (value < thresholds[0].getValue()) { + return new IntegerLiteral(0); + } + int l = 0; + int r = thresholds.length; + while (l < r) { + int mid = (l + r) >> 1; + if (thresholds[mid].getValue() <= value) { + l = mid + 1; + } else { + r = mid; + } + } + return new IntegerLiteral(l); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Interval.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Interval.java index 0c8f604b1edf1a..78668cb0cb2315 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Interval.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Interval.java @@ -19,13 +19,11 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; -import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BigIntType; import org.apache.doris.nereids.types.IntegerType; -import org.apache.doris.nereids.types.LargeIntType; -import org.apache.doris.nereids.types.SmallIntType; -import org.apache.doris.nereids.types.TinyIntType; import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.base.Preconditions; @@ -36,14 +34,10 @@ /** * ScalarFunction 'interval'. */ -public class Interval extends ScalarFunction implements ExplicitlyCastableSignature, PropagateNullable { +public class Interval extends ScalarFunction implements ExplicitlyCastableSignature, AlwaysNotNullable { public static final List SIGNATURES = ImmutableList.of( - FunctionSignature.ret(IntegerType.INSTANCE).varArgs(TinyIntType.INSTANCE, TinyIntType.INSTANCE), - FunctionSignature.ret(IntegerType.INSTANCE).varArgs(SmallIntType.INSTANCE, SmallIntType.INSTANCE), - FunctionSignature.ret(IntegerType.INSTANCE).varArgs(IntegerType.INSTANCE, IntegerType.INSTANCE), - FunctionSignature.ret(IntegerType.INSTANCE).varArgs(BigIntType.INSTANCE, BigIntType.INSTANCE), - FunctionSignature.ret(IntegerType.INSTANCE).varArgs(LargeIntType.INSTANCE, LargeIntType.INSTANCE)); + FunctionSignature.ret(IntegerType.INSTANCE).varArgs(BigIntType.INSTANCE, BigIntType.INSTANCE)); public Interval(Expression arg, Expression... varArgs) { super("interval", ExpressionUtils.mergeArguments(arg, varArgs)); @@ -59,4 +53,9 @@ public Interval withChildren(List children) { public List getSignatures() { return SIGNATURES; } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitInterval(this, context); + } } diff --git a/regression-test/data/function_p0/test_interval_function.out b/regression-test/data/function_p0/test_interval_function.out index 1ad9d2863e359b..f1c4c66f9384c9 100644 --- a/regression-test/data/function_p0/test_interval_function.out +++ b/regression-test/data/function_p0/test_interval_function.out @@ -77,26 +77,23 @@ -- !interval_two_thresholds_5 -- 2 --- !interval_tinyint -- -1 - --- !interval_smallint -- -2 +-- !interval_null_first_arg -- +-1 --- !interval_int -- +-- !interval_null_threshold -- 2 --- !interval_bigint -- +-- !interval_thresh_partial_null_1 -- 2 --- !interval_largeint -- +-- !interval_thresh_partial_null_2 -- 2 --- !interval_null_first_arg -- -\N +-- !interval_thresh_partial_null_3 -- +3 --- !interval_null_threshold -- -\N +-- !interval_thresh_partial_null_4 -- +3 -- !interval_from_table_int -- 1 2 @@ -161,6 +158,42 @@ 6 2 7 5 +-- !interval_thresh_partial_null_tbl_1 -- +1 2 +2 3 +3 3 +4 1 +5 3 +6 3 +7 3 + +-- !interval_thresh_partial_null_tbl_2 -- +1 2 +2 3 +3 3 +4 2 +5 3 +6 3 +7 3 + +-- !interval_thresh_partial_null_tbl_3 -- +1 3 +2 3 +3 3 +4 0 +5 3 +6 3 +7 3 + +-- !interval_thresh_partial_null_tbl_4 -- +1 3 +2 3 +3 3 +4 3 +5 3 +6 3 +7 3 + -- !interval_mixed_thresholds_1 -- 1 2 2 3 diff --git a/regression-test/data/query_p0/sql_functions/string_functions/test_string_function.out b/regression-test/data/query_p0/sql_functions/string_functions/test_string_function.out index 64cef848f05f2de558cca470198428bb7be9bcd9..e9189ab000c2f6d3d3f2be68749b9416fa13fd23 100644 GIT binary patch delta 184 zcmZ41#dNisX+v#&eQHHPeo;w$acW7tiGhNyE|(#vp$-&s8F3oG*~UO|5YIrzfXf7l zZHmM;gQ^FqF$bFi*3V@DRs+_}Wyu-i$7N^$)ujniY6!9uX11XbG6!Mn=Cb+?tN_gv BB}V`N delta 14 VcmccG&9tnGX+v%OW}SwWtN=1z22KC~ diff --git a/regression-test/suites/function_p0/test_interval_function.groovy b/regression-test/suites/function_p0/test_interval_function.groovy index d3ea194dd27877..b3aff8cb7d7b46 100644 --- a/regression-test/suites/function_p0/test_interval_function.groovy +++ b/regression-test/suites/function_p0/test_interval_function.groovy @@ -17,46 +17,46 @@ suite("test_interval_function") { - qt_interval_basic_1 """ SELECT `INTERVAL`(23, 1, 15, 17, 30, 44, 200); """ - qt_interval_basic_2 """ SELECT `INTERVAL`(10, 1, 10, 100, 1000); """ - qt_interval_basic_3 """ SELECT `INTERVAL`(22, 23, 30, 44, 200); """ - qt_interval_basic_4 """ SELECT `INTERVAL`(33, 1, 10, 32, 32, 102, 200); """ - qt_interval_basic_5 """ SELECT `INTERVAL`(33, 1, 10, 32, 33, 102, 200); """ - - qt_interval_boundary_min """ SELECT `INTERVAL`(0, 1, 10, 100); """ - qt_interval_boundary_first """ SELECT `INTERVAL`(1, 1, 10, 100); """ - qt_interval_boundary_last """ SELECT `INTERVAL`(100, 1, 10, 100); """ - qt_interval_boundary_max """ SELECT `INTERVAL`(200, 1, 10, 100); """ - qt_interval_boundary_between_1 """ SELECT `INTERVAL`(5, 1, 10, 100); """ - qt_interval_boundary_between_2 """ SELECT `INTERVAL`(50, 1, 10, 100); """ - - qt_interval_negative_1 """ SELECT `INTERVAL`(-10, -100, -50, -10, 0, 50, 100); """ - qt_interval_negative_2 """ SELECT `INTERVAL`(-5, -100, -50, -10, 0, 50, 100); """ - qt_interval_negative_3 """ SELECT `INTERVAL`(5, -100, -50, -10, 0, 50, 100); """ - qt_interval_negative_4 """ SELECT `INTERVAL`(0, -100, -50, -10, 0, 50, 100); """ - - qt_interval_duplicate_thresholds_1 """ SELECT `INTERVAL`(10, 1, 10, 10, 20, 20, 30); """ - qt_interval_duplicate_thresholds_2 """ SELECT `INTERVAL`(15, 1, 10, 10, 20, 20, 30); """ - qt_interval_duplicate_thresholds_3 """ SELECT `INTERVAL`(25, 1, 10, 10, 20, 20, 30); """ - - qt_interval_single_threshold_1 """ SELECT `INTERVAL`(0, 10); """ - qt_interval_single_threshold_2 """ SELECT `INTERVAL`(10, 10); """ - qt_interval_single_threshold_3 """ SELECT `INTERVAL`(20, 10); """ - - qt_interval_two_thresholds_1 """ SELECT `INTERVAL`(0, 10, 20); """ - qt_interval_two_thresholds_2 """ SELECT `INTERVAL`(10, 10, 20); """ - qt_interval_two_thresholds_3 """ SELECT `INTERVAL`(15, 10, 20); """ - qt_interval_two_thresholds_4 """ SELECT `INTERVAL`(20, 10, 20); """ - qt_interval_two_thresholds_5 """ SELECT `INTERVAL`(30, 10, 20); """ - - qt_interval_tinyint """ SELECT `INTERVAL`(CAST(5 AS TINYINT), CAST(1 AS TINYINT), CAST(10 AS TINYINT), CAST(20 AS TINYINT)); """ - qt_interval_smallint """ SELECT `INTERVAL`(CAST(15 AS SMALLINT), CAST(1 AS SMALLINT), CAST(10 AS SMALLINT), CAST(20 AS SMALLINT)); """ - qt_interval_int """ SELECT `INTERVAL`(CAST(15 AS INT), CAST(1 AS INT), CAST(10 AS INT), CAST(20 AS INT)); """ - qt_interval_bigint """ SELECT `INTERVAL`(CAST(15 AS BIGINT), CAST(1 AS BIGINT), CAST(10 AS BIGINT), CAST(20 AS BIGINT)); """ - qt_interval_largeint """ SELECT `INTERVAL`(CAST(15 AS LARGEINT), CAST(1 AS LARGEINT), CAST(10 AS LARGEINT), CAST(20 AS LARGEINT)); """ - - qt_interval_null_first_arg """ SELECT `INTERVAL`(NULL, 1, 10, 100); """ - qt_interval_null_threshold """ SELECT `INTERVAL`(50, NULL, 10, 100); """ + qt_interval_basic_1 """ SELECT INTERVAL(23, 1, 15, 17, 30, 44, 200); """ + qt_interval_basic_2 """ SELECT INTERVAL(10, 1, 10, 100, 1000); """ + qt_interval_basic_3 """ SELECT INTERVAL(22, 23, 30, 44, 200); """ + qt_interval_basic_4 """ SELECT INTERVAL(33, 1, 10, 32, 32, 102, 200); """ + qt_interval_basic_5 """ SELECT INTERVAL(33, 1, 10, 32, 33, 102, 200); """ + + qt_interval_boundary_min """ SELECT INTERVAL(0, 1, 10, 100); """ + qt_interval_boundary_first """ SELECT INTERVAL(1, 1, 10, 100); """ + qt_interval_boundary_last """ SELECT INTERVAL(100, 1, 10, 100); """ + qt_interval_boundary_max """ SELECT INTERVAL(200, 1, 10, 100); """ + qt_interval_boundary_between_1 """ SELECT INTERVAL(5, 1, 10, 100); """ + qt_interval_boundary_between_2 """ SELECT INTERVAL(50, 1, 10, 100); """ + + qt_interval_negative_1 """ SELECT INTERVAL(-10, -100, -50, -10, 0, 50, 100); """ + qt_interval_negative_2 """ SELECT INTERVAL(-5, -100, -50, -10, 0, 50, 100); """ + qt_interval_negative_3 """ SELECT INTERVAL(5, -100, -50, -10, 0, 50, 100); """ + qt_interval_negative_4 """ SELECT INTERVAL(0, -100, -50, -10, 0, 50, 100); """ + + qt_interval_duplicate_thresholds_1 """ SELECT INTERVAL(10, 1, 10, 10, 20, 20, 30); """ + qt_interval_duplicate_thresholds_2 """ SELECT INTERVAL(15, 1, 10, 10, 20, 20, 30); """ + qt_interval_duplicate_thresholds_3 """ SELECT INTERVAL(25, 1, 10, 10, 20, 20, 30); """ + + qt_interval_single_threshold_1 """ SELECT INTERVAL(0, 10); """ + qt_interval_single_threshold_2 """ SELECT INTERVAL(10, 10); """ + qt_interval_single_threshold_3 """ SELECT INTERVAL(20, 10); """ + + qt_interval_two_thresholds_1 """ SELECT INTERVAL(0, 10, 20); """ + qt_interval_two_thresholds_2 """ SELECT INTERVAL(10, 10, 20); """ + qt_interval_two_thresholds_3 """ SELECT INTERVAL(15, 10, 20); """ + qt_interval_two_thresholds_4 """ SELECT INTERVAL(20, 10, 20); """ + qt_interval_two_thresholds_5 """ SELECT INTERVAL(30, 10, 20); """ + + qt_interval_null_first_arg """ SELECT INTERVAL(NULL, 1, 10, 100); """ + qt_interval_null_threshold """ SELECT INTERVAL(50, NULL, 10, 100); """ + + // Value not NULL, thresholds partially or fully NULL + qt_interval_thresh_partial_null_1 """ SELECT INTERVAL(50, NULL, 20, 100); """ + qt_interval_thresh_partial_null_2 """ SELECT INTERVAL(50, 10, NULL, 100); """ + qt_interval_thresh_partial_null_3 """ SELECT INTERVAL(50, 10, 20, NULL); """ + qt_interval_thresh_partial_null_4 """ SELECT INTERVAL(50, NULL, NULL, NULL); """ def intervalTestTable = "interval_function_test_table" @@ -96,49 +96,78 @@ suite("test_interval_function") { """ qt_interval_from_table_int """ - SELECT id, `INTERVAL`(val_int, thresh1, thresh2, thresh3, thresh4, thresh5) as result + SELECT id, INTERVAL(val_int, thresh1, thresh2, thresh3, thresh4, thresh5) as result FROM ${intervalTestTable} WHERE id <= 7 ORDER BY id; """ qt_interval_from_table_tinyint """ - SELECT id, `INTERVAL`(val_tinyint, thresh1, thresh2, thresh3, thresh4, thresh5) as result + SELECT id, INTERVAL(val_tinyint, thresh1, thresh2, thresh3, thresh4, thresh5) as result FROM ${intervalTestTable} WHERE id <= 7 ORDER BY id; """ qt_interval_from_table_smallint """ - SELECT id, `INTERVAL`(val_smallint, thresh1, thresh2, thresh3, thresh4, thresh5) as result + SELECT id, INTERVAL(val_smallint, thresh1, thresh2, thresh3, thresh4, thresh5) as result FROM ${intervalTestTable} WHERE id <= 7 ORDER BY id; """ qt_interval_from_table_bigint """ - SELECT id, `INTERVAL`(val_bigint, thresh1, thresh2, thresh3, thresh4, thresh5) as result + SELECT id, INTERVAL(val_bigint, thresh1, thresh2, thresh3, thresh4, thresh5) as result FROM ${intervalTestTable} WHERE id <= 7 ORDER BY id; """ qt_interval_from_table_largeint """ - SELECT id, `INTERVAL`(val_largeint, thresh1, thresh2, thresh3, thresh4, thresh5) as result + SELECT id, INTERVAL(val_largeint, thresh1, thresh2, thresh3, thresh4, thresh5) as result FROM ${intervalTestTable} WHERE id <= 7 ORDER BY id; """ qt_interval_with_const_thresholds """ - SELECT id, `INTERVAL`(val_int, 10, 20, 30, 40, 50) as result + SELECT id, INTERVAL(val_int, 10, 20, 30, 40, 50) as result FROM ${intervalTestTable} WHERE id <= 7 ORDER BY id; """ qt_interval_with_const_value """ - SELECT id, `INTERVAL`(25, thresh1, thresh2, thresh3, thresh4, thresh5) as result + SELECT id, INTERVAL(25, thresh1, thresh2, thresh3, thresh4, thresh5) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + // Value from column, thresholds with mixed column and NULL + qt_interval_thresh_partial_null_tbl_1 """ + SELECT id, INTERVAL(val_int, NULL, thresh2, thresh3) AS result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_thresh_partial_null_tbl_2 """ + SELECT id, INTERVAL(val_int, thresh1, NULL, thresh3) AS result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_thresh_partial_null_tbl_3 """ + SELECT id, INTERVAL(val_int, thresh1, thresh2, NULL) AS result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_thresh_partial_null_tbl_4 """ + SELECT id, INTERVAL(val_int, NULL, NULL, NULL) AS result FROM ${intervalTestTable} WHERE id <= 7 ORDER BY id; @@ -146,49 +175,93 @@ suite("test_interval_function") { // Mixed thresholds: some constants, some columns qt_interval_mixed_thresholds_1 """ - SELECT id, `INTERVAL`(val_int, thresh1, 20, thresh3, 40, 50) as result + SELECT id, INTERVAL(val_int, thresh1, 20, thresh3, 40, 50) as result FROM ${intervalTestTable} WHERE id <= 7 ORDER BY id; """ qt_interval_mixed_thresholds_2 """ - SELECT id, `INTERVAL`(val_int, 10, thresh2, thresh3, 40, thresh5) as result + SELECT id, INTERVAL(val_int, 10, thresh2, thresh3, 40, thresh5) as result FROM ${intervalTestTable} WHERE id <= 7 ORDER BY id; """ qt_interval_mixed_thresholds_3 """ - SELECT id, `INTERVAL`(val_int, 10, 20, thresh3, thresh4, 50) as result + SELECT id, INTERVAL(val_int, 10, 20, thresh3, thresh4, 50) as result FROM ${intervalTestTable} WHERE id <= 7 ORDER BY id; """ qt_interval_const_value_mixed_thresholds """ - SELECT id, `INTERVAL`(25, thresh1, 20, thresh3, 40, 50) as result + SELECT id, INTERVAL(25, thresh1, 20, thresh3, 40, 50) as result FROM ${intervalTestTable} WHERE id <= 7 ORDER BY id; """ qt_interval_complex_1 """ - SELECT `INTERVAL`(100, 1, 10, 50, 100, 200, 500, 1000); + SELECT INTERVAL(100, 1, 10, 50, 100, 200, 500, 1000); """ qt_interval_complex_2 """ - SELECT `INTERVAL`(0, 1, 10, 50, 100, 200, 500, 1000); + SELECT INTERVAL(0, 1, 10, 50, 100, 200, 500, 1000); """ qt_interval_complex_3 """ - SELECT `INTERVAL`(1000, 1, 10, 50, 100, 200, 500, 1000); + SELECT INTERVAL(1000, 1, 10, 50, 100, 200, 500, 1000); """ qt_interval_complex_4 """ - SELECT `INTERVAL`(1001, 1, 10, 50, 100, 200, 500, 1000); + SELECT INTERVAL(1001, 1, 10, 50, 100, 200, 500, 1000); """ sql """ DROP TABLE IF EXISTS ${intervalTestTable}; """ + + testFoldConst("SELECT INTERVAL(23, 1, 15, 17, 30, 44, 200);") + testFoldConst("SELECT INTERVAL(10, 1, 10, 100, 1000);") + testFoldConst("SELECT INTERVAL(22, 23, 30, 44, 200);") + testFoldConst("SELECT INTERVAL(33, 1, 10, 32, 32, 102, 200);") + testFoldConst("SELECT INTERVAL(33, 1, 10, 32, 33, 102, 200);") + + testFoldConst("SELECT INTERVAL(0, 1, 10, 100);") + testFoldConst("SELECT INTERVAL(1, 1, 10, 100);") + testFoldConst("SELECT INTERVAL(100, 1, 10, 100);") + testFoldConst("SELECT INTERVAL(200, 1, 10, 100);") + testFoldConst("SELECT INTERVAL(5, 1, 10, 100);") + testFoldConst("SELECT INTERVAL(50, 1, 10, 100);") + + testFoldConst("SELECT INTERVAL(-10, -100, -50, -10, 0, 50, 100);") + testFoldConst("SELECT INTERVAL(-5, -100, -50, -10, 0, 50, 100);") + testFoldConst("SELECT INTERVAL(5, -100, -50, -10, 0, 50, 100);") + testFoldConst("SELECT INTERVAL(0, -100, -50, -10, 0, 50, 100);") + + testFoldConst("SELECT INTERVAL(10, 1, 10, 10, 20, 20, 30);") + testFoldConst("SELECT INTERVAL(15, 1, 10, 10, 20, 20, 30);") + testFoldConst("SELECT INTERVAL(25, 1, 10, 10, 20, 20, 30);") + + testFoldConst("SELECT INTERVAL(0, 10);") + testFoldConst("SELECT INTERVAL(10, 10);") + testFoldConst("SELECT INTERVAL(20, 10);") + + testFoldConst("SELECT INTERVAL(0, 10, 20);") + testFoldConst("SELECT INTERVAL(10, 10, 20);") + testFoldConst("SELECT INTERVAL(15, 10, 20);") + testFoldConst("SELECT INTERVAL(20, 10, 20);") + testFoldConst("SELECT INTERVAL(30, 10, 20);") + + testFoldConst("SELECT INTERVAL(NULL, 1, 10, 100);") + testFoldConst("SELECT INTERVAL(50, NULL, 10, 100);") + testFoldConst("SELECT INTERVAL(50, NULL, 20, 100);") + testFoldConst("SELECT INTERVAL(50, 10, NULL, 100);") + testFoldConst("SELECT INTERVAL(50, 10, 20, NULL);") + testFoldConst("SELECT INTERVAL(50, NULL, NULL, NULL);") + + testFoldConst("SELECT INTERVAL(100, 1, 10, 50, 100, 200, 500, 1000);") + testFoldConst("SELECT INTERVAL(0, 1, 10, 50, 100, 200, 500, 1000);") + testFoldConst("SELECT INTERVAL(1000, 1, 10, 50, 100, 200, 500, 1000);") + testFoldConst("SELECT INTERVAL(1001, 1, 10, 50, 100, 200, 500, 1000);") } diff --git a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy index f7b4d27e0ac5d3..34084fbdd20685 100644 --- a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy @@ -588,6 +588,7 @@ suite("test_string_function", "arrow_flight_sql") { qt_export_set_37 """SELECT id, EXPORT_SET(-7, `on`, `off`) FROM `test_export_set` ORDER BY `id`;""" qt_export_set_38 """SELECT id, EXPORT_SET(114514, '1', '0', `sep`) FROM `test_export_set` ORDER BY `id`;""" qt_export_set_39 """SELECT id, EXPORT_SET(`bits`, `on`, '0', '世界!?你好')FROM `test_export_set` ORDER BY `id`;""" + qt_export_set_40 """SELECT id, EXPORT_SET(`bits`, '1', '0', ',', 5) FROM `test_export_set` ORDER BY `id`;""" testFoldConst("SELECT EXPORT_SET(7, '1', '0');") testFoldConst("SELECT EXPORT_SET(7, '你好', '0', '?');") testFoldConst("SELECT EXPORT_SET(BIT_SHIFT_LEFT(1, 64), '1', '0');") From c3309c09463c8df94d0c3fa58a89763fbdce5ef3 Mon Sep 17 00:00:00 2001 From: linzhenqi Date: Sat, 3 Jan 2026 23:23:15 +0800 Subject: [PATCH 6/7] fix --- be/src/vec/functions/function_interval.cpp | 118 ++++++++---------- .../executable/NumericArithmetic.java | 32 +++-- 2 files changed, 69 insertions(+), 81 deletions(-) diff --git a/be/src/vec/functions/function_interval.cpp b/be/src/vec/functions/function_interval.cpp index 909fa860241bc8..2ef572c72e1c85 100644 --- a/be/src/vec/functions/function_interval.cpp +++ b/be/src/vec/functions/function_interval.cpp @@ -39,107 +39,87 @@ #include "vec/functions/function.h" #include "vec/functions/function_needs_to_handle_null.h" #include "vec/functions/simple_function_factory.h" +#include "vec/utils/util.hpp" namespace doris::vectorized { -class FunctionIntervalImpl { +class FunctionInterval : public IFunction { public: static constexpr auto name = "interval"; - static size_t get_number_of_arguments() { return 0; } - static bool is_variadic() { return true; } - static DataTypePtr get_return_type_impl(const DataTypes& /*arguments*/) { + static FunctionPtr create() { return std::make_shared(); } + String get_name() const override { return name; } + bool is_variadic() const override { return true; } + size_t get_number_of_arguments() const override { return 0; } + bool use_default_implementation_for_nulls() const override { return false; } + DataTypePtr get_return_type_impl(const DataTypes& arguments) const override { return std::make_shared(); } - static bool is_return_nullable(bool has_nullable, - const std::vector& cols_info) { - return false; - } - - static bool execute_const_null(ColumnInt32::MutablePtr& res_col, - PaddedPODArray& res_null_map_data, - size_t input_rows_count, size_t null_index) { - if (null_index == 0) { - res_col->insert_many_vals(-1, input_rows_count); - return true; - } - return false; - } + Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, + uint32_t result, size_t input_rows_count) const override { + auto res_col = ColumnInt32::create(); - static void collect_columns_info(std::vector& columns_info, - Block& block, const ColumnNumbers& arguments, - bool& has_nullable) { const size_t arg_size = arguments.size(); - std::vector arg_col(arg_size); + std::vector arg_cols(arg_size); std::vector is_const(arg_size); std::vector param_idx(arg_size - 1); bool all_const = true; - for (size_t i = 0; i < arg_size; ++i) { + for (int i = 0; i < arg_size; ++i) { is_const[i] = is_column_const(*block.get_by_position(arguments[i]).column); if (i != 0) { param_idx[i - 1] = i; all_const = all_const & is_const[i]; } } - arg_col[0] = is_const[0] ? assert_cast( - *block.get_by_position(arguments[0]).column) - .get_data_column_ptr() - : block.get_by_position(arguments[0]).column; - default_preprocess_parameter_columns(arg_col.data(), + arg_cols[0] = is_const[0] ? assert_cast( + *block.get_by_position(arguments[0]).column) + .get_data_column_ptr() + : block.get_by_position(arguments[0]).column; + default_preprocess_parameter_columns(arg_cols.data(), reinterpret_cast(is_const.data()), param_idx, block, arguments); - - // The second parameter's `is_const` is used to indicate whether all the parameters are constants. - columns_info[0].is_const = is_const[0]; - columns_info[1].is_const = all_const; - for (size_t i = 0; i < arguments.size(); ++i) { - ColumnPtr col_ptr = arg_col[i]; - columns_info[i].type = block.get_by_position(arguments[i]).type; - columns_info[i].column = col_ptr; - - if (const auto* nullable = check_and_get_column(col_ptr.get())) { - has_nullable = true; - columns_info[i].nested_col = &nullable->get_nested_column(); - columns_info[i].null_map = &nullable->get_null_map_data(); - } else { - columns_info[i].nested_col = col_ptr.get(); - } + for (int i = 1; i < arg_size; ++i) { + arg_cols[i] = remove_nullable(arg_cols[i]); } - } - static void execute(const std::vector& columns_info, - ColumnInt32::MutablePtr& res_col, PaddedPODArray& res_null_map_data, - size_t input_rows_count) { - res_col->resize(input_rows_count); - const auto& compare_data = - assert_cast(*columns_info[0].nested_col).get_data(); - const size_t num_thresholds = columns_info.size() - 1; - - for (size_t row = 0; row < input_rows_count; ++row) { - if (columns_info[0].is_null_at(row)) { - res_col->get_data()[row] = -1; + const NullMap* compare_null_map = VectorizedUtils::get_null_map(arg_cols[0]); + arg_cols[0] = remove_nullable(arg_cols[0]); + const auto& compare_data = assert_cast(*arg_cols[0]).get_data(); + for (int row = 0; row < input_rows_count; ++row) { + const size_t compare_idx = index_check_const(row, is_const[0]); + const size_t arr_idx = all_const ? 0 : row; + if (compare_null_map && (*compare_null_map)[compare_idx]) { + res_col->insert_value(-1); continue; } - // Determine whether all the threshold columns are constant columns. - size_t row_idx = columns_info[1].is_const ? 0 : row; - auto compare_val = compare_data[index_check_const(row, columns_info[0].is_const)]; - std::vector thresholds; - thresholds.reserve(num_thresholds); + res_col->insert_value(compute_interval(compare_data[compare_idx], arg_cols, is_const, + arr_idx, arg_size)); + } + block.get_by_position(result).column = std::move(res_col); + return Status::OK(); + } - for (size_t i = 1; i < columns_info.size(); ++i) { - const auto& th_col = assert_cast(*columns_info[i].nested_col); - thresholds.push_back( - columns_info[i].is_null_at(row_idx) ? 0 : th_col.get_data()[row_idx]); +private: + int32_t compute_interval(int64_t compare_val, const std::vector& arg_cols, + std::vector& is_const, size_t row_idx, + size_t arg_size) const { + size_t l = 1, r = arg_size; + while (l < r) { + size_t mid = (l + r) >> 1; + const auto mid_val = + assert_cast(*arg_cols[mid]).get_data()[row_idx]; + if (mid_val <= compare_val) { + l = mid + 1; + } else { + r = mid; } - - auto it = std::upper_bound(thresholds.begin(), thresholds.end(), compare_val); - res_col->get_data()[row] = static_cast(it - thresholds.begin()); } + return static_cast(l - 1); } }; void register_function_interval(SimpleFunctionFactory& factory) { - factory.register_function>(); + factory.register_function(); } } // namespace doris::vectorized diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/NumericArithmetic.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/NumericArithmetic.java index 4169ed6532cc68..47d1973769049a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/NumericArithmetic.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/NumericArithmetic.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.expressions.functions.executable; +import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.ExecFunction; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -45,6 +46,7 @@ import java.math.BigDecimal; import java.math.BigInteger; import java.nio.charset.StandardCharsets; +import java.util.Arrays; /** * executable functions: @@ -1121,21 +1123,27 @@ public static Expression interval(NullLiteral compareValue, Literal... threshold * interval */ @ExecFunction(name = "interval") - public static Expression interval(BigIntLiteral compareValue, BigIntLiteral... thresholds) { + public static Expression interval(BigIntLiteral compareValue, Literal... thresholds) { long value = compareValue.getValue(); - if (value < thresholds[0].getValue()) { - return new IntegerLiteral(0); - } - int l = 0; - int r = thresholds.length; - while (l < r) { - int mid = (l + r) >> 1; - if (thresholds[mid].getValue() <= value) { - l = mid + 1; + + long[] thresholdValues = new long[thresholds.length]; + for (int i = 0; i < thresholds.length; i++) { + if (thresholds[i] instanceof NullLiteral) { + thresholdValues[i] = 0; + } else if (thresholds[i] instanceof BigIntLiteral) { + thresholdValues[i] = ((BigIntLiteral) thresholds[i]).getValue(); } else { - r = mid; + throw new AnalysisException("Thresholds must be BigIntLiteral or NullLiteral"); } } - return new IntegerLiteral(l); + + int pos = Arrays.binarySearch(thresholdValues, value); + + if (pos >= 0) { + return new IntegerLiteral(pos + 1); + } else { + int insertionPoint = -(pos + 1); + return new IntegerLiteral(insertionPoint); + } } } From 530a6060d7de9f3c4dbd9be8ed55382a178822a7 Mon Sep 17 00:00:00 2001 From: linzhenqi Date: Sun, 4 Jan 2026 22:32:45 +0800 Subject: [PATCH 7/7] revert FunctionNeedsToHandleNull --- .../functions/function_needs_to_handle_null.h | 27 +++++-------------- 1 file changed, 7 insertions(+), 20 deletions(-) diff --git a/be/src/vec/functions/function_needs_to_handle_null.h b/be/src/vec/functions/function_needs_to_handle_null.h index 6fc3f60528b548..0e60da92e68981 100644 --- a/be/src/vec/functions/function_needs_to_handle_null.h +++ b/be/src/vec/functions/function_needs_to_handle_null.h @@ -17,7 +17,6 @@ #pragma once #include -#include "udf/udf.h" #include "vec/functions/function.h" namespace doris::vectorized { @@ -25,9 +24,7 @@ namespace doris::vectorized { // Helper struct to store information about const+nullable columns struct ColumnWithConstAndNullMap { - ColumnPtr column = nullptr; const IColumn* nested_col = nullptr; - DataTypePtr type = nullptr; const NullMap* null_map = nullptr; bool is_const = false; @@ -83,12 +80,8 @@ class FunctionNeedsToHandleNull : public IFunction { if (columns_info[i].is_const && columns_info[i].null_map && (*columns_info[i].null_map)[0] && execute_const_null(res_col, null_map_data, input_rows_count, i)) { - if (is_return_nullable(has_nullable, columns_info)) { - block.replace_by_position(result, ColumnNullable::create(std::move(res_col), - std::move(null_map))); - } else { - block.replace_by_position(result, std::move(res_col)); - } + block.replace_by_position( + result, ColumnNullable::create(std::move(res_col), std::move(null_map))); return Status::OK(); } } @@ -128,23 +121,17 @@ class FunctionNeedsToHandleNull : public IFunction { // Collect the required information for each column into columns_info // Including whether it is a constant column, nested column and null map(if exists). - void collect_columns_info(std::vector& columns_info, Block& block, - const ColumnNumbers& arguments, bool& has_nullable) const { - if constexpr (requires { - Impl::collect_columns_info(columns_info, block, arguments, has_nullable); - }) { - Impl::collect_columns_info(columns_info, block, arguments, has_nullable); - return; - } + void collect_columns_info(std::vector& columns_info, + const Block& block, const ColumnNumbers& arguments, + bool& has_nullable) const { for (size_t i = 0; i < arguments.size(); ++i) { ColumnPtr col_ptr; const auto& col_with_type = block.get_by_position(arguments[i]); std::tie(col_ptr, columns_info[i].is_const) = unpack_if_const(col_with_type.column); - columns_info[i].type = col_with_type.type; - columns_info[i].column = col_ptr; - if (const auto* nullable = check_and_get_column(col_ptr.get())) { + if (is_column_nullable(*col_ptr)) { has_nullable = true; + const auto* nullable = check_and_get_column(col_ptr.get()); columns_info[i].nested_col = &nullable->get_nested_column(); columns_info[i].null_map = &nullable->get_null_map_data(); } else {