This is an automated email from the ASF dual-hosted git repository.
yiguolei pushed a commit to branch branch-4.0
in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/branch-4.0 by this push:
new 3f651d0ac41 branch-4.0: [Feature](func) Support INTERVAL function and
fix EXPORT_SET constant process #58885 (#59655)
3f651d0ac41 is described below
commit 3f651d0ac41318600e728743ebde7e3fcec588df
Author: github-actions[bot]
<41898282+github-actions[bot]@users.noreply.github.com>
AuthorDate: Thu Jan 8 14:11:41 2026 +0800
branch-4.0: [Feature](func) Support INTERVAL function and fix EXPORT_SET
constant process #58885 (#59655)
Cherry-picked from #58885
Co-authored-by: linrrarity <[email protected]>
Co-authored-by: jianhao <[email protected]>
---
be/src/vec/columns/column_const.cpp | 8 +
be/src/vec/columns/column_const.h | 4 +
be/src/vec/functions/function_interval.cpp | 125 ++++++++++
be/src/vec/functions/function_string.h | 2 +-
be/src/vec/functions/simple_function_factory.h | 2 +
.../antlr4/org/apache/doris/nereids/DorisParser.g4 | 1 +
.../doris/catalog/BuiltinScalarFunctions.java | 4 +-
.../functions/executable/NumericArithmetic.java | 38 +++
.../expressions/functions/scalar/Interval.java | 61 +++++
.../expressions/visitor/ScalarFunctionVisitor.java | 5 +
.../data/function_p0/test_interval_function.out | 244 +++++++++++++++++++
.../string_functions/test_string_function.out | 15 ++
.../function_p0/test_interval_function.groovy | 267 +++++++++++++++++++++
.../string_functions/test_string_function.groovy | 1 +
14 files changed, 775 insertions(+), 2 deletions(-)
diff --git a/be/src/vec/columns/column_const.cpp
b/be/src/vec/columns/column_const.cpp
index 6f7dd517233..16f80f5baa2 100644
--- a/be/src/vec/columns/column_const.cpp
+++ b/be/src/vec/columns/column_const.cpp
@@ -136,6 +136,14 @@ std::pair<const ColumnPtr&, bool> unpack_if_const(const
ColumnPtr& ptr) noexcept
void default_preprocess_parameter_columns(ColumnPtr* columns, const bool*
col_const,
const std::initializer_list<size_t>&
parameters,
Block& block, const ColumnNumbers&
arg_indexes) {
+ default_preprocess_parameter_columns(
+ columns, col_const, std::span<const size_t>(parameters.begin(),
parameters.end()),
+ block, arg_indexes);
+}
+
+void default_preprocess_parameter_columns(ColumnPtr* columns, const bool*
col_const,
+ const std::span<const size_t>&
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 58e6786c776..6432b9d913c 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<size_t>&
parameters,
Block& block, const ColumnNumbers&
arg_indexes);
+void default_preprocess_parameter_columns(ColumnPtr* columns, const bool*
col_const,
+ const std::span<const size_t>&
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
new file mode 100644
index 00000000000..2ef572c72e1
--- /dev/null
+++ b/be/src/vec/functions/function_interval.cpp
@@ -0,0 +1,125 @@
+// 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 <algorithm>
+#include <cstddef>
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <vector>
+
+#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"
+#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/function_needs_to_handle_null.h"
+#include "vec/functions/simple_function_factory.h"
+#include "vec/utils/util.hpp"
+
+namespace doris::vectorized {
+class FunctionInterval : public IFunction {
+public:
+ static constexpr auto name = "interval";
+ static FunctionPtr create() { return std::make_shared<FunctionInterval>();
}
+ 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<DataTypeInt32>();
+ }
+
+ 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();
+
+ const size_t arg_size = arguments.size();
+ std::vector<ColumnPtr> arg_cols(arg_size);
+ std::vector<uint8_t> is_const(arg_size);
+ std::vector<size_t> param_idx(arg_size - 1);
+ bool all_const = true;
+ 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_cols[0] = is_const[0] ? assert_cast<const ColumnConst&>(
+
*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<const
bool*>(is_const.data()),
+ param_idx, block, arguments);
+ for (int i = 1; i < arg_size; ++i) {
+ arg_cols[i] = remove_nullable(arg_cols[i]);
+ }
+
+ 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<const
ColumnInt64&>(*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;
+ }
+
+ 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();
+ }
+
+private:
+ int32_t compute_interval(int64_t compare_val, const
std::vector<ColumnPtr>& arg_cols,
+ std::vector<uint8_t>& 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<const
ColumnInt64&>(*arg_cols[mid]).get_data()[row_idx];
+ if (mid_val <= compare_val) {
+ l = mid + 1;
+ } else {
+ r = mid;
+ }
+ }
+ return static_cast<int32_t>(l - 1);
+ }
+};
+
+void register_function_interval(SimpleFunctionFactory& factory) {
+ factory.register_function<FunctionInterval>();
+}
+
+} // namespace doris::vectorized
diff --git a/be/src/vec/functions/function_string.h
b/be/src/vec/functions/function_string.h
index 354b603f5b0..e8c2afac627 100644
--- a/be/src/vec/functions/function_string.h
+++ b/be/src/vec/functions/function_string.h
@@ -5091,7 +5091,7 @@ public:
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/be/src/vec/functions/simple_function_factory.h
b/be/src/vec/functions/simple_function_factory.h
index a0297020ddc..b52ed3ac158 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);
@@ -335,6 +336,7 @@ public:
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/antlr4/org/apache/doris/nereids/DorisParser.g4
b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
index 293599c648a..3ce42a3a48d 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
@@ -1703,6 +1703,7 @@ functionNameIdentifier
| CURRENT_USER
| DATABASE
| IF
+ | INTERVAL
| LEFT
| LIKE
| PASSWORD
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 92cabf7f59d..f06ef3e7752 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.Initcap;
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;
@@ -784,6 +785,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"),
@@ -1067,8 +1069,8 @@ public class BuiltinScalarFunctions implements
FunctionHelper {
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(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/executable/NumericArithmetic.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/NumericArithmetic.java
index 3aeb33ff094..47d19737690 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 org.apache.commons.math3.util.FastMath;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
/**
* executable functions:
@@ -1108,4 +1110,40 @@ public class NumericArithmetic {
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, Literal...
thresholds) {
+ long value = compareValue.getValue();
+
+ 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 {
+ throw new AnalysisException("Thresholds must be BigIntLiteral
or NullLiteral");
+ }
+ }
+
+ int pos = Arrays.binarySearch(thresholdValues, value);
+
+ if (pos >= 0) {
+ return new IntegerLiteral(pos + 1);
+ } else {
+ int insertionPoint = -(pos + 1);
+ return new IntegerLiteral(insertionPoint);
+ }
+ }
}
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 00000000000..78668cb0cb2
--- /dev/null
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Interval.java
@@ -0,0 +1,61 @@
+// 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.AlwaysNotNullable;
+import
org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+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.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, AlwaysNotNullable {
+
+ public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+
FunctionSignature.ret(IntegerType.INSTANCE).varArgs(BigIntType.INSTANCE,
BigIntType.INSTANCE));
+
+ public Interval(Expression arg, Expression... varArgs) {
+ super("interval", ExpressionUtils.mergeArguments(arg, varArgs));
+ }
+
+ @Override
+ public Interval withChildren(List<Expression> children) {
+ Preconditions.checkArgument(children.size() >= 2);
+ return new Interval(children.get(0), children.subList(1,
children.size()).toArray(new Expression[0]));
+ }
+
+ @Override
+ public List<FunctionSignature> getSignatures() {
+ return SIGNATURES;
+ }
+
+ @Override
+ public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+ return visitor.visitInterval(this, context);
+ }
+}
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 07ee2b87045..7f70a6bf586 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.Initcap;
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;
@@ -1458,6 +1459,10 @@ public interface ScalarFunctionVisitor<R, C> {
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 00000000000..f1c4c66f938
--- /dev/null
+++ b/regression-test/data/function_p0/test_interval_function.out
@@ -0,0 +1,244 @@
+-- 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_null_first_arg --
+-1
+
+-- !interval_null_threshold --
+2
+
+-- !interval_thresh_partial_null_1 --
+2
+
+-- !interval_thresh_partial_null_2 --
+2
+
+-- !interval_thresh_partial_null_3 --
+3
+
+-- !interval_thresh_partial_null_4 --
+3
+
+-- !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_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
+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
+
+-- !interval_complex_2 --
+0
+
+-- !interval_complex_3 --
+7
+
+-- !interval_complex_4 --
+7
+
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 64cef848f05..e9189ab000c 100644
---
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
@@ -1161,6 +1161,21 @@ x2,x4,x26,x51,x62,x63,x64
12
1世界!?你好0世界!?你好1世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0
13
1世界!?你好0世界!?你好1世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0世界!?你好0
+-- !export_set_40 --
+1 1,1,1,1,1
+2 0,1,1,1,1
+3 1,0,1,0,0
+4 1,0,1,0,0
+5 1,0,1,0,0
+6 0,1,1,0,0
+7 1,1,0,1,1
+8 1,0,0,1,0
+9 \N
+10 1,0,1,0,0
+11 1,0,1,0,0
+12 1,0,1,0,0
+13 1,0,1,0,0
+
-- !insert_1 --
1 QuWhattic
2 Quadratic
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 00000000000..b3aff8cb7d7
--- /dev/null
+++ b/regression-test/suites/function_p0/test_interval_function.groovy
@@ -0,0 +1,267 @@
+// 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_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"
+
+ 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;
+ """
+
+ // 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;
+ """
+
+ // 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);
+ """
+
+ 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}; """
+
+ 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 f7b4d27e0ac..34084fbdd20 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');")
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]