From 6dc5b65e1cf0a26d906cad6e4bd1d2fb0f57561e Mon Sep 17 00:00:00 2001 From: jianhao <1367919489@qq.com> Date: Fri, 31 Oct 2025 14:31:16 +0800 Subject: [PATCH 1/4] 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 94b3714ee9c9b9..6a6e5bae91ee85 100644 --- a/be/src/vec/functions/simple_function_factory.h +++ b/be/src/vec/functions/simple_function_factory.h @@ -102,6 +102,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); @@ -327,6 +328,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 c3504c5918d0b8..6296c561af6b72 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 @@ -243,6 +243,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; @@ -771,6 +772,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"), @@ -1049,10 +1051,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(Uniform.class, "uniform"), scalar(UrlDecode.class, "url_decode"), scalar(UrlEncode.class, "url_encode"), 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 db50d98e0623a4..ee1d7f11bf085d 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 @@ -249,6 +249,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; @@ -1429,6 +1430,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 ce0d67620a5ffad18e7eea27de11309417eaaa2d Mon Sep 17 00:00:00 2001 From: jianhao <1367919489@qq.com> Date: Fri, 31 Oct 2025 14:53:15 +0800 Subject: [PATCH 2/4] 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 b6f90eea7444d00054efc2633b19434080900b4f Mon Sep 17 00:00:00 2001 From: jianhao <1367919489@qq.com> Date: Mon, 3 Nov 2025 10:49:39 +0800 Subject: [PATCH 3/4] 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 5e59c1d3d8356994f2265894f4649b2e3e651ae7 Mon Sep 17 00:00:00 2001 From: jianhao <1367919489@qq.com> Date: Mon, 3 Nov 2025 10:55:52 +0800 Subject: [PATCH 4/4] 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 6296c561af6b72..41dc26f8d35b35 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 @@ -1051,10 +1051,10 @@ 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(Uncompress.class, "uncompress"), scalar(Uniform.class, "uniform"), scalar(UrlDecode.class, "url_decode"), scalar(UrlEncode.class, "url_encode"),