You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2023/01/30 01:44:15 UTC

[doris] branch branch-1.2-lts updated: [cherry-pick](branch-1.2) pick 3 decimalv3/datev2 commits from master (#16181)

This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/branch-1.2-lts by this push:
     new c08b1a91f6 [cherry-pick](branch-1.2) pick 3 decimalv3/datev2 commits from master (#16181)
c08b1a91f6 is described below

commit c08b1a91f65da9669f14973c0ab41225b22a2b01
Author: Gabriel <ga...@gmail.com>
AuthorDate: Mon Jan 30 09:44:09 2023 +0800

    [cherry-pick](branch-1.2) pick 3 decimalv3/datev2 commits from master (#16181)
    
    pick #16152 #16102 #16132
---
 be/src/exprs/bloomfilter_predicate.h               | 19 +++--
 be/src/exprs/minmax_predicate.h                    |  5 +-
 be/src/runtime/datetime_value.h                    |  2 +-
 be/src/vec/runtime/vdatetime_value.cpp             |  2 +-
 be/src/vec/runtime/vdatetime_value.h               |  2 +-
 be/src/vec/sink/vmysql_result_writer.cpp           | 28 ++++----
 be/src/vec/sink/vmysql_result_writer.h             |  2 +-
 .../org/apache/doris/analysis/DateLiteral.java     |  3 +-
 .../apache/doris/analysis/FunctionCallExpr.java    | 53 +++++++++++++-
 .../doris/rewrite/RewriteDateLiteralRuleTest.java  |  4 +-
 .../datatype_p0/date/test_date_runtime_filter.out  | 26 +++++++
 .../data/datatype_p0/decimalv3/test_functions.out  |  5 ++
 .../array_functions/test_array_functions.out       |  8 +--
 .../array_functions/test_array_with_scale_type.out | 80 ++++++++++++++++++++++
 .../cast_function/test_cast_with_scale_type.out    | 16 +++++
 .../date/test_date_runtime_filter.groovy           | 49 +++++++++++++
 .../datatype_p0/decimalv3/test_functions.groovy    | 53 ++++++++++++++
 .../array_functions/test_array_functions.groovy    |  5 +-
 .../test_array_with_scale_type.groovy              | 72 +++++++++++++++++++
 .../cast_function/test_cast_with_scale_type.groovy | 49 +++++++++++++
 20 files changed, 448 insertions(+), 35 deletions(-)

diff --git a/be/src/exprs/bloomfilter_predicate.h b/be/src/exprs/bloomfilter_predicate.h
index ec3b24434e..b6db148fe4 100644
--- a/be/src/exprs/bloomfilter_predicate.h
+++ b/be/src/exprs/bloomfilter_predicate.h
@@ -330,9 +330,13 @@ struct FixedStringFindOp : public StringFindOp {
 
 struct DateTimeFindOp : public CommonFindOp<DateTimeValue> {
     bool find_olap_engine(const BloomFilterAdaptor& bloom_filter, const void* data) const {
-        DateTimeValue value;
+        vectorized::VecDateTimeValue value;
         value.from_olap_datetime(*reinterpret_cast<const uint64_t*>(data));
-        return bloom_filter.test(Slice((char*)&value, sizeof(DateTimeValue)));
+        return bloom_filter.test(Slice((char*)&value, sizeof(vectorized::VecDateTimeValue)));
+    }
+
+    void insert(BloomFilterAdaptor& bloom_filter, const void* data) const {
+        bloom_filter.add_bytes((char*)data, sizeof(vectorized::VecDateTimeValue));
     }
 };
 
@@ -344,13 +348,14 @@ struct DateFindOp : public CommonFindOp<DateTimeValue> {
         uint24_t date = *static_cast<const uint24_t*>(data);
         uint64_t value = uint32_t(date);
 
-        DateTimeValue date_value;
+        vectorized::VecDateTimeValue date_value;
         date_value.from_olap_date(value);
-        date_value.to_datetime();
 
-        char data_bytes[sizeof(date_value)];
-        memcpy(&data_bytes, &date_value, sizeof(date_value));
-        return bloom_filter.test(Slice(data_bytes, sizeof(DateTimeValue)));
+        return bloom_filter.test(Slice((char*)&date_value, sizeof(vectorized::VecDateTimeValue)));
+    }
+
+    void insert(BloomFilterAdaptor& bloom_filter, const void* data) const {
+        bloom_filter.add_bytes((char*)data, sizeof(vectorized::VecDateTimeValue));
     }
 };
 
diff --git a/be/src/exprs/minmax_predicate.h b/be/src/exprs/minmax_predicate.h
index a237f7f8b2..58da808339 100644
--- a/be/src/exprs/minmax_predicate.h
+++ b/be/src/exprs/minmax_predicate.h
@@ -49,7 +49,10 @@ public:
         }
 
         T val_data;
-        if constexpr (sizeof(T) >= sizeof(int128_t)) {
+        if constexpr (std::is_same_v<T, DateTimeValue>) {
+            reinterpret_cast<const vectorized::VecDateTimeValue*>(data)->convert_vec_dt_to_dt(
+                    &val_data);
+        } else if constexpr (sizeof(T) >= sizeof(int128_t)) {
             // use dereference operator on unalign address maybe lead segmentation fault
             memcpy(&val_data, data, sizeof(T));
         } else {
diff --git a/be/src/runtime/datetime_value.h b/be/src/runtime/datetime_value.h
index dec991ece2..c9a046019c 100644
--- a/be/src/runtime/datetime_value.h
+++ b/be/src/runtime/datetime_value.h
@@ -614,7 +614,7 @@ public:
 private:
     // Used to make sure sizeof DateTimeValue
     friend class UnusedClass;
-    friend void doris::vectorized::VecDateTimeValue::convert_vec_dt_to_dt(DateTimeValue* dt);
+    friend void doris::vectorized::VecDateTimeValue::convert_vec_dt_to_dt(DateTimeValue* dt) const;
     friend void doris::vectorized::VecDateTimeValue::convert_dt_to_vec_dt(DateTimeValue* dt);
 
     void from_packed_time(int64_t packed_time) {
diff --git a/be/src/vec/runtime/vdatetime_value.cpp b/be/src/vec/runtime/vdatetime_value.cpp
index 8111569c6b..69eae65d99 100644
--- a/be/src/vec/runtime/vdatetime_value.cpp
+++ b/be/src/vec/runtime/vdatetime_value.cpp
@@ -1749,7 +1749,7 @@ void VecDateTimeValue::create_from_date_v2(DateV2Value<T>& value, TimeType type)
 }
 
 void VecDateTimeValue::convert_vec_dt_to_dt(
-        doris::DateTimeValue* dt) { //use convert VecDateTimeValue to DateTimeValue
+        doris::DateTimeValue* dt) const { //use convert VecDateTimeValue to DateTimeValue
     dt->_neg = this->_neg;
     dt->_type = this->_type;
     dt->_hour = this->_hour;
diff --git a/be/src/vec/runtime/vdatetime_value.h b/be/src/vec/runtime/vdatetime_value.h
index 7abdeb0064..100bbe91ff 100644
--- a/be/src/vec/runtime/vdatetime_value.h
+++ b/be/src/vec/runtime/vdatetime_value.h
@@ -629,7 +629,7 @@ public:
                _day > 0;
     }
 
-    void convert_vec_dt_to_dt(doris::DateTimeValue* dt);
+    void convert_vec_dt_to_dt(doris::DateTimeValue* dt) const;
     void convert_dt_to_vec_dt(doris::DateTimeValue* dt);
     int64_t to_datetime_int64() const;
 
diff --git a/be/src/vec/sink/vmysql_result_writer.cpp b/be/src/vec/sink/vmysql_result_writer.cpp
index d563e0a0a0..aa662635d9 100644
--- a/be/src/vec/sink/vmysql_result_writer.cpp
+++ b/be/src/vec/sink/vmysql_result_writer.cpp
@@ -174,10 +174,10 @@ Status VMysqlResultWriter::_add_one_column(const ColumnPtr& column_ptr,
                 } else {
                     if (WhichDataType(remove_nullable(nested_type_ptr)).is_string()) {
                         buf_ret = _buffer.push_string("'", 1);
-                        buf_ret = _add_one_cell(data, j, nested_type_ptr, _buffer);
+                        buf_ret = _add_one_cell(data, j, nested_type_ptr, _buffer, scale);
                         buf_ret = _buffer.push_string("'", 1);
                     } else {
-                        buf_ret = _add_one_cell(data, j, nested_type_ptr, _buffer);
+                        buf_ret = _add_one_cell(data, j, nested_type_ptr, _buffer, scale);
                     }
                 }
                 begin = false;
@@ -255,8 +255,7 @@ Status VMysqlResultWriter::_add_one_column(const ColumnPtr& column_ptr,
             if constexpr (type == TYPE_DATETIME) {
                 char buf[64];
                 auto time_num = data[i];
-                VecDateTimeValue time_val;
-                memcpy(static_cast<void*>(&time_val), &time_num, sizeof(Int64));
+                VecDateTimeValue time_val = binary_cast<Int64, VecDateTimeValue>(time_num);
                 // TODO(zhaochun), this function has core risk
                 char* pos = time_val.to_string(buf);
                 buf_ret = _buffer.push_string(buf, pos - buf - 1);
@@ -264,16 +263,16 @@ Status VMysqlResultWriter::_add_one_column(const ColumnPtr& column_ptr,
             if constexpr (type == TYPE_DATEV2) {
                 char buf[64];
                 auto time_num = data[i];
-                doris::vectorized::DateV2Value<DateV2ValueType> date_val;
-                memcpy(static_cast<void*>(&date_val), &time_num, sizeof(UInt32));
+                DateV2Value<DateV2ValueType> date_val =
+                        binary_cast<UInt32, DateV2Value<DateV2ValueType>>(time_num);
                 char* pos = date_val.to_string(buf);
                 buf_ret = _buffer.push_string(buf, pos - buf - 1);
             }
             if constexpr (type == TYPE_DATETIMEV2) {
-                char buf[64];
                 auto time_num = data[i];
-                doris::vectorized::DateV2Value<DateTimeV2ValueType> date_val;
-                memcpy(static_cast<void*>(&date_val), &time_num, sizeof(UInt64));
+                char buf[64];
+                DateV2Value<DateTimeV2ValueType> date_val =
+                        binary_cast<UInt64, DateV2Value<DateTimeV2ValueType>>(time_num);
                 char* pos = date_val.to_string(buf, scale);
                 buf_ret = _buffer.push_string(buf, pos - buf - 1);
             }
@@ -294,7 +293,7 @@ Status VMysqlResultWriter::_add_one_column(const ColumnPtr& column_ptr,
 }
 
 int VMysqlResultWriter::_add_one_cell(const ColumnPtr& column_ptr, size_t row_idx,
-                                      const DataTypePtr& type, MysqlRowBuffer& buffer) {
+                                      const DataTypePtr& type, MysqlRowBuffer& buffer, int scale) {
     WhichDataType which(type->get_type_id());
     if (which.is_nullable() && column_ptr->is_null_at(row_idx)) {
         return buffer.push_null();
@@ -372,7 +371,7 @@ int VMysqlResultWriter::_add_one_cell(const ColumnPtr& column_ptr, size_t row_id
         DateV2Value<DateTimeV2ValueType> datetimev2 =
                 binary_cast<UInt64, DateV2Value<DateTimeV2ValueType>>(value);
         char buf[64];
-        char* pos = datetimev2.to_string(buf);
+        char* pos = datetimev2.to_string(buf, scale);
         return buffer.push_string(buf, pos - buf - 1);
     } else if (which.is_decimal32()) {
         DataTypePtr nested_type = type;
@@ -653,16 +652,19 @@ Status VMysqlResultWriter::append_block(Block& input_block) {
             break;
         }
         case TYPE_ARRAY: {
+            // Currently all functions only support single-level nested arrays,
+            // so we use Array's child scale to represent the scale of nested type.
+            scale = _output_vexpr_ctxs[i]->root()->type().children[0].scale;
             if (type_ptr->is_nullable()) {
                 auto& nested_type =
                         assert_cast<const DataTypeNullable&>(*type_ptr).get_nested_type();
                 auto& sub_type = assert_cast<const DataTypeArray&>(*nested_type).get_nested_type();
                 status = _add_one_column<PrimitiveType::TYPE_ARRAY, true>(column_ptr, result,
-                                                                          sub_type);
+                                                                          sub_type, scale);
             } else {
                 auto& sub_type = assert_cast<const DataTypeArray&>(*type_ptr).get_nested_type();
                 status = _add_one_column<PrimitiveType::TYPE_ARRAY, false>(column_ptr, result,
-                                                                           sub_type);
+                                                                           sub_type, scale);
             }
             break;
         }
diff --git a/be/src/vec/sink/vmysql_result_writer.h b/be/src/vec/sink/vmysql_result_writer.h
index 7b8df80306..0d6dacc1ad 100644
--- a/be/src/vec/sink/vmysql_result_writer.h
+++ b/be/src/vec/sink/vmysql_result_writer.h
@@ -52,7 +52,7 @@ private:
     Status _add_one_column(const ColumnPtr& column_ptr, std::unique_ptr<TFetchDataResult>& result,
                            const DataTypePtr& nested_type_ptr = nullptr, int scale = -1);
     int _add_one_cell(const ColumnPtr& column_ptr, size_t row_idx, const DataTypePtr& type,
-                      MysqlRowBuffer& buffer);
+                      MysqlRowBuffer& buffer, int scale = -1);
 
 private:
     BufferControlBlock* _sinker;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java
index 82e80a766f..a87811bcfe 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java
@@ -546,6 +546,7 @@ public class DateLiteral extends LiteralExpr {
         if (type.isDate() || type.isDateV2()) {
             return String.format("%04d-%02d-%02d", year, month, day);
         } else if (type.isDatetimeV2()) {
+            int scale = ((ScalarType) type).getScalarScale();
             long ms = Double.valueOf(microsecond / (int) (Math.pow(10, 6 - ((ScalarType) type).getScalarScale()))
                     * (Math.pow(10, 6 - ((ScalarType) type).getScalarScale()))).longValue();
             String tmp = String.format("%04d-%02d-%02d %02d:%02d:%02d",
@@ -553,7 +554,7 @@ public class DateLiteral extends LiteralExpr {
             if (ms == 0) {
                 return tmp;
             }
-            return tmp + String.format(".%06d", ms);
+            return tmp + String.format(".%06d", ms).substring(0, scale + 1);
         } else {
             return String.format("%04d-%02d-%02d %02d:%02d:%02d", year, month, day, hour, minute, second);
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java
index 6ee34851ac..8a312dba50 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java
@@ -92,6 +92,9 @@ public class FunctionCallExpr extends Expr {
             } else if (children != null && children.size() > 0 && children.get(0).getType().isDatetimeV2()
                     && returnType.isDatetimeV2()) {
                 return children.get(0).getType();
+            } else if (children != null && children.size() > 0 && children.get(0).getType().isDecimalV2()
+                    && returnType.isDecimalV2()) {
+                return children.get(0).getType();
             } else {
                 return returnType;
             }
@@ -147,6 +150,30 @@ public class FunctionCallExpr extends Expr {
             }
         });
 
+        PRECISION_INFER_RULE.put("array_min", (children, returnType) -> {
+            Preconditions.checkArgument(children != null && children.size() > 0);
+            if (children.get(0).getType().isArrayType() && (
+                    ((ArrayType) children.get(0).getType()).getItemType().isDecimalV3() || ((ArrayType) children.get(0)
+                            .getType()).getItemType().isDecimalV2() || ((ArrayType) children.get(0)
+                            .getType()).getItemType().isDatetimeV2())) {
+                return ((ArrayType) children.get(0).getType()).getItemType();
+            } else {
+                return returnType;
+            }
+        });
+
+        PRECISION_INFER_RULE.put("array_max", (children, returnType) -> {
+            Preconditions.checkArgument(children != null && children.size() > 0);
+            if (children.get(0).getType().isArrayType() && (
+                    ((ArrayType) children.get(0).getType()).getItemType().isDecimalV3() || ((ArrayType) children.get(0)
+                            .getType()).getItemType().isDecimalV2() || ((ArrayType) children.get(0)
+                            .getType()).getItemType().isDatetimeV2())) {
+                return ((ArrayType) children.get(0).getType()).getItemType();
+            } else {
+                return returnType;
+            }
+        });
+
         PRECISION_INFER_RULE.put("round", roundRule);
         PRECISION_INFER_RULE.put("ceil", roundRule);
         PRECISION_INFER_RULE.put("floor", roundRule);
@@ -1174,6 +1201,14 @@ public class FunctionCallExpr extends Expr {
         } else if (fnName.getFunction().equalsIgnoreCase("if")) {
             Type[] childTypes = collectChildReturnTypes();
             Type assignmentCompatibleType = ScalarType.getAssignmentCompatibleType(childTypes[1], childTypes[2], true);
+            if (assignmentCompatibleType.isDecimalV3()) {
+                if (childTypes[1].isDecimalV3() && !((ScalarType) childTypes[1]).equals(assignmentCompatibleType)) {
+                    uncheckedCastChild(assignmentCompatibleType, 1);
+                }
+                if (childTypes[2].isDecimalV3() && !((ScalarType) childTypes[2]).equals(assignmentCompatibleType)) {
+                    uncheckedCastChild(assignmentCompatibleType, 2);
+                }
+            }
             childTypes[1] = assignmentCompatibleType;
             childTypes[2] = assignmentCompatibleType;
             fn = getBuiltinFunction(fnName.getFunction(), childTypes,
@@ -1181,6 +1216,8 @@ public class FunctionCallExpr extends Expr {
             if (assignmentCompatibleType.isDatetimeV2()) {
                 fn.setReturnType(assignmentCompatibleType);
             }
+
+
         } else if (AggregateFunction.SUPPORT_ORDER_BY_AGGREGATE_FUNCTION_NAME_SET.contains(
                 fnName.getFunction().toLowerCase())) {
             // order by elements add as child like windows function. so if we get the
@@ -1317,6 +1354,20 @@ public class FunctionCallExpr extends Expr {
                     } else if (fnName.getFunction().equalsIgnoreCase("money_format")
                             && children.get(0).getType().isDecimalV3() && args[ix].isDecimalV3()) {
                         continue;
+                    } else if (fnName.getFunction().equalsIgnoreCase("array")
+                            && (children.get(0).getType().isDecimalV3() && args[ix].isDecimalV3()
+                            || children.get(0).getType().isDatetimeV2() && args[ix].isDatetimeV2())) {
+                        continue;
+                    } else if ((fnName.getFunction().equalsIgnoreCase("array_min") || fnName.getFunction()
+                            .equalsIgnoreCase("array_max"))
+                            && ((
+                            children.get(0).getType().isDecimalV3() && ((ArrayType) args[ix]).getItemType()
+                                    .isDecimalV3())
+                            || (children.get(0).getType().isDatetimeV2()
+                            && ((ArrayType) args[ix]).getItemType().isDatetimeV2())
+                            || (children.get(0).getType().isDecimalV2()
+                            && ((ArrayType) args[ix]).getItemType().isDecimalV2()))) {
+                        continue;
                     } else if (!argTypes[i].matchesType(args[ix]) && !(
                             argTypes[i].isDateOrDateTime() && args[ix].isDateOrDateTime())
                             && (!fn.getReturnType().isDecimalV3()
@@ -1424,7 +1475,7 @@ public class FunctionCallExpr extends Expr {
     // if return type is nested type, need to be determined the sub-element type
     private void analyzeNestedFunction() {
         // array
-        if ("array".equalsIgnoreCase(fnName.getFunction())) {
+        if (fnName.getFunction().equalsIgnoreCase("array")) {
             if (children.size() > 0) {
                 this.type = new ArrayType(children.get(0).getType());
             }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/rewrite/RewriteDateLiteralRuleTest.java b/fe/fe-core/src/test/java/org/apache/doris/rewrite/RewriteDateLiteralRuleTest.java
index 40bc9bb2c3..55a875fd2d 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/rewrite/RewriteDateLiteralRuleTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/rewrite/RewriteDateLiteralRuleTest.java
@@ -122,7 +122,7 @@ public class RewriteDateLiteralRuleTest {
     public void testWithStringFormatDateV2() throws Exception {
         String query = "select * from " + DB_NAME + ".tb2 where k1 > '2021030112334455'";
         String planString = dorisAssert.query(query).explainQuery();
-        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 12:33:44.550000'"));
+        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 12:33:44.550'"));
 
         query = "select k1 > '20210301' from " + DB_NAME + ".tb2";
         planString = dorisAssert.query(query).explainQuery();
@@ -130,7 +130,7 @@ public class RewriteDateLiteralRuleTest {
 
         query = "select k1 > '20210301233234.34' from " + DB_NAME + ".tb2";
         planString = dorisAssert.query(query).explainQuery();
-        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 23:32:34.340000'"));
+        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 23:32:34.340'"));
 
         query = "select * from " + DB_NAME + ".tb2 where k1 > '2021-03-01'";
         planString = dorisAssert.query(query).explainQuery();
diff --git a/regression-test/data/datatype_p0/date/test_date_runtime_filter.out b/regression-test/data/datatype_p0/date/test_date_runtime_filter.out
new file mode 100644
index 0000000000..a6051c0236
--- /dev/null
+++ b/regression-test/data/datatype_p0/date/test_date_runtime_filter.out
@@ -0,0 +1,26 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !sql1 --
+1	2000-01-01	2000-01-01T11:11:11
+2	2000-02-02	2000-02-02T11:11:11
+3	2000-03-02	2000-03-02T11:11:11
+
+-- !sql2 --
+1	2000-01-01	2000-01-01T11:11:11	1	2000-01-01	2000-01-01T11:11:11
+2	2000-02-02	2000-02-02T11:11:11	2	2000-02-02	2000-02-02T11:11:11
+3	2000-03-02	2000-03-02T11:11:11	3	2000-03-02	2000-03-02T11:11:11
+
+-- !sql2 --
+1	2000-01-01	2000-01-01T11:11:11	1	2000-01-01	2000-01-01T11:11:11
+2	2000-02-02	2000-02-02T11:11:11	2	2000-02-02	2000-02-02T11:11:11
+3	2000-03-02	2000-03-02T11:11:11	3	2000-03-02	2000-03-02T11:11:11
+
+-- !sql2 --
+1	2000-01-01	2000-01-01T11:11:11	1	2000-01-01	2000-01-01T11:11:11
+2	2000-02-02	2000-02-02T11:11:11	2	2000-02-02	2000-02-02T11:11:11
+3	2000-03-02	2000-03-02T11:11:11	3	2000-03-02	2000-03-02T11:11:11
+
+-- !sql2 --
+1	2000-01-01	2000-01-01T11:11:11	1	2000-01-01	2000-01-01T11:11:11
+2	2000-02-02	2000-02-02T11:11:11	2	2000-02-02	2000-02-02T11:11:11
+3	2000-03-02	2000-03-02T11:11:11	3	2000-03-02	2000-03-02T11:11:11
+
diff --git a/regression-test/data/datatype_p0/decimalv3/test_functions.out b/regression-test/data/datatype_p0/decimalv3/test_functions.out
new file mode 100644
index 0000000000..6651ecd9af
--- /dev/null
+++ b/regression-test/data/datatype_p0/decimalv3/test_functions.out
@@ -0,0 +1,5 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select_default --
+10001	13.300000000
+10007	13.300000000
+
diff --git a/regression-test/data/query_p0/sql_functions/array_functions/test_array_functions.out b/regression-test/data/query_p0/sql_functions/array_functions/test_array_functions.out
index d8fb5e32af..22a42eb4cf 100644
--- a/regression-test/data/query_p0/sql_functions/array_functions/test_array_functions.out
+++ b/regression-test/data/query_p0/sql_functions/array_functions/test_array_functions.out
@@ -590,14 +590,14 @@
 10006	[60002, 60002, 60003, NULL, 60005]	[NULL]
 
 -- !select_array_datetimev2_1 --
-1	[2023-01-19 18:11:11.111100, 2023-01-19 18:22:22.222200, 2023-01-19 18:33:33.333300]	[2023-01-19 18:22:22.222200, 2023-01-19 18:33:33.333300, 2023-01-19 18:44:44.444400]	[2023-01-19 18:11:11.111111, 2023-01-19 18:22:22.222222, 2023-01-19 18:33:33.333333]
+1	[2023-01-19 18:11:11.111, 2023-01-19 18:22:22.222, 2023-01-19 18:33:33.333]	[2023-01-19 18:22:22.222, 2023-01-19 18:33:33.333, 2023-01-19 18:44:44.444]	[2023-01-19 18:11:11.111111, 2023-01-19 18:22:22.222222, 2023-01-19 18:33:33.333333]
 
 -- !select_array_datetimev2_2 --
-[2023-01-19 18:11:11.111100, 2023-01-19 18:22:22.222200, 2023-01-19 18:33:33.333300]
+[2023-01-19 18:11:11, 2023-01-19 18:22:22, 2023-01-19 18:33:33]
 
 -- !select_array_datetimev2_3 --
-[2023-01-19 18:22:22.222200, 2023-01-19 18:33:33.333300, 2023-01-19 18:44:44.444400]
+[2023-01-19 18:22:22, 2023-01-19 18:33:33, 2023-01-19 18:44:44]
 
 -- !select_array_datetimev2_4 --
-[2023-01-19 18:11:11.111111, 2023-01-19 18:22:22.222222, 2023-01-19 18:33:33.333333]
+[2023-01-19 18:11:11, 2023-01-19 18:22:22, 2023-01-19 18:33:33]
 
diff --git a/regression-test/data/query_p0/sql_functions/array_functions/test_array_with_scale_type.out b/regression-test/data/query_p0/sql_functions/array_functions/test_array_with_scale_type.out
new file mode 100644
index 0000000000..2376a7bb1f
--- /dev/null
+++ b/regression-test/data/query_p0/sql_functions/array_functions/test_array_with_scale_type.out
@@ -0,0 +1,80 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select --
+2022-12-02T22:23:24.999999
+
+-- !select --
+2022-12-02T22:23:24.999
+
+-- !select --
+2022-12-01T22:23:24.999
+2022-12-02T22:23:24.999
+
+-- !select --
+2022-12-02T22:23:24.999999
+
+-- !select --
+2022-12-02T22:23:24.999
+
+-- !select --
+2022-12-01T23:23:24.999
+2022-12-02T23:23:24.999
+
+-- !select --
+22.000000000
+
+-- !select --
+22.990000000
+
+-- !select --
+22.990000000
+
+-- !select --
+22.678000000
+23.678000000
+
+-- !select --
+22.000000000
+
+-- !select --
+22.990000000
+
+-- !select --
+22.990000000
+
+-- !select --
+33.678900000
+34.678900000
+
+-- !select --
+[22.678]
+[23.678]
+
+-- !select --
+[24.99, 25.99]
+[24.99, 25.99]
+
+-- !select --
+[24.990, 25.990]
+
+-- !select --
+[33.678]
+[34.678]
+
+-- !select --
+[24.990, 25.990]
+[24.990, 25.990]
+
+-- !select --
+[24.990, 25.990]
+
+-- !select --
+[2022-12-01 22:23:24.999]
+[2022-12-02 22:23:24.999]
+
+-- !select --
+[2022-12-02 22:23:24.999, 2022-12-02 22:23:23.997]
+[2022-12-02 22:23:24.999, 2022-12-02 22:23:23.997]
+
+-- !select --
+[2022-12-02 22:23:24.999, 2022-12-02 22:23:23.997]
+
diff --git a/regression-test/data/query_p0/sql_functions/cast_function/test_cast_with_scale_type.out b/regression-test/data/query_p0/sql_functions/cast_function/test_cast_with_scale_type.out
new file mode 100644
index 0000000000..def023304e
--- /dev/null
+++ b/regression-test/data/query_p0/sql_functions/cast_function/test_cast_with_scale_type.out
@@ -0,0 +1,16 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select1 --
+1	2022-12-01T22:23:24.999	2022-12-01 22:23:24.999999
+2	2022-12-02T22:23:24.999	2022-12-02 22:23:24.999999
+
+-- !select2 --
+1	2022-12-01T22:23:24.999	2022-12-01T22:23:24.999
+2	2022-12-02T22:23:24.999	2022-12-02T22:23:24.999
+
+-- !select3 --
+2022-12-02T22:23:24.999	2022-12-02T22:23:23.999
+2022-12-02T22:23:24.999	2022-12-02T22:23:23.999
+
+-- !select4 --
+2022-12-02T22:23:24.999	2022-12-02T22:23:23.999
+
diff --git a/regression-test/suites/datatype_p0/date/test_date_runtime_filter.groovy b/regression-test/suites/datatype_p0/date/test_date_runtime_filter.groovy
new file mode 100644
index 0000000000..f1526c1517
--- /dev/null
+++ b/regression-test/suites/datatype_p0/date/test_date_runtime_filter.groovy
@@ -0,0 +1,49 @@
+
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite("test_date_runtime_filter") {
+    def tbName = "test_date_runtime_filter"
+    sql "DROP TABLE IF EXISTS ${tbName}"
+    sql """
+            CREATE TABLE IF NOT EXISTS ${tbName} (
+                c0 int,
+                c2 date,
+                c3 datetime
+            )
+            DISTRIBUTED BY HASH(c0) BUCKETS 5 properties("replication_num" = "1");
+        """
+    sql "insert into ${tbName} values(1, '2000-01-01', '2000-01-01 11:11:11')"
+    sql "insert into ${tbName} values(2, '2000-02-02', '2000-02-02 11:11:11')"
+    sql "insert into ${tbName} values(3, '2000-03-02', '2000-03-02 11:11:11')"
+
+    qt_sql1 "select * from ${tbName} ORDER BY c2"
+
+    sql " set runtime_filter_type = 1; "
+    qt_sql2 "select * from ${tbName} a, ${tbName} b WHERE a.c3 = b.c3 ORDER BY a.c2"
+
+    sql " set runtime_filter_type = 2; "
+    qt_sql2 "select * from ${tbName} a, ${tbName} b WHERE a.c3 = b.c3 ORDER BY a.c2"
+
+    sql " set runtime_filter_type = 4; "
+    qt_sql2 "select * from ${tbName} a, ${tbName} b WHERE a.c3 = b.c3 ORDER BY a.c2"
+
+    sql " set runtime_filter_type = 8; "
+    qt_sql2 "select * from ${tbName} a, ${tbName} b WHERE a.c3 = b.c3 ORDER BY a.c2"
+
+    sql "DROP TABLE ${tbName}"
+}
diff --git a/regression-test/suites/datatype_p0/decimalv3/test_functions.groovy b/regression-test/suites/datatype_p0/decimalv3/test_functions.groovy
new file mode 100644
index 0000000000..a0257e6441
--- /dev/null
+++ b/regression-test/suites/datatype_p0/decimalv3/test_functions.groovy
@@ -0,0 +1,53 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+import org.codehaus.groovy.runtime.IOGroovyMethods
+
+import java.nio.charset.StandardCharsets
+import java.nio.file.Files
+import java.nio.file.Paths
+
+suite("test_functions") {
+    def dbName = "test_functions"
+    sql "CREATE DATABASE IF NOT EXISTS ${dbName}"
+    sql "USE $dbName"
+
+    def tableName = "test_decimal_load"
+    try {
+        sql """ DROP TABLE IF EXISTS ${tableName} """
+        sql """
+        CREATE TABLE IF NOT EXISTS ${tableName} (
+              id int(11) NULL,
+              s_count int(11) NULL,
+              fee DECIMALV3(15,4) NULL
+            ) ENGINE=OLAP
+            DUPLICATE KEY(`id`)
+            COMMENT 'OLAP'
+            DISTRIBUTED BY HASH(`id`) BUCKETS 1
+            PROPERTIES (
+            "replication_allocation" = "tag.location.default: 1"
+        );
+        """
+
+        sql """ INSERT INTO ${tableName} VALUES(10007,26,13.3), (10001,12,13.3) """
+        qt_select_default """ SELECT id, if(1 = 2,
+                                     1.0*s_count - fee,
+                                     fee) FROM ${tableName} t ORDER BY id; """
+    } finally {
+        try_sql("DROP TABLE IF EXISTS ${tableName}")
+    }
+}
diff --git a/regression-test/suites/query_p0/sql_functions/array_functions/test_array_functions.groovy b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_functions.groovy
index 4b4203a461..0d692bf982 100644
--- a/regression-test/suites/query_p0/sql_functions/array_functions/test_array_functions.groovy
+++ b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_functions.groovy
@@ -160,8 +160,8 @@ suite("test_array_functions") {
     sql """
             CREATE TABLE IF NOT EXISTS ${tableName4} (
               `k1` int COMMENT "",
-              `k2` ARRAY<datetimev2(4)> COMMENT "",
-              `k3` ARRAY<datetimev2(4)> COMMENT "",
+              `k2` ARRAY<datetimev2(3)> COMMENT "",
+              `k3` ARRAY<datetimev2(3)> COMMENT "",
               `k4` ARRAY<datetimev2(6)> COMMENT ""
             ) ENGINE=OLAP
             DUPLICATE KEY(`k1`)
@@ -177,6 +177,7 @@ suite("test_array_functions") {
                                             ["2023-01-19 18:11:11.111111","2023-01-19 18:22:22.222222","2023-01-19 18:33:33.333333"]) """
 
     qt_select_array_datetimev2_1 "SELECT * FROM ${tableName4}"
+    // Todo(Yukang-Lian): will fix if function bugs in the future
     qt_select_array_datetimev2_2 "SELECT if(1,k2,k3) FROM ${tableName4}"
     qt_select_array_datetimev2_3 "SELECT if(0,k2,k3) FROM ${tableName4}"
     qt_select_array_datetimev2_4 "SELECT if(0,k2,k4) FROM ${tableName4}"
diff --git a/regression-test/suites/query_p0/sql_functions/array_functions/test_array_with_scale_type.groovy b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_with_scale_type.groovy
new file mode 100644
index 0000000000..827f03858c
--- /dev/null
+++ b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_with_scale_type.groovy
@@ -0,0 +1,72 @@
+// 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_array_with_scale_type") {
+
+    def tableName = "test_array_with_scale_type_table"
+        sql "DROP TABLE IF EXISTS ${tableName}"
+        sql """
+            CREATE TABLE IF NOT EXISTS `${tableName}` (
+            `uid` int(11) NULL COMMENT "",
+            `c_datetimev2` datetimev2(3) NULL COMMENT "",
+            `c_decimal` decimal(8,3) NULL COMMENT "",
+            `c_decimalv3` decimalv3(8,3) NULL COMMENT "",
+            `c_array_datetimev2` ARRAY<datetimev2(3)> NULL COMMENT "",
+            `c_array_decimal` ARRAY<decimal(8,3)> NULL COMMENT ""
+            ) ENGINE=OLAP
+        DUPLICATE KEY(`uid`)
+        DISTRIBUTED BY HASH(`uid`) BUCKETS 1
+        PROPERTIES (
+        "replication_allocation" = "tag.location.default: 1",
+        "storage_format" = "V2"
+        )
+        """
+
+        sql """INSERT INTO ${tableName} values
+        (1,"2022-12-01 22:23:24.999999",22.678,33.6789,["2022-12-01 22:23:24.999999","2022-12-01 23:23:24.999999"],[22.678,33.6789]),
+        (2,"2022-12-02 22:23:24.999999",23.678,34.6789,["2022-12-02 22:23:24.999999","2022-12-02 23:23:24.999999"],[23.678,34.6789])
+        """
+        qt_select  "select array_min(array(cast ('2022-12-02 22:23:24.999999' as datetimev2(6))))"
+        qt_select  "select array_min(array(cast ('2022-12-02 22:23:24.999999' as datetimev2(3))))"
+        qt_select  "select array_min(c_array_datetimev2) from ${tableName}"
+        qt_select  "select array_max(array(cast ('2022-12-02 22:23:24.999999' as datetimev2(6))))"
+        qt_select  "select array_max(array(cast ('2022-12-02 22:23:24.999999' as datetimev2(3))))"
+        qt_select  "select array_max(c_array_datetimev2) from ${tableName}"
+
+        qt_select  "select array_min(array(cast (22.99 as decimal)))"
+        qt_select  "select array_min(array(cast (22.99 as decimal(10,3))))"
+        qt_select  "select array_min(array(cast (22.99 as decimal(10,6))))"
+        qt_select  "select array_min(c_array_decimal) from ${tableName}"
+        qt_select  "select array_max(array(cast (22.99 as decimal)))"
+        qt_select  "select array_max(array(cast (22.99 as decimal(10,3))))"
+        qt_select  "select array_max(array(cast (22.99 as decimal(10,6))))"
+        qt_select  "select array_max(c_array_decimal) from ${tableName}"
+
+        qt_select  "select array(c_decimal) from ${tableName}"
+        qt_select  "select array(cast (24.99 as decimal(10,3)),cast (25.99 as decimal(10,3))) from ${tableName}"
+        qt_select  "select array(cast (24.99 as decimal(10,3)),cast (25.99 as decimal(10,3)))"
+
+        qt_select  "select array(c_decimalv3) from ${tableName}"
+        qt_select  "select array(cast (24.99 as decimalv3(10,3)),cast (25.99 as decimalv3(10,3))) from ${tableName}"
+        qt_select  "select array(cast (24.99 as decimalv3(10,3)),cast (25.99 as decimalv3(10,3)))"
+
+        qt_select "select array(c_datetimev2) from ${tableName}"
+        qt_select "select array(cast ('2022-12-02 22:23:24.999999' as datetimev2(3)),cast ('2022-12-02 22:23:23.997799' as datetimev2(3))) from ${tableName}"
+        qt_select "select array(cast ('2022-12-02 22:23:24.999999' as datetimev2(3)),cast ('2022-12-02 22:23:23.997799' as datetimev2(3)))"
+
+        sql "DROP TABLE IF EXISTS ${tableName}"
+}
\ No newline at end of file
diff --git a/regression-test/suites/query_p0/sql_functions/cast_function/test_cast_with_scale_type.groovy b/regression-test/suites/query_p0/sql_functions/cast_function/test_cast_with_scale_type.groovy
new file mode 100644
index 0000000000..5b22a0a479
--- /dev/null
+++ b/regression-test/suites/query_p0/sql_functions/cast_function/test_cast_with_scale_type.groovy
@@ -0,0 +1,49 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite("test_cast_with_scale_type") {
+
+    def tableName = "test_cast_with_scale_type_table"
+        sql "DROP TABLE IF EXISTS ${tableName}"
+        sql """
+            CREATE TABLE IF NOT EXISTS `${tableName}` (
+            `uid` int(11) NULL COMMENT "",
+            `c_datetimev2` datetimev2(3) NULL COMMENT "",
+            `c_string` varchar(30) NULL COMMENT ""
+            ) ENGINE=OLAP
+        DUPLICATE KEY(`uid`)
+        COMMENT "OLAP"
+        DISTRIBUTED BY HASH(`uid`) BUCKETS 1
+        PROPERTIES (
+        "replication_allocation" = "tag.location.default: 1",
+        "in_memory" = "false",
+        "storage_format" = "V2"
+        )
+        """
+
+        sql """INSERT INTO ${tableName} values
+        (1,"2022-12-01 22:23:24.999999",'2022-12-01 22:23:24.999999'),
+        (2,"2022-12-02 22:23:24.999999",'2022-12-02 22:23:24.999999')
+        """
+        
+        qt_select1 "select * from ${tableName}"
+        qt_select2 "select uid, c_datetimev2, cast(c_string as datetimev2(3)) from ${tableName} order by uid asc"
+        qt_select3 "select cast ('2022-12-02 22:23:24.999999' as datetimev2(3)),cast ('2022-12-02 22:23:23.999999' as datetimev2(3)) from ${tableName}"
+        qt_select4 "select cast ('2022-12-02 22:23:24.999999' as datetimev2(3)),cast ('2022-12-02 22:23:23.999999' as datetimev2(3))"
+
+        sql "DROP TABLE IF EXISTS ${tableName}"
+}
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org