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 2022/10/21 02:00:08 UTC

[doris] branch master updated: [Improvement](decimal) print decimal according to the real precision and scale (#13437)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 9a3c1f0867 [Improvement](decimal) print decimal according to the real precision and scale (#13437)
9a3c1f0867 is described below

commit 9a3c1f0867d6d0acd8a0cac7d66eabbca3487d52
Author: Gabriel <ga...@gmail.com>
AuthorDate: Fri Oct 21 10:00:01 2022 +0800

    [Improvement](decimal) print decimal according to the real precision and scale (#13437)
---
 be/src/runtime/mysql_result_writer.cpp             |   2 +-
 be/src/vec/sink/vmysql_result_writer.cpp           |  20 +-
 .../org/apache/doris/analysis/ArithmeticExpr.java  |  14 +-
 .../java/org/apache/doris/analysis/CastExpr.java   |   5 +
 .../apache/doris/analysis/ExpressionFunctions.java |   2 +
 .../apache/doris/analysis/SetOperationStmt.java    |  27 ++-
 .../java/org/apache/doris/catalog/FunctionSet.java |  80 ++++----
 .../java/org/apache/doris/catalog/ScalarType.java  |  18 +-
 .../main/java/org/apache/doris/catalog/Type.java   |   7 +-
 gensrc/script/gen_builtins_functions.py            |   2 +
 .../data/datev2/tpcds_sf1_p1/sql/q58.out           |   6 +-
 .../data/datev2/tpcds_sf1_p1/sql/q83.out           |  42 ++--
 .../data/load_p0/broker_load/test_array_load.out   |  94 ++-------
 .../query/operator/test_arithmetic_operators.out   | 142 ++++++-------
 .../data/query/operator/test_logical_operators.out |  72 +++----
 regression-test/data/query_p0/union/test_union.out | 222 +++++++++------------
 regression-test/data/tpcds_sf1_p1/sql/q58.out      |   6 +-
 regression-test/data/tpcds_sf1_p1/sql/q83.out      |  42 ++--
 .../data/tpcds_sf1_unique_p1/sql/q58.out           |   6 +-
 .../data/tpcds_sf1_unique_p1/sql/q83.out           |  42 ++--
 .../duplicate/storage/test_dup_tab_decimal.groovy  |  28 +--
 .../storage/test_dup_tab_decimal_nullable.groovy   |  24 +--
 .../suites/query_p0/union/test_union.groovy        |   2 -
 23 files changed, 426 insertions(+), 479 deletions(-)

diff --git a/be/src/runtime/mysql_result_writer.cpp b/be/src/runtime/mysql_result_writer.cpp
index f7c5f43930..2d4e3541c9 100644
--- a/be/src/runtime/mysql_result_writer.cpp
+++ b/be/src/runtime/mysql_result_writer.cpp
@@ -156,7 +156,7 @@ int MysqlResultWriter::_add_row_value(int index, const TypeDescriptor& type, voi
         // TODO: Support decimal output_scale after we support FE can sure
         // accuracy of output_scale
         // int output_scale = _output_expr_ctxs[index]->root()->output_scale();
-        buf_ret = _row_buffer->push_decimal(decimal_val, -1);
+        buf_ret = _row_buffer->push_decimal(decimal_val, type.scale);
         break;
     }
 
diff --git a/be/src/vec/sink/vmysql_result_writer.cpp b/be/src/vec/sink/vmysql_result_writer.cpp
index c53a1d7cfe..3d423b053e 100644
--- a/be/src/vec/sink/vmysql_result_writer.cpp
+++ b/be/src/vec/sink/vmysql_result_writer.cpp
@@ -264,7 +264,7 @@ Status VMysqlResultWriter::_add_one_column(const ColumnPtr& column_ptr,
             }
             if constexpr (type == TYPE_DECIMALV2) {
                 DecimalV2Value decimal_val(data[i]);
-                auto decimal_str = decimal_val.to_string();
+                auto decimal_str = decimal_val.to_string(scale);
                 buf_ret = _buffer.push_string(decimal_str.c_str(), decimal_str.length());
             }
 
@@ -449,6 +449,7 @@ Status VMysqlResultWriter::append_block(Block& input_block) {
         auto column_ptr = block.get_by_position(i).column->convert_to_full_column_if_const();
         auto type_ptr = block.get_by_position(i).type;
 
+        int scale = _output_vexpr_ctxs[i]->root()->type().scale;
         switch (_output_vexpr_ctxs[i]->root()->result_type()) {
         case TYPE_BOOLEAN:
             if (type_ptr->is_nullable()) {
@@ -544,10 +545,10 @@ Status VMysqlResultWriter::append_block(Block& input_block) {
                 auto& nested_type =
                         assert_cast<const DataTypeNullable&>(*type_ptr).get_nested_type();
                 status = _add_one_column<PrimitiveType::TYPE_DECIMALV2, true>(column_ptr, result,
-                                                                              nested_type);
+                                                                              nested_type, scale);
             } else {
                 status = _add_one_column<PrimitiveType::TYPE_DECIMALV2, false>(column_ptr, result,
-                                                                               type_ptr);
+                                                                               type_ptr, scale);
             }
             break;
         }
@@ -556,10 +557,10 @@ Status VMysqlResultWriter::append_block(Block& input_block) {
                 auto& nested_type =
                         assert_cast<const DataTypeNullable&>(*type_ptr).get_nested_type();
                 status = _add_one_column<PrimitiveType::TYPE_DECIMAL32, true>(column_ptr, result,
-                                                                              nested_type);
+                                                                              nested_type, scale);
             } else {
                 status = _add_one_column<PrimitiveType::TYPE_DECIMAL32, false>(column_ptr, result,
-                                                                               type_ptr);
+                                                                               type_ptr, scale);
             }
             break;
         }
@@ -568,10 +569,10 @@ Status VMysqlResultWriter::append_block(Block& input_block) {
                 auto& nested_type =
                         assert_cast<const DataTypeNullable&>(*type_ptr).get_nested_type();
                 status = _add_one_column<PrimitiveType::TYPE_DECIMAL64, true>(column_ptr, result,
-                                                                              nested_type);
+                                                                              nested_type, scale);
             } else {
                 status = _add_one_column<PrimitiveType::TYPE_DECIMAL64, false>(column_ptr, result,
-                                                                               type_ptr);
+                                                                               type_ptr, scale);
             }
             break;
         }
@@ -580,10 +581,10 @@ Status VMysqlResultWriter::append_block(Block& input_block) {
                 auto& nested_type =
                         assert_cast<const DataTypeNullable&>(*type_ptr).get_nested_type();
                 status = _add_one_column<PrimitiveType::TYPE_DECIMAL128, true>(column_ptr, result,
-                                                                               nested_type);
+                                                                               nested_type, scale);
             } else {
                 status = _add_one_column<PrimitiveType::TYPE_DECIMAL128, false>(column_ptr, result,
-                                                                                type_ptr);
+                                                                                type_ptr, scale);
             }
             break;
         }
@@ -613,7 +614,6 @@ Status VMysqlResultWriter::append_block(Block& input_block) {
             break;
         }
         case TYPE_DATETIMEV2: {
-            int scale = _output_vexpr_ctxs[i]->root()->type().scale;
             if (type_ptr->is_nullable()) {
                 status = _add_one_column<PrimitiveType::TYPE_DATETIMEV2, true>(column_ptr, result,
                                                                                nullptr, scale);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java
index 0e8619b827..38d0898ffe 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java
@@ -117,8 +117,8 @@ public class ArithmeticExpr extends Expr {
                 Type.DOUBLE, Function.NullableMode.ALWAYS_NULLABLE));
         functionSet.addBuiltin(ScalarFunction.createBuiltinOperator(
                 Operator.DIVIDE.getName(),
-                Lists.<Type>newArrayList(Type.DECIMALV2, Type.DECIMALV2),
-                Type.DECIMALV2, Function.NullableMode.ALWAYS_NULLABLE));
+                Lists.<Type>newArrayList(Type.MAX_DECIMALV2_TYPE, Type.MAX_DECIMALV2_TYPE),
+                Type.MAX_DECIMALV2_TYPE, Function.NullableMode.ALWAYS_NULLABLE));
         functionSet.addBuiltin(ScalarFunction.createBuiltinOperator(
                 Operator.DIVIDE.getName(),
                 Lists.<Type>newArrayList(Type.DECIMAL32, Type.DECIMAL32),
@@ -164,8 +164,8 @@ public class ArithmeticExpr extends Expr {
                 Type.DOUBLE, Function.NullableMode.ALWAYS_NULLABLE));
         functionSet.addBuiltin(ScalarFunction.createVecBuiltinOperator(
                 Operator.DIVIDE.getName(),
-                Lists.<Type>newArrayList(Type.DECIMALV2, Type.DECIMALV2),
-                Type.DECIMALV2, Function.NullableMode.ALWAYS_NULLABLE));
+                Lists.<Type>newArrayList(Type.MAX_DECIMALV2_TYPE, Type.MAX_DECIMALV2_TYPE),
+                Type.MAX_DECIMALV2_TYPE, Function.NullableMode.ALWAYS_NULLABLE));
         functionSet.addBuiltin(ScalarFunction.createVecBuiltinOperator(
                 Operator.DIVIDE.getName(),
                 Lists.<Type>newArrayList(Type.DECIMAL32, Type.DECIMAL32),
@@ -189,8 +189,8 @@ public class ArithmeticExpr extends Expr {
                 Type.DOUBLE, Function.NullableMode.ALWAYS_NULLABLE));
         functionSet.addBuiltin(ScalarFunction.createVecBuiltinOperator(
                 Operator.MOD.getName(),
-                Lists.<Type>newArrayList(Type.DECIMALV2, Type.DECIMALV2),
-                Type.DECIMALV2, Function.NullableMode.ALWAYS_NULLABLE));
+                Lists.<Type>newArrayList(Type.MAX_DECIMALV2_TYPE, Type.MAX_DECIMALV2_TYPE),
+                Type.MAX_DECIMALV2_TYPE, Function.NullableMode.ALWAYS_NULLABLE));
         functionSet.addBuiltin(ScalarFunction.createVecBuiltinOperator(
                 Operator.MOD.getName(),
                 Lists.<Type>newArrayList(Type.DECIMAL32, Type.DECIMAL32),
@@ -309,7 +309,7 @@ public class ArithmeticExpr extends Expr {
         if (pt1 == PrimitiveType.DOUBLE || pt2 == PrimitiveType.DOUBLE) {
             return Type.DOUBLE;
         } else if (pt1 == PrimitiveType.DECIMALV2 || pt2 == PrimitiveType.DECIMALV2) {
-            return Type.DECIMALV2;
+            return Type.MAX_DECIMALV2_TYPE;
         } else if (pt1 == PrimitiveType.DECIMAL32 || pt2 == PrimitiveType.DECIMAL32) {
             return Type.DECIMAL32;
         } else if (pt1 == PrimitiveType.DECIMAL64 || pt2 == PrimitiveType.DECIMAL64) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CastExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CastExpr.java
index 1a095ad827..87bfff145a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CastExpr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CastExpr.java
@@ -284,6 +284,11 @@ public class CastExpr extends Expr {
         noOp = Type.matchExactType(childType, type);
 
         if (noOp) {
+            // For decimalv2, we do not perform an actual cast between different precision/scale. Instead, we just
+            // set the target type as the child's type.
+            if (type.isDecimalV2() && childType.isDecimalV2()) {
+                getChild(0).setType(type);
+            }
             return;
         }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ExpressionFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ExpressionFunctions.java
index 1162827784..02c7b26b25 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ExpressionFunctions.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ExpressionFunctions.java
@@ -120,6 +120,7 @@ public enum ExpressionFunctions {
             if (!(invoker.getSignature().returnType.isDate() && signature.getReturnType().isDateV2())
                     && !(invoker.getSignature().returnType.isDatetime() && signature.getReturnType().isDatetimeV2())
                     && !(invoker.getSignature().returnType.isDecimalV2() && signature.getReturnType().isDecimalV3())
+                    && !(invoker.getSignature().returnType.isDecimalV2() && signature.getReturnType().isDecimalV2())
                     && !invoker.getSignature().returnType.equals(signature.getReturnType())) {
                 continue;
             }
@@ -135,6 +136,7 @@ public enum ExpressionFunctions {
                 if (!(argTypes1[i].isDate() && argTypes2[i].isDateV2())
                         && !(argTypes1[i].isDatetime() && argTypes2[i].isDatetimeV2())
                         && !(argTypes1[i].isDecimalV2() && argTypes2[i].isDecimalV3())
+                        && !(argTypes1[i].isDecimalV2() && argTypes2[i].isDecimalV2())
                         && !argTypes1[i].equals(argTypes2[i])) {
                     match = false;
                     break;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetOperationStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetOperationStmt.java
index c8cb2a8ccb..b599d2cb98 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetOperationStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetOperationStmt.java
@@ -17,8 +17,11 @@
 
 package org.apache.doris.analysis;
 
+import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.catalog.TableIf;
+import org.apache.doris.catalog.Type;
 import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.Pair;
 import org.apache.doris.common.UserException;
 import org.apache.doris.rewrite.ExprRewriter;
 
@@ -32,6 +35,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 /**
  * Representation of a set ops with its list of operands, and optional order by and limit.
@@ -476,8 +480,20 @@ public class SetOperationStmt extends QueryStmt {
             LOG.trace("SetOperationStmt.createMetadata: tupleId=" + tupleId.toString());
         }
 
-        // One slot per expr in the select blocks. Use first select block as representative.
-        List<Expr> firstSelectExprs = operands.get(0).getQueryStmt().getResultExprs();
+        // When multiple operands exist here, we should use compatible type for each slot. For example,
+        // for `SELECT decimal(10,1) UNION ALL decimal(6,4)`, we should use decimal(10,4) as the result type.
+        List<Pair<Type, Boolean>> selectTypeWithNullable = operands.get(0).getQueryStmt().getResultExprs().stream()
+                .map(expr -> Pair.of(expr.getType(), expr.isNullable())).collect(Collectors.toList());
+        for (int i = 1; i < operands.size(); i++) {
+            for (int j = 1; j < selectTypeWithNullable.size(); j++) {
+                if (selectTypeWithNullable.get(j).first.isDecimalV2()
+                        && operands.get(i).getQueryStmt().getResultExprs().get(j).getType().isDecimalV2()) {
+                    selectTypeWithNullable.get(j).first = ScalarType.getAssignmentCompatibleDecimalV2Type(
+                            (ScalarType) selectTypeWithNullable.get(j).first,
+                            (ScalarType) operands.get(i).getQueryStmt().getResultExprs().get(j).getType());
+                }
+            }
+        }
 
         // TODO(zc) Column stats
         /*
@@ -497,12 +513,11 @@ public class SetOperationStmt extends QueryStmt {
         */
 
         // Create tuple descriptor and slots.
-        for (int i = 0; i < firstSelectExprs.size(); ++i) {
-            Expr expr = firstSelectExprs.get(i);
+        for (int i = 0; i < selectTypeWithNullable.size(); ++i) {
             SlotDescriptor slotDesc = analyzer.addSlotDescriptor(tupleDesc);
             slotDesc.setLabel(getColLabels().get(i));
-            slotDesc.setType(expr.getType());
-            slotDesc.setIsNullable(expr.isNullable());
+            slotDesc.setType(selectTypeWithNullable.get(i).first);
+            slotDesc.setIsNullable(selectTypeWithNullable.get(i).second);
             // TODO(zc)
             // slotDesc.setStats(columnStats.get(i));
             SlotRef outputSlotRef = new SlotRef(slotDesc);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java
index eab49faa5e..c380485dab 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java
@@ -140,7 +140,7 @@ public class FunctionSet<T> {
                             "8min_initIN9doris_udf11DateTimeValEEEvPNS2_15FunctionContextEPT_")
                     .put(Type.DATETIME,
                             "8min_initIN9doris_udf11DateTimeValEEEvPNS2_15FunctionContextEPT_")
-                    .put(Type.DECIMALV2,
+                    .put(Type.MAX_DECIMALV2_TYPE,
                             "8min_initIN9doris_udf12DecimalV2ValEEEvPNS2_15FunctionContextEPT_")
                     .put(Type.LARGEINT,
                             "8min_initIN9doris_udf11LargeIntValEEEvPNS2_15FunctionContextEPT_")
@@ -172,7 +172,7 @@ public class FunctionSet<T> {
                             "8max_initIN9doris_udf11DateTimeValEEEvPNS2_15FunctionContextEPT_")
                     .put(Type.DATETIME,
                             "8max_initIN9doris_udf11DateTimeValEEEvPNS2_15FunctionContextEPT_")
-                    .put(Type.DECIMALV2,
+                    .put(Type.MAX_DECIMALV2_TYPE,
                             "8max_initIN9doris_udf12DecimalV2ValEEEvPNS2_15FunctionContextEPT_")
                     .put(Type.LARGEINT,
                             "8max_initIN9doris_udf11LargeIntValEEEvPNS2_15FunctionContextEPT_")
@@ -204,7 +204,7 @@ public class FunctionSet<T> {
                             "3minIN9doris_udf11DateTimeValEEEvPNS2_15FunctionContextERKT_PS6_")
                     .put(Type.DATETIME,
                             "3minIN9doris_udf11DateTimeValEEEvPNS2_15FunctionContextERKT_PS6_")
-                    .put(Type.DECIMALV2,
+                    .put(Type.MAX_DECIMALV2_TYPE,
                             "3minIN9doris_udf12DecimalV2ValEEEvPNS2_15FunctionContextERKT_PS6_")
                     .put(Type.LARGEINT,
                             "3minIN9doris_udf11LargeIntValEEEvPNS2_15FunctionContextERKT_PS6_")
@@ -236,7 +236,7 @@ public class FunctionSet<T> {
                     "3maxIN9doris_udf11DateTimeValEEEvPNS2_15FunctionContextERKT_PS6_")
                 .put(Type.DATETIME,
                     "3maxIN9doris_udf11DateTimeValEEEvPNS2_15FunctionContextERKT_PS6_")
-                .put(Type.DECIMALV2,
+                .put(Type.MAX_DECIMALV2_TYPE,
                     "3maxIN9doris_udf12DecimalV2ValEEEvPNS2_15FunctionContextERKT_PS6_")
                 .put(Type.LARGEINT,
                     "3maxIN9doris_udf11LargeIntValEEEvPNS2_15FunctionContextERKT_PS6_")
@@ -268,7 +268,7 @@ public class FunctionSet<T> {
                             "8any_initIN9doris_udf11DateTimeValEEEvPNS2_15FunctionContextEPT_")
                     .put(Type.DATETIME,
                             "8any_initIN9doris_udf11DateTimeValEEEvPNS2_15FunctionContextEPT_")
-                    .put(Type.DECIMALV2,
+                    .put(Type.MAX_DECIMALV2_TYPE,
                             "8any_initIN9doris_udf12DecimalV2ValEEEvPNS2_15FunctionContextEPT_")
                     .put(Type.LARGEINT,
                             "8any_initIN9doris_udf11LargeIntValEEEvPNS2_15FunctionContextEPT_")
@@ -300,7 +300,7 @@ public class FunctionSet<T> {
                             "3anyIN9doris_udf11DateTimeValEEEvPNS2_15FunctionContextERKT_PS6_")
                     .put(Type.DATETIME,
                             "3anyIN9doris_udf11DateTimeValEEEvPNS2_15FunctionContextERKT_PS6_")
-                    .put(Type.DECIMALV2,
+                    .put(Type.MAX_DECIMALV2_TYPE,
                             "3anyIN9doris_udf12DecimalV2ValEEEvPNS2_15FunctionContextERKT_PS6_")
                     .put(Type.LARGEINT,
                             "3anyIN9doris_udf11LargeIntValEEEvPNS2_15FunctionContextERKT_PS6_")
@@ -315,7 +315,7 @@ public class FunctionSet<T> {
                     .put(Type.FLOAT, Type.DOUBLE)
                     .put(Type.DOUBLE, Type.DOUBLE)
                     .put(Type.LARGEINT, Type.LARGEINT)
-                    .put(Type.DECIMALV2, Type.DECIMALV2)
+                    .put(Type.MAX_DECIMALV2_TYPE, Type.MAX_DECIMALV2_TYPE)
                     .put(Type.DECIMAL32, Type.DECIMAL32)
                     .put(Type.DECIMAL64, Type.DECIMAL64)
                     .put(Type.DECIMAL128, Type.DECIMAL128)
@@ -432,7 +432,7 @@ public class FunctionSet<T> {
                     .put(Type.BIGINT, Type.DOUBLE)
                     .put(Type.FLOAT, Type.DOUBLE)
                     .put(Type.DOUBLE, Type.DOUBLE)
-                    .put(Type.DECIMALV2, Type.DECIMALV2)
+                    .put(Type.MAX_DECIMALV2_TYPE, Type.MAX_DECIMALV2_TYPE)
                     .put(Type.DECIMAL32, Type.DECIMAL32)
                     .put(Type.DECIMAL64, Type.DECIMAL64)
                     .put(Type.DECIMAL128, Type.DECIMAL128)
@@ -452,7 +452,7 @@ public class FunctionSet<T> {
                         "14knuth_var_initEPN9doris_udf15FunctionContextEPNS1_9StringValE")
                 .put(Type.DOUBLE,
                         "14knuth_var_initEPN9doris_udf15FunctionContextEPNS1_9StringValE")
-                .put(Type.DECIMALV2,
+                .put(Type.MAX_DECIMALV2_TYPE,
                         "24decimalv2_knuth_var_initEPN9doris_udf15FunctionContextEPNS1_9StringValE")
                 .build();
 
@@ -470,7 +470,7 @@ public class FunctionSet<T> {
                         "16knuth_var_updateIN9doris_udf8FloatValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE")
                 .put(Type.DOUBLE,
                         "16knuth_var_updateIN9doris_udf9DoubleValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE")
-                .put(Type.DECIMALV2,
+                .put(Type.MAX_DECIMALV2_TYPE,
                         "16knuth_var_updateEPN9doris_udf15FunctionContextERKNS1_12DecimalV2ValEPNS1_9StringValE")
                 .build();
 
@@ -489,7 +489,7 @@ public class FunctionSet<T> {
                         "16knuth_var_removeIN9doris_udf8FloatValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE")
                 .put(Type.DOUBLE,
                         "16knuth_var_removeIN9doris_udf9DoubleValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE")
-                .put(Type.DECIMALV2,
+                .put(Type.MAX_DECIMALV2_TYPE,
                         "16knuth_var_removeEPN9doris_udf15FunctionContextERKNS1_12DecimalV2ValEPNS1_9StringValE")
                 .build();
     private static final Map<Type, String> STDDEV_MERGE_SYMBOL =
@@ -506,7 +506,7 @@ public class FunctionSet<T> {
                         "15knuth_var_mergeEPN9doris_udf15FunctionContextERKNS1_9StringValEPS4_")
                 .put(Type.DOUBLE,
                         "15knuth_var_mergeEPN9doris_udf15FunctionContextERKNS1_9StringValEPS4_")
-                .put(Type.DECIMALV2,
+                .put(Type.MAX_DECIMALV2_TYPE,
                         "25decimalv2_knuth_var_mergeEPN9doris_udf15FunctionContextERKNS1_9StringValEPS4_")
                 .build();
 
@@ -524,7 +524,7 @@ public class FunctionSet<T> {
                         "21knuth_stddev_finalizeEPN9doris_udf15FunctionContextERKNS1_9StringValE")
                 .put(Type.DOUBLE,
                         "21knuth_stddev_finalizeEPN9doris_udf15FunctionContextERKNS1_9StringValE")
-                .put(Type.DECIMALV2,
+                .put(Type.MAX_DECIMALV2_TYPE,
                         "31decimalv2_knuth_stddev_finalizeEPN9doris_udf15FunctionContextERKNS1_9StringValE")
                 .build();
 
@@ -542,7 +542,7 @@ public class FunctionSet<T> {
                         "22knuth_stddev_get_valueEPN9doris_udf15FunctionContextERKNS1_9StringValE")
                 .put(Type.DOUBLE,
                         "22knuth_stddev_get_valueEPN9doris_udf15FunctionContextERKNS1_9StringValE")
-                .put(Type.DECIMALV2,
+                .put(Type.MAX_DECIMALV2_TYPE,
                         "32decimalv2_knuth_stddev_get_valueEPN9doris_udf15FunctionContextERKNS1_9StringValE")
                 .build();
 
@@ -560,7 +560,7 @@ public class FunctionSet<T> {
                             "25knuth_stddev_pop_finalizeEPN9doris_udf15FunctionContextERKNS1_9StringValE")
                     .put(Type.DOUBLE,
                             "25knuth_stddev_pop_finalizeEPN9doris_udf15FunctionContextERKNS1_9StringValE")
-                    .put(Type.DECIMALV2,
+                    .put(Type.MAX_DECIMALV2_TYPE,
                             "35decimalv2_knuth_stddev_pop_finalizeEPN9doris_udf15FunctionContextERKNS1_9StringValE")
                     .build();
 
@@ -578,7 +578,7 @@ public class FunctionSet<T> {
                 "26knuth_stddev_pop_get_valueEPN9doris_udf15FunctionContextERKNS1_9StringValE")
                 .put(Type.DOUBLE,
                 "26knuth_stddev_pop_get_valueEPN9doris_udf15FunctionContextERKNS1_9StringValE")
-                .put(Type.DECIMALV2,
+                .put(Type.MAX_DECIMALV2_TYPE,
                 "36decimalv2_knuth_stddev_pop_get_valueEPN9doris_udf15FunctionContextERKNS1_9StringValE")
                 .build();
 
@@ -596,7 +596,7 @@ public class FunctionSet<T> {
                         "18knuth_var_finalizeEPN9doris_udf15FunctionContextERKNS1_9StringValE")
                 .put(Type.DOUBLE,
                         "18knuth_var_finalizeEPN9doris_udf15FunctionContextERKNS1_9StringValE")
-                .put(Type.DECIMALV2,
+                .put(Type.MAX_DECIMALV2_TYPE,
                         "28decimalv2_knuth_var_finalizeEPN9doris_udf15FunctionContextERKNS1_9StringValE")
                 .build();
 
@@ -614,7 +614,7 @@ public class FunctionSet<T> {
                         "19knuth_var_get_valueEPN9doris_udf15FunctionContextERKNS1_9StringValE")
                 .put(Type.DOUBLE,
                         "19knuth_var_get_valueEPN9doris_udf15FunctionContextERKNS1_9StringValE")
-                .put(Type.DECIMALV2,
+                .put(Type.MAX_DECIMALV2_TYPE,
                         "29decimalv2_knuth_var_get_valueEPN9doris_udf15FunctionContextERKNS1_9StringValE")
                 .build();
 
@@ -632,7 +632,7 @@ public class FunctionSet<T> {
                         "22knuth_var_pop_finalizeEPN9doris_udf15FunctionContextERKNS1_9StringValE")
                 .put(Type.DOUBLE,
                         "22knuth_var_pop_finalizeEPN9doris_udf15FunctionContextERKNS1_9StringValE")
-                .put(Type.DECIMALV2,
+                .put(Type.MAX_DECIMALV2_TYPE,
                         "32decimalv2_knuth_var_pop_finalizeEPN9doris_udf15FunctionContextERKNS1_9StringValE")
                 .build();
 
@@ -650,7 +650,7 @@ public class FunctionSet<T> {
                         "23knuth_var_pop_get_valueEPN9doris_udf15FunctionContextERKNS1_9StringValE")
                 .put(Type.DOUBLE,
                         "23knuth_var_pop_get_valueEPN9doris_udf15FunctionContextERKNS1_9StringValE")
-                .put(Type.DECIMALV2,
+                .put(Type.MAX_DECIMALV2_TYPE,
                         "33decimalv2_knuth_var_pop_get_valueEPN9doris_udf15FunctionContextERKNS1_9StringValE")
                 .build();
     public static final String HLL_HASH = "hll_hash";
@@ -682,7 +682,7 @@ public class FunctionSet<T> {
                     "10hll_updateIN9doris_udf11DateTimeValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE")
                 .put(Type.DATETIME,
                     "10hll_updateIN9doris_udf11DateTimeValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE")
-                .put(Type.DECIMALV2,
+                .put(Type.MAX_DECIMALV2_TYPE,
                     "10hll_updateIN9doris_udf12DecimalV2ValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE")
                 .put(Type.LARGEINT,
                     "10hll_updateIN9doris_udf11LargeIntValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE")
@@ -703,7 +703,7 @@ public class FunctionSet<T> {
             ImmutableMap.<Type, String>builder()
                 .put(Type.BOOLEAN,
                      "14offset_fn_initIN9doris_udf10BooleanValEEEvPNS2_15FunctionContextEPT_")
-                .put(Type.DECIMALV2,
+                .put(Type.MAX_DECIMALV2_TYPE,
                      "14offset_fn_initIN9doris_udf12DecimalV2ValEEEvPNS2_15FunctionContextEPT_")
                 .put(Type.TINYINT,
                      "14offset_fn_initIN9doris_udf10TinyIntValEEEvPNS2_15FunctionContextEPT_")
@@ -736,7 +736,7 @@ public class FunctionSet<T> {
             ImmutableMap.<Type, String>builder()
                 .put(Type.BOOLEAN,
                      "16offset_fn_updateIN9doris_udf10BooleanValEEEvPNS2_15FunctionContextERKT_RKNS2_9BigIntValES8_PS6_")
-                .put(Type.DECIMALV2,
+                .put(Type.MAX_DECIMALV2_TYPE,
                      "16offset_fn_updateIN9doris_udf12DecimalV2ValEEEvPNS2_15FunctionContextERKT_RKNS2_9BigIntValES8_PS6_")
                 .put(Type.TINYINT,
                      "16offset_fn_updateIN9doris_udf10TinyIntValEEEvPNS2_15"
@@ -773,7 +773,7 @@ public class FunctionSet<T> {
             ImmutableMap.<Type, String>builder()
                     .put(Type.BOOLEAN,
                             "15last_val_updateIN9doris_udf10BooleanValEEEvPNS2_15FunctionContextERKT_PS6_")
-                    .put(Type.DECIMALV2,
+                    .put(Type.MAX_DECIMALV2_TYPE,
                             "15last_val_updateIN9doris_udf12DecimalV2ValEEEvPNS2_15FunctionContextERKT_PS6_")
                     .put(Type.TINYINT,
                             "15last_val_updateIN9doris_udf10TinyIntValEEEvPNS2_15FunctionContextERKT_PS6_")
@@ -806,7 +806,7 @@ public class FunctionSet<T> {
                     .put(Type.BOOLEAN,
                             "24first_val_rewrite_updateIN9doris_udf10BooleanValEEEvPNS2_15"
                                     + "FunctionContextERKT_RKNS2_9BigIntValEPS6_")
-                    .put(Type.DECIMALV2,
+                    .put(Type.MAX_DECIMALV2_TYPE,
                             "24first_val_rewrite_updateIN9doris_udf12DecimalV2ValEEEvPNS2_15"
                                     + "FunctionContextERKT_RKNS2_9BigIntValEPS6_")
                     .put(Type.TINYINT,
@@ -850,7 +850,7 @@ public class FunctionSet<T> {
             ImmutableMap.<Type, String>builder()
                     .put(Type.BOOLEAN,
                             "15last_val_removeIN9doris_udf10BooleanValEEEvPNS2_15FunctionContextERKT_PS6_")
-                    .put(Type.DECIMALV2,
+                    .put(Type.MAX_DECIMALV2_TYPE,
                             "15last_val_removeIN9doris_udf12DecimalV2ValEEEvPNS2_15FunctionContextERKT_PS6_")
                     .put(Type.TINYINT,
                             "15last_val_removeIN9doris_udf10TinyIntValEEEvPNS2_15FunctionContextERKT_PS6_")
@@ -882,7 +882,7 @@ public class FunctionSet<T> {
             ImmutableMap.<Type, String>builder()
                     .put(Type.BOOLEAN,
                             "16first_val_updateIN9doris_udf10BooleanValEEEvPNS2_15FunctionContextERKT_PS6_")
-                    .put(Type.DECIMALV2,
+                    .put(Type.MAX_DECIMALV2_TYPE,
                             "16first_val_updateIN9doris_udf12DecimalV2ValEEEvPNS2_15FunctionContextERKT_PS6_")
                     .put(Type.TINYINT,
                             "16first_val_updateIN9doris_udf10TinyIntValEEEvPNS2_15FunctionContextERKT_PS6_")
@@ -1045,7 +1045,7 @@ public class FunctionSet<T> {
                             "_ZN5doris15BitmapFunctions21bitmap_intersect_initINS_13DateTimeValueEN9doris_udf11DateTimeValEEEvPNS3_15FunctionContextEPNS3_9StringValE")
                     .put(Type.DATETIME,
                             "_ZN5doris15BitmapFunctions21bitmap_intersect_initINS_13DateTimeValueEN9doris_udf11DateTimeValEEEvPNS3_15FunctionContextEPNS3_9StringValE")
-                    .put(Type.DECIMALV2,
+                    .put(Type.MAX_DECIMALV2_TYPE,
                             "_ZN5doris15BitmapFunctions21bitmap_intersect_initINS_14DecimalV2ValueEN9doris_udf12DecimalV2ValEEEvPNS3_15FunctionContextEPNS3_9StringValE")
                     .put(Type.CHAR,
                             "_ZN5doris15BitmapFunctions21bitmap_intersect_initINS_11StringValueEN9doris_udf9StringValEEEvPNS3_15FunctionContextEPS4_")
@@ -1075,7 +1075,7 @@ public class FunctionSet<T> {
                             "_ZN5doris15BitmapFunctions23bitmap_intersect_updateINS_13DateTimeValueEN9doris_udf11DateTimeValEEEvPNS3_15FunctionContextERKNS3_9StringValERKT0_iPSB_PS8_")
                     .put(Type.DATETIME,
                             "_ZN5doris15BitmapFunctions23bitmap_intersect_updateINS_13DateTimeValueEN9doris_udf11DateTimeValEEEvPNS3_15FunctionContextERKNS3_9StringValERKT0_iPSB_PS8_")
-                    .put(Type.DECIMALV2,
+                    .put(Type.MAX_DECIMALV2_TYPE,
                             "_ZN5doris15BitmapFunctions23bitmap_intersect_updateINS_14DecimalV2ValueEN9doris_udf12DecimalV2ValEEEvPNS3_15FunctionContextERKNS3_9StringValERKT0_iPSB_PS8_")
                     .put(Type.CHAR,
                             "_ZN5doris15BitmapFunctions23bitmap_intersect_updateINS_11StringValueEN9doris_udf9StringValEEEvPNS3_15FunctionContextERKS4_RKT0_iPSA_PS7_")
@@ -1105,7 +1105,7 @@ public class FunctionSet<T> {
                             "_ZN5doris15BitmapFunctions22bitmap_intersect_mergeINS_13DateTimeValueEEEvPN9doris_udf15FunctionContextERKNS3_9StringValEPS7_")
                     .put(Type.DATETIME,
                             "_ZN5doris15BitmapFunctions22bitmap_intersect_mergeINS_13DateTimeValueEEEvPN9doris_udf15FunctionContextERKNS3_9StringValEPS7_")
-                    .put(Type.DECIMALV2,
+                    .put(Type.MAX_DECIMALV2_TYPE,
                             "_ZN5doris15BitmapFunctions22bitmap_intersect_mergeINS_14DecimalV2ValueEEEvPN9doris_udf15FunctionContextERKNS3_9StringValEPS7_")
                     .put(Type.CHAR,
                             "_ZN5doris15BitmapFunctions22bitmap_intersect_mergeINS_11StringValueEEEvPN9doris_udf15FunctionContextERKNS3_9StringValEPS7_")
@@ -1135,7 +1135,7 @@ public class FunctionSet<T> {
                             "_ZN5doris15BitmapFunctions26bitmap_intersect_serializeINS_13DateTimeValueEEEN9doris_udf9StringValEPNS3_15FunctionContextERKS4_")
                     .put(Type.DATETIME,
                             "_ZN5doris15BitmapFunctions26bitmap_intersect_serializeINS_13DateTimeValueEEEN9doris_udf9StringValEPNS3_15FunctionContextERKS4_")
-                    .put(Type.DECIMALV2,
+                    .put(Type.MAX_DECIMALV2_TYPE,
                             "_ZN5doris15BitmapFunctions26bitmap_intersect_serializeINS_14DecimalV2ValueEEEN9doris_udf9StringValEPNS3_15FunctionContextERKS4_")
                     .put(Type.CHAR,
                             "_ZN5doris15BitmapFunctions26bitmap_intersect_serializeINS_11StringValueEEEN9doris_udf9StringValEPNS3_15FunctionContextERKS4_")
@@ -1165,7 +1165,7 @@ public class FunctionSet<T> {
                             "_ZN5doris15BitmapFunctions25bitmap_intersect_finalizeINS_13DateTimeValueEEEN9doris_udf9BigIntValEPNS3_15FunctionContextERKNS3_9StringValE")
                     .put(Type.DATETIME,
                             "_ZN5doris15BitmapFunctions25bitmap_intersect_finalizeINS_13DateTimeValueEEEN9doris_udf9BigIntValEPNS3_15FunctionContextERKNS3_9StringValE")
-                    .put(Type.DECIMALV2,
+                    .put(Type.MAX_DECIMALV2_TYPE,
                             "_ZN5doris15BitmapFunctions25bitmap_intersect_finalizeINS_14DecimalV2ValueEEEN9doris_udf9BigIntValEPNS3_15FunctionContextERKNS3_9StringValE")
                     .put(Type.CHAR,
                             "_ZN5doris15BitmapFunctions25bitmap_intersect_finalizeINS_11StringValueEEEN9doris_udf9BigIntValEPNS3_15FunctionContextERKNS3_9StringValE")
@@ -1598,7 +1598,7 @@ public class FunctionSet<T> {
                         false, true, true));
                 // vectorized
                 // now we don't support datetime distinct
-            } else if (t.equals(Type.DECIMALV2)) {
+            } else if (t.equals(Type.MAX_DECIMALV2_TYPE)) {
                 addBuiltin(AggregateFunction.createBuiltin("multi_distinct_count", Lists.newArrayList(t),
                         Type.BIGINT,
                         Type.VARCHAR,
@@ -1613,7 +1613,7 @@ public class FunctionSet<T> {
                 // vectorized
                 addBuiltin(AggregateFunction.createBuiltin("multi_distinct_count", Lists.newArrayList(t),
                         Type.BIGINT,
-                        Type.DECIMALV2,
+                        Type.MAX_DECIMALV2_TYPE,
                         prefix + "",
                         prefix + "",
                         prefix + "",
@@ -1689,7 +1689,7 @@ public class FunctionSet<T> {
                         null,
                         prefix + MULTI_DISTINCT_SUM_FINALIZE_SYMBOL.get(t),
                         false, true, true, true));
-            }  else if (t.equals(Type.DECIMALV2)) {
+            }  else if (t.equals(Type.MAX_DECIMALV2_TYPE)) {
                 addBuiltin(AggregateFunction.createBuiltin("multi_distinct_sum", Lists.newArrayList(t),
                         MULTI_DISTINCT_SUM_RETURN_TYPE.get(t),
                         Type.VARCHAR,
@@ -1704,7 +1704,7 @@ public class FunctionSet<T> {
                 // vectorized
                 addBuiltin(AggregateFunction.createBuiltin("multi_distinct_sum", Lists.newArrayList(t),
                         MULTI_DISTINCT_SUM_RETURN_TYPE.get(t),
-                        Type.DECIMALV2,
+                        Type.MAX_DECIMALV2_TYPE,
                         prefix + "",
                         prefix + "",
                         prefix + "",
@@ -2160,7 +2160,7 @@ public class FunctionSet<T> {
                     prefix + "10sum_removeIN9doris_udf9DoubleValES3_EEvPNS2_15FunctionContextERKT_PT0_",
                     null, false, true, false));
             addBuiltin(AggregateFunction.createBuiltin(name,
-                    Lists.<Type>newArrayList(Type.DECIMALV2), Type.DECIMALV2, Type.DECIMALV2, prefix + "14init_zero_nullIN9doris_udf12DecimalV2ValEEEvPNS2_15FunctionContextEPT_",
+                    Lists.<Type>newArrayList(Type.MAX_DECIMALV2_TYPE), Type.MAX_DECIMALV2_TYPE, Type.MAX_DECIMALV2_TYPE, prefix + "14init_zero_nullIN9doris_udf12DecimalV2ValEEEvPNS2_15FunctionContextEPT_",
                     prefix + "3sumIN9doris_udf12DecimalV2ValES3_EEvPNS2_15FunctionContextERKT_PT0_",
                     prefix + "3sumIN9doris_udf12DecimalV2ValES3_EEvPNS2_15FunctionContextERKT_PT0_",
                     null, null,
@@ -2211,7 +2211,7 @@ public class FunctionSet<T> {
                     prefix + "10sum_removeIN9doris_udf9DoubleValES3_EEvPNS2_15FunctionContextERKT_PT0_",
                     null, false, true, false, true));
             addBuiltin(AggregateFunction.createBuiltin(name,
-                    Lists.<Type>newArrayList(Type.DECIMALV2), Type.DECIMALV2, Type.DECIMALV2, initNull,
+                    Lists.<Type>newArrayList(Type.MAX_DECIMALV2_TYPE), Type.MAX_DECIMALV2_TYPE, Type.MAX_DECIMALV2_TYPE, initNull,
                     prefix + "3sumIN9doris_udf12DecimalV2ValES3_EEvPNS2_15FunctionContextERKT_PT0_",
                     prefix + "3sumIN9doris_udf12DecimalV2ValES3_EEvPNS2_15FunctionContextERKT_PT0_",
                     null, null,
@@ -2492,7 +2492,7 @@ public class FunctionSet<T> {
                 false, true, false));
 
         addBuiltin(AggregateFunction.createBuiltin("avg",
-                Lists.<Type>newArrayList(Type.DECIMALV2), Type.DECIMALV2, Type.VARCHAR,
+                Lists.<Type>newArrayList(Type.MAX_DECIMALV2_TYPE), Type.MAX_DECIMALV2_TYPE, Type.VARCHAR,
                 prefix + "18decimalv2_avg_initEPN9doris_udf15FunctionContextEPNS1_9StringValE",
                 prefix + "20decimalv2_avg_updateEPN9doris_udf15FunctionContextERKNS1_12DecimalV2ValEPNS1_9StringValE",
                 prefix + "19decimalv2_avg_mergeEPN9doris_udf15FunctionContextERKNS1_9StringValEPS4_",
@@ -2545,7 +2545,7 @@ public class FunctionSet<T> {
                 prefix + "12avg_finalizeEPN9doris_udf15FunctionContextERKNS1_9StringValE",
                 false, true, false, true));
         addBuiltin(AggregateFunction.createBuiltin("avg",
-                Lists.<Type>newArrayList(Type.DECIMALV2), Type.DECIMALV2, Type.DECIMALV2,
+                Lists.<Type>newArrayList(Type.MAX_DECIMALV2_TYPE), Type.MAX_DECIMALV2_TYPE, Type.MAX_DECIMALV2_TYPE,
                 prefix + "18decimalv2_avg_initEPN9doris_udf15FunctionContextEPNS1_9StringValE",
                 prefix + "20decimalv2_avg_updateEPN9doris_udf15FunctionContextERKNS1_12DecimalV2ValEPNS1_9StringValE",
                 prefix + "19decimalv2_avg_mergeEPN9doris_udf15FunctionContextERKNS1_9StringValEPS4_",
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ScalarType.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ScalarType.java
index 4775311240..d0dee96ca4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/ScalarType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ScalarType.java
@@ -77,6 +77,8 @@ public class ScalarType extends Type {
 
     // Hive, mysql, sql server standard.
     public static final int MAX_PRECISION = 38;
+    public static final int MAX_DECIMALV2_PRECISION = 27;
+    public static final int MAX_DECIMALV2_SCALE = 9;
     public static final int MAX_DECIMAL32_PRECISION = 9;
     public static final int MAX_DECIMAL64_PRECISION = 18;
     public static final int MAX_DECIMAL128_PRECISION = 38;
@@ -996,8 +998,12 @@ public class ScalarType extends Type {
             return INVALID;
         }
 
+        if (t1.isDecimalV2() && t2.isDecimalV2()) {
+            return getAssignmentCompatibleDecimalV2Type(t1, t2);
+        }
+
         if (t1.isDecimalV2() || t2.isDecimalV2()) {
-            return DECIMALV2;
+            return MAX_DECIMALV2_TYPE;
         }
 
         PrimitiveType smallerType =
@@ -1018,9 +1024,19 @@ public class ScalarType extends Type {
             result = compatibilityMatrix[smallerType.ordinal()][largerType.ordinal()];
         }
         Preconditions.checkNotNull(result);
+        if (result == PrimitiveType.DECIMALV2) {
+            return Type.MAX_DECIMALV2_TYPE;
+        }
         return createType(result);
     }
 
+    public static ScalarType getAssignmentCompatibleDecimalV2Type(ScalarType t1, ScalarType t2) {
+        int targetPrecision = Math.max(t1.decimalPrecision(), t2.decimalPrecision());
+        int targetScale = Math.max(t1.decimalScale(), t2.decimalScale());
+        return ScalarType.createDecimalType(PrimitiveType.DECIMALV2,
+                targetPrecision, targetScale);
+    }
+
     /**
      * Returns true t1 can be implicitly cast to t2, false otherwise.
      * If strict is true, only consider casts that result in no loss of precision.
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java
index 467ba4208f..3f6fd5ef0e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Type.java
@@ -69,6 +69,9 @@ public abstract class Type {
     public static final ScalarType DEFAULT_DECIMALV2 = ScalarType.createDecimalType(PrimitiveType.DECIMALV2,
                     ScalarType.DEFAULT_PRECISION, ScalarType.DEFAULT_SCALE);
 
+    public static final ScalarType MAX_DECIMALV2_TYPE = ScalarType.createDecimalType(PrimitiveType.DECIMALV2,
+            ScalarType.MAX_DECIMALV2_PRECISION, ScalarType.MAX_DECIMALV2_SCALE);
+
     public static final ScalarType DEFAULT_DECIMAL32 =
             ScalarType.createDecimalType(PrimitiveType.DECIMAL32, ScalarType.MAX_DECIMAL32_PRECISION,
                     ScalarType.DEFAULT_SCALE);
@@ -121,7 +124,7 @@ public abstract class Type {
         numericTypes.addAll(integerTypes);
         numericTypes.add(FLOAT);
         numericTypes.add(DOUBLE);
-        numericTypes.add(DECIMALV2);
+        numericTypes.add(MAX_DECIMALV2_TYPE);
         numericTypes.add(DECIMAL32);
         numericTypes.add(DECIMAL64);
         numericTypes.add(DECIMAL128);
@@ -538,7 +541,7 @@ public abstract class Type {
             case DOUBLE:
                 return DOUBLE;
             case DECIMALV2:
-                return DECIMALV2;
+                return MAX_DECIMALV2_TYPE;
             case DECIMAL32:
                 return DECIMAL32;
             case DECIMAL64:
diff --git a/gensrc/script/gen_builtins_functions.py b/gensrc/script/gen_builtins_functions.py
index 1f9f2d7570..4dcda38162 100755
--- a/gensrc/script/gen_builtins_functions.py
+++ b/gensrc/script/gen_builtins_functions.py
@@ -113,6 +113,8 @@ def generate_fe_datatype(str_type):
         vec_type = str_type.split('_', 1);
         if len(vec_type) > 1 and vec_type[0] == "ARRAY":
             return "new ArrayType(" + generate_fe_datatype(vec_type[1]) + ")"
+    if str_type == "DECIMALV2":
+        return "Type.MAX_DECIMALV2_TYPE"
     return "Type." + str_type
 
 """
diff --git a/regression-test/data/datev2/tpcds_sf1_p1/sql/q58.out b/regression-test/data/datev2/tpcds_sf1_p1/sql/q58.out
index fc6b60f5f2..c83e736233 100644
--- a/regression-test/data/datev2/tpcds_sf1_p1/sql/q58.out
+++ b/regression-test/data/datev2/tpcds_sf1_p1/sql/q58.out
@@ -1,6 +1,6 @@
 -- This file is automatically generated. You should know what you did if you want to edit this
 -- !q58 --
-AAAAAAAACNGBAAAA	1900.15	11.1499568	1950.92	11.4478719	1829.52	10.7355046	1893.53
-AAAAAAAAIDOAAAAA	6605.22	11.5748247	6078.33	10.6515156	6338.25	11.106993	6340.6
-AAAAAAAAJMFCAAAA	3608.52	11.4503737	3590.47	11.3930983	3305.82	10.4898613	3501.603333333
+AAAAAAAACNGBAAAA	1900.150000000	11.14	1950.920000000	11.44	1829.520000000	10.73	1893.530000000
+AAAAAAAAIDOAAAAA	6605.220000000	11.57	6078.330000000	10.65	6338.250000000	11.10	6340.600000000
+AAAAAAAAJMFCAAAA	3608.520000000	11.45	3590.470000000	11.39	3305.820000000	10.48	3501.603333333
 
diff --git a/regression-test/data/datev2/tpcds_sf1_p1/sql/q83.out b/regression-test/data/datev2/tpcds_sf1_p1/sql/q83.out
index a4577094bc..821a2547c5 100644
--- a/regression-test/data/datev2/tpcds_sf1_p1/sql/q83.out
+++ b/regression-test/data/datev2/tpcds_sf1_p1/sql/q83.out
@@ -1,24 +1,24 @@
 -- This file is automatically generated. You should know what you did if you want to edit this
 -- !q83 --
-AAAAAAAAANECAAAA	34	18.579235	9	4.9180328	18	9.8360656	20.333333333
-AAAAAAAACAOBAAAA	27	11.6883117	38	16.4502165	12	5.1948052	25.666666667
-AAAAAAAACBNDAAAA	38	11.6207951	52	15.9021407	19	5.8103976	36.333333333
-AAAAAAAACCGAAAAA	30	15.1515152	18	9.0909091	18	9.0909091	22
-AAAAAAAACKBDAAAA	32	12.6984127	22	8.7301587	30	11.9047619	28
-AAAAAAAACOIBAAAA	31	8.0729167	29	7.5520833	68	17.7083333	42.666666667
-AAAAAAAAEBLDAAAA	9	3.7037037	34	13.9917695	38	15.6378601	27
-AAAAAAAAELFDAAAA	54	20.2247191	4	1.4981273	31	11.6104869	29.666666667
-AAAAAAAAFEBAAAAA	19	15.4471545	17	13.8211382	5	4.0650407	13.666666667
-AAAAAAAAFODDAAAA	30	11.4942529	7	2.6819923	50	19.1570881	29
-AAAAAAAAGLMCAAAA	37	26.8115942	3	2.173913	6	4.3478261	15.333333333
-AAAAAAAAHAGDAAAA	16	8.3333333	15	7.8125	33	17.1875	21.333333333
-AAAAAAAAHCDEAAAA	41	19.2488263	1	0.4694836	29	13.6150235	23.666666667
-AAAAAAAALAEBAAAA	19	6.959707	21	7.6923077	51	18.6813187	30.333333333
-AAAAAAAAMBDEAAAA	26	6.372549	66	16.1764706	44	10.7843137	45.333333333
-AAAAAAAAMBGBAAAA	5	5.0505051	1	1.010101	27	27.2727273	11
-AAAAAAAAMDODAAAA	17	10.8974359	32	20.5128205	3	1.9230769	17.333333333
-AAAAAAAAMOIAAAAA	84	25.4545455	12	3.6363636	14	4.2424242	36.666666667
-AAAAAAAANMAAAAAA	12	25	1	2.0833333	3	6.25	5.333333333
-AAAAAAAAODOAAAAA	5	4.3859649	3	2.6315789	30	26.3157895	12.666666667
-AAAAAAAAPJOCAAAA	29	15.3439153	21	11.1111111	13	6.8783069	21
+AAAAAAAAANECAAAA	34	18.57	9	4.91	18	9.83	20.333333333
+AAAAAAAACAOBAAAA	27	11.68	38	16.45	12	5.19	25.666666667
+AAAAAAAACBNDAAAA	38	11.62	52	15.90	19	5.81	36.333333333
+AAAAAAAACCGAAAAA	30	15.15	18	9.09	18	9.09	22.000000000
+AAAAAAAACKBDAAAA	32	12.69	22	8.73	30	11.90	28.000000000
+AAAAAAAACOIBAAAA	31	8.07	29	7.55	68	17.70	42.666666667
+AAAAAAAAEBLDAAAA	9	3.70	34	13.99	38	15.63	27.000000000
+AAAAAAAAELFDAAAA	54	20.22	4	1.49	31	11.61	29.666666667
+AAAAAAAAFEBAAAAA	19	15.44	17	13.82	5	4.06	13.666666667
+AAAAAAAAFODDAAAA	30	11.49	7	2.68	50	19.15	29.000000000
+AAAAAAAAGLMCAAAA	37	26.81	3	2.17	6	4.34	15.333333333
+AAAAAAAAHAGDAAAA	16	8.33	15	7.81	33	17.18	21.333333333
+AAAAAAAAHCDEAAAA	41	19.24	1	0.46	29	13.61	23.666666667
+AAAAAAAALAEBAAAA	19	6.95	21	7.69	51	18.68	30.333333333
+AAAAAAAAMBDEAAAA	26	6.37	66	16.17	44	10.78	45.333333333
+AAAAAAAAMBGBAAAA	5	5.05	1	1.01	27	27.27	11.000000000
+AAAAAAAAMDODAAAA	17	10.89	32	20.51	3	1.92	17.333333333
+AAAAAAAAMOIAAAAA	84	25.45	12	3.63	14	4.24	36.666666667
+AAAAAAAANMAAAAAA	12	25.00	1	2.08	3	6.25	5.333333333
+AAAAAAAAODOAAAAA	5	4.38	3	2.63	30	26.31	12.666666667
+AAAAAAAAPJOCAAAA	29	15.34	21	11.11	13	6.87	21.000000000
 
diff --git a/regression-test/data/load_p0/broker_load/test_array_load.out b/regression-test/data/load_p0/broker_load/test_array_load.out
index 68caec4f68..baff23e6f2 100644
--- a/regression-test/data/load_p0/broker_load/test_array_load.out
+++ b/regression-test/data/load_p0/broker_load/test_array_load.out
@@ -8,12 +8,12 @@
 100	[1, 2, 3]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c']	['hello', 'world']	[2022-07-13]	[2022-07-13 12:30:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[4, 5.5, 6.67]
 
 -- !select --
-1	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-2	[6, 7, 8, 9, 10]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-3	[]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['happy', 'birthday']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-4	[NULL]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-5	[NULL, NULL]	[32767, 32768, NULL]	[65534, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-100	[1, 2, 3]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c']	['hello', 'world']	[2022-07-13]	[2022-07-13 12:30:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[4, 5.5, 6.67]
+1	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1.000000, 1.200000, 1.300000]
+2	[6, 7, 8, 9, 10]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1.000000, 1.200000, 1.300000]
+3	[]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['happy', 'birthday']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1.000000, 1.200000, 1.300000]
+4	[NULL]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1.000000, 1.200000, 1.300000]
+5	[NULL, NULL]	[32767, 32768, NULL]	[65534, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1.000000, 1.200000, 1.300000]
+100	[1, 2, 3]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c']	['hello', 'world']	[2022-07-13]	[2022-07-13 12:30:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[4.000000, 5.500000, 6.670000]
 
 -- !select --
 1	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
@@ -24,82 +24,18 @@
 100	[1, 2, 3]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c']	['hello', 'world']	[2022-07-13]	[2022-07-13 12:30:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[4, 5.5, 6.67]
 
 -- !select --
-1	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-2	[6, 7, 8, 9, 10]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-3	[]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['happy', 'birthday']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-4	[NULL]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-5	[NULL, NULL]	[32767, 32768, NULL]	[65534, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-100	[1, 2, 3]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c']	['hello', 'world']	[2022-07-13]	[2022-07-13 12:30:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[4, 5.5, 6.67]
+1	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1.000000, 1.200000, 1.300000]
+2	[6, 7, 8, 9, 10]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1.000000, 1.200000, 1.300000]
+3	[]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['happy', 'birthday']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1.000000, 1.200000, 1.300000]
+4	[NULL]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1.000000, 1.200000, 1.300000]
+5	[NULL, NULL]	[32767, 32768, NULL]	[65534, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1.000000, 1.200000, 1.300000]
+100	[1, 2, 3]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c']	['hello', 'world']	[2022-07-13]	[2022-07-13 12:30:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[4.000000, 5.500000, 6.670000]
 
 -- !select --
-1	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01, 1992-02-02, 1993-03-03]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-2	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01, 1992-02-02, 1993-03-03]	\N	\N	\N	[1, NULL, 1.3]
+1	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01, 1992-02-02, 1993-03-03]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1.000000, 1.200000, 1.300000]
+2	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01, 1992-02-02, 1993-03-03]	\N	\N	\N	[1.000000, NULL, 1.300000]
 3	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N
 4	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N
 5	\N	\N	\N	\N	\N	\N	\N	\N	\N	\N
-100	[1, 2, 3]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c']	['hello', 'world']	[2022-07-13]	[2022-07-13 12:30:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[4, 5.5, 6.67]
-
--- !select --
-1	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-2	[6, 7, 8, 9, 10]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-3	[]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['happy', 'birthday']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-4	[NULL]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-5	[NULL, NULL]	[32767, 32768, NULL]	[65534, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-100	[1, 2, 3]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c']	['hello', 'world']	[2022-07-13]	[2022-07-13 12:30:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[4, 5.5, 6.67]
-
--- !select --
-1	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-2	[6, 7, 8, 9, 10]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-3	[]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['happy', 'birthday']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-4	[NULL]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-5	[NULL, NULL]	[32767, 32768, NULL]	[65534, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-100	[1, 2, 3]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c']	['hello', 'world']	[2022-07-13]	[2022-07-13 12:30:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[4, 5.5, 6.67]
-
--- !select --
-1	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-2	[6, 7, 8, 9, 10]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-3	[]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['happy', 'birthday']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-4	[NULL]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-5	[NULL, NULL]	[32767, 32768, NULL]	[65534, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-100	[1, 2, 3]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c']	['hello', 'world']	[2022-07-13]	[2022-07-13 12:30:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[4, 5.5, 6.67]
-
--- !select --
-1	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-2	[6, 7, 8, 9, 10]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-3	[]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['happy', 'birthday']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-4	[NULL]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-5	[NULL, NULL]	[32767, 32768, NULL]	[65534, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-100	[1, 2, 3]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c']	['hello', 'world']	[2022-07-13]	[2022-07-13 12:30:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[4, 5.5, 6.67]
-
--- !select --
-1	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-2	[6, 7, 8, 9, 10]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-3	[]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['happy', 'birthday']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-4	[NULL]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-5	[NULL, NULL]	[32767, 32768, NULL]	[65534, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-100	[1, 2, 3]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c']	['hello', 'world']	[2022-07-13]	[2022-07-13 12:30:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[4, 5.5, 6.67]
-
--- !select --
-1	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-2	[6, 7, 8, 9, 10]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-3	[]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['happy', 'birthday']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-4	[NULL]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-5	[NULL, NULL]	[32767, 32768, NULL]	[65534, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-100	[1, 2, 3]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c']	['hello', 'world']	[2022-07-13]	[2022-07-13 12:30:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[4, 5.5, 6.67]
-
--- !select --
-1	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-2	[6, 7, 8, 9, 10]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-3	[]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['happy', 'birthday']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-4	[NULL]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-5	[NULL, NULL]	[32767, 32768, NULL]	[65534, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-100	[1, 2, 3]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c']	['hello', 'world']	[2022-07-13]	[2022-07-13 12:30:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[4, 5.5, 6.67]
-
--- !select --
-1	[1, 2, 3, 4, 5]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-2	[6, 7, 8, 9, 10]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-3	[]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['happy', 'birthday']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-4	[NULL]	[32767, 32768, 32769]	[NULL, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-5	[NULL, NULL]	[32767, 32768, NULL]	[65534, NULL, 65536]	['a', 'b', 'c', 'd', 'e']	['hello', 'world']	[1991-01-01]	[1991-01-01 00:00:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[1, 1.2, 1.3]
-100	[1, 2, 3]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c']	['hello', 'world']	[2022-07-13]	[2022-07-13 12:30:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[4, 5.5, 6.67]
+100	[1, 2, 3]	[32767, 32768, 32769]	[65534, 65535, 65536]	['a', 'b', 'c']	['hello', 'world']	[2022-07-13]	[2022-07-13 12:30:00]	[0.33, 0.67]	[3.1415926, 0.878787878]	[4.000000, 5.500000, 6.670000]
 
diff --git a/regression-test/data/query/operator/test_arithmetic_operators.out b/regression-test/data/query/operator/test_arithmetic_operators.out
index c841b15c8c..60e54c7eef 100644
--- a/regression-test/data/query/operator/test_arithmetic_operators.out
+++ b/regression-test/data/query/operator/test_arithmetic_operators.out
@@ -5,29 +5,29 @@
 3	3670635	5536	10989	1
 
 -- !arith_op2 --
-1	2.0	123.123	12312300
-2	3.0	1243.5	124350000
-3	4.0	24453.325	2445332500
+1	2.0	123.123	12312300.000000000
+2	3.0	1243.500	124350000.000000000
+3	4.0	24453.325	2445332500.000000000
 
 -- !arith_op3 --
-1	123.123	244891.647
-2	1243.5	2469591
-3	24453.325	48637663.425
+1	123.123	244891.647000000
+2	1243.500	2469591.000000000
+3	24453.325	48637663.425000000
 
 -- !arith_op4 --
-1	123.123	12.3123	779.73798264	12596.337364196777	198.9
-2	1243.5	25203.258	981432.353193984	1567450.5	40252.248
-3	24453.325	1.930467742125E9	89352450.776970035	7267806.0	1.57021605E8
+1	123.123	12.3123	779.737982640	12596.337364196777	198.9
+2	1243.500	25203.258	981432.353193984	1567450.5	40252.248
+3	24453.325	1.930467742125E9	89352450.776970036	7267806.0	1.57021605E8
 
 -- !arith_op5 --
-1	12.3123	0.010000000000000002	0.633300019
-2	124.35	2.0268	78.924998246
-3	2445.3325	7894.5	365.400005018
+1	12.312300000	0.010000000000000002	0.633300019
+2	124.350000000	2.0268	78.924998246
+3	2445.332500000	7894.5	365.400005018
 
 -- !arith_op6 --
-1	-198.9	-100.1	-1101190.2	-12.3123	-0.010000000000000002	-0.633300019
-2	-198.6	-100.1	-1101190.3	-124.35	-2.0268	-78.924998246
-3	-198.9	-100.2	-1101190.5	-2445.3325	-7894.5	-365.400005018
+1	-198.900000000	-100.100000000	-1101190.200000000	-12.312300000	-0.010000000000000002	-0.633300019
+2	-198.600000000	-100.100000000	-1101190.300000000	-124.350000000	-2.0268	-78.924998246
+3	-198.900000000	-100.200000000	-1101190.500000000	-2445.332500000	-7894.5	-365.400005018
 
 -- !arith_op7 --
 1	1.1356092352936706E19	9.2233720368547763E17	5.8411616798251155E19
@@ -35,9 +35,9 @@
 3	2.2554211401312182E21	7.281391054495003E23	3.370220142266735E22
 
 -- !arith_op8 --
-1	2.1564781210736683E-16	1.08528637465799E-16	1.1939128071597471E-12	0	1.0842021724855045E-20	6.866252556874047E-19
-2	2.1532255145562118E-16	1.08528637465799E-16	1.1939129155799644E-12	0	2.1974609631936205E-18	8.557065646341844E-17
-3	2.1564781210736683E-16	1.0863705768304754E-16	1.1939131324203989E-12	0	8.559234050686815E-15	3.961674738262033E-16
+1	2.1564781210736683E-16	1.08528637465799E-16	1.1939128071597471E-12	0E-9	1.0842021724855045E-20	6.866252556874047E-19
+2	2.1532255145562118E-16	1.08528637465799E-16	1.1939129155799644E-12	0E-9	2.1974609631936205E-18	8.557065646341844E-17
+3	2.1564781210736683E-16	1.0863705768304754E-16	1.1939131324203989E-12	0E-9	8.559234050686815E-15	3.961674738262033E-16
 
 -- !arith_op9 --
 1	9.2233720368547888E16	9.223372036854776E18	9.223372036854776E18
@@ -50,41 +50,41 @@
 3	-9.2233720368523312E16	-9.223372036854697E18	-9.2233720368547717E18
 
 -- !arith_op11 --
-1	123123000	100000.00000000001	6333000.183105469
-2	1243500000	2.0268E7	7.8925E8
-3	24453325000	7.8945E10	3.654E9
+1	123123000.000000000	100000.00000000001	6333000.183105469
+2	1243500000.000000000	2.0268E7	7.8925E8
+3	24453325000.000000000	7.8945E10	3.654E9
 
 -- !arith_op12 --
-1	0.1	198.9	100.1	1101190.2	12.3123	0.010000000000000002	0.633300019
-2	0.2	198.6	100.1	1101190.3	124.35	2.0268	78.924998246
-3	0.3	198.9	100.2	1101190.5	2445.3325	7894.5	365.400005018
+1	0.100000000	198.900000000	100.100000000	1101190.200000000	12.312300000	0.010000000000000002	0.633300019
+2	0.200000000	198.600000000	100.100000000	1101190.300000000	124.350000000	2.0268	78.924998246
+3	0.300000000	198.900000000	100.200000000	1101190.500000000	2445.332500000	7894.5	365.400005018
 
 -- !arith_op13 --
-1	0.1	198.9	100.1	1101190.2	12.3123	0.01	0.6333000183105468
-2	0.2	198.6	100.1	1101190.3	124.35	2.0268	78.925
-3	0.3	198.9	100.2	1101190.5	2445.3325	7894.5	365.4
+1	0.1	198.9	100.1	1101190.2	12.312300000	0.01	0.6333000183105468
+2	0.2	198.6	100.1	1101190.3	124.350000000	2.0268	78.925
+3	0.3	198.9	100.2	1101190.5	2445.332500000	7894.5	365.4
 
 -- !arith_op14 --
-1	0.9	1988.9	1000.9	11011901.9	123.023	0.0	6.233000192
-2	1.9	1985.9	1000.9	11011902.9	1243.4	20.168	789.149982464
-3	2.9	1988.9	1001.9	11011904.9	24453.225	78944.9	3653.900050176
+1	0.900000000	1988.900000000	1000.900000000	11011901.900000000	123.023000000	0.0	6.233000192
+2	1.900000000	1985.900000000	1000.900000000	11011902.900000000	1243.400000000	20.168	789.149982464
+3	2.900000000	1988.900000000	1001.900000000	11011904.900000000	24453.225000000	78944.9	3653.900050176
 
 -- !arith_op15 --
-1	1.1	1989.1	1001.1	11011902.1	123.223	0.2	6.433000192
-2	2.1	1986.1	1001.1	11011903.1	1243.6	20.368000000000002	789.349982464
-3	3.1	1989.1	1002.1	11011905.1	24453.425	78945.1	3654.100050176
+1	1.100000000	1989.100000000	1001.100000000	11011902.100000000	123.223000000	0.2	6.433000192
+2	2.100000000	1986.100000000	1001.100000000	11011903.100000000	1243.600000000	20.368000000000002	789.349982464
+3	3.100000000	1989.100000000	1002.100000000	11011905.100000000	24453.425000000	78945.1	3654.100050176
 
 -- !arith_op16 --
-11	1999	1002.6	11011902	117.123	-234.56	6.333000183105469
-12	1996	1002.6	11011903	1237.5	-214.392	789.25
-13	1999	1003.6	11011905	24447.325	78710.34	3654.0
+11	1999.000000000	1002.600000000	11011902	117.123000000	-234.56	6.333000183105469
+12	1996.000000000	1002.600000000	11011903	1237.500000000	-214.392	789.25
+13	1999.000000000	1003.600000000	11011905	24447.325000000	78710.34	3654.0
 
 -- !arith_op17 --
 
 -- !arith_op18 --
-245136538.647
-4944121182
-146204816255.55
+245136538.647000000
+4944121182.000000000
+146204816255.550000000
 
 -- !arith_op19 --
 1.5524497441372922E8
@@ -112,9 +112,9 @@
 3006	1005	-999	0.0029940119760479044
 
 -- !arith_op21 --
-123.123	124.123	-122.123	0.008121959
-2487	1245.5	-1241.5	0.001608363
-73359.975	24456.325	-24450.325	0.000122683
+123.123000000	124.123000000	-122.123000000	0.008121959
+2487.000000000	1245.500000000	-1241.500000000	0.001608363
+73359.975000000	24456.325000000	-24450.325000000	0.000122683
 
 -- !arith_op21 --
 0.1	1.1	0.9	10.0
@@ -142,9 +142,9 @@
 1992978	2991	987	1.9850299401197604
 
 -- !arith_op21 --
-244891.647	2112.123	1865.877	16.15457713
-2469591	3229.5	742.5	1.597104946
-48637663.425	26442.325	-22464.325	0.081338632
+244891.647000000	2112.123000000	1865.877000000	16.154577130
+2469591.000000000	3229.500000000	742.500000000	1.597104946
+48637663.425000000	26442.325000000	-22464.325000000	0.081338632
 
 -- !arith_op21 --
 198.9	1989.1	1988.9	19890.0
@@ -172,9 +172,9 @@
 1004004	2004	0	1.0
 
 -- !arith_op21 --
-123246.123	1124.123	877.877	8.130081301
-1244743.5	2244.5	-242.5	0.804985927
-24502231.65	25455.325	-23451.325	0.040976023
+123246.123000000	1124.123000000	877.877000000	8.130081301
+1244743.500000000	2244.500000000	-242.500000000	0.804985927
+24502231.650000000	25455.325000000	-23451.325000000	0.040976023
 
 -- !arith_op21 --
 100.10000000000001	1001.1	1000.9	10010.0
@@ -187,24 +187,24 @@
 3661308.0	4656.0	-2652.0	0.2742200328407225
 
 -- !arith_op21 --
-123.123	124.123	122.123	123.123
-2487	1245.5	1241.5	621.75
-73359.975	24456.325	24450.325	8151.108333333
+123.123000000	124.123000000	122.123000000	123.123000000
+2487.000000000	1245.500000000	1241.500000000	621.750000000
+73359.975000000	24456.325000000	24450.325000000	8151.108333333
 
 -- !arith_op21 --
-244891.647	2112.123	-1865.877	0.061901961
-2469591	3229.5	-742.5	0.626132931
-48637663.425	26442.325	22464.325	12.294281046
+244891.647000000	2112.123000000	-1865.877000000	0.061901961
+2469591.000000000	3229.500000000	-742.500000000	0.626132931
+48637663.425000000	26442.325000000	22464.325000000	12.294281046
 
 -- !arith_op21 --
-123246.123	1124.123	-877.877	0.123
-1244743.5	2244.5	242.5	1.242257742
-24502231.65	25455.325	23451.325	24.404515968
+123246.123000000	1124.123000000	-877.877000000	0.123000000
+1244743.500000000	2244.500000000	242.500000000	1.242257742
+24502231.650000000	25455.325000000	23451.325000000	24.404515968
 
 -- !arith_op21 --
-15159.273129	246.246	0	1
-1546292.25	2487	0	1
-597965103.555625	48906.65	0	1
+15159.273129000	246.246000000	0E-9	1.000000000
+1546292.250000000	2487.000000000	0E-9	1.000000000
+597965103.555625000	48906.650000000	0E-9	1.000000000
 
 -- !arith_op21 --
 12.3123	123.223	123.02300000000001	1231.23
@@ -212,9 +212,9 @@
 1.930467742125E9	103398.325	-54491.675	0.30975140920894295
 
 -- !arith_op21 --
-779.73798264	129.456000192	116.789999808	19.44149635878031
+779.737982640	129.456000192	116.789999808	19.44149635878031
 981432.353193984	2032.749982464	454.250017536	1.5755464048146974
-89352450.776970035	28107.325050176	20799.324949824	6.692207170224412
+89352450.776970036	28107.325050176	20799.324949824	6.692207170224412
 
 -- !arith_op21 --
 0.1	1.1	-0.9	0.1
@@ -262,9 +262,9 @@
 3661308.0	4656.0	2652.0	3.6467065868263475
 
 -- !arith_op21 --
-779.73798264	129.456000192	-116.789999808	0.051436369996714415
+779.737982640	129.456000192	-116.789999808	0.051436369996714415
 981432.353193984	2032.749982464	-454.250017536	0.6347004422999598
-89352450.776970035	28107.325050176	-20799.324949824	0.14942753183871724
+89352450.776970036	28107.325050176	-20799.324949824	0.14942753183871724
 
 -- !arith_op21 --
 0.6333000183105469	6.433000183105468	6.233000183105469	63.33000183105469
@@ -277,17 +277,17 @@
 1.3351716E7	7308.0	0.0	1.0
 
 -- !arith_op22 --
-1987.21
-1990.21
-1990.21
+1987.210000000
+1990.210000000
+1990.210000000
 
 -- !arith_op23 --
-124.333
-1244.71
-24454.535
+124.333000000
+1244.710000000
+24454.535000000
 
 -- !arith_op24 --
-2.31
+2.310000000
 
 -- !arith_op25 --
 \N	\N	\N	\N
diff --git a/regression-test/data/query/operator/test_logical_operators.out b/regression-test/data/query/operator/test_logical_operators.out
index 17f680e079..fe5e12b481 100644
--- a/regression-test/data/query/operator/test_logical_operators.out
+++ b/regression-test/data/query/operator/test_logical_operators.out
@@ -15,14 +15,14 @@
 \N
 
 -- !logical_op4 --
-1
-2
-0.9
+1.000000000
+2.000000000
+0.900000000
 
 -- !logical_op5 --
-1
-2
-0.9
+1.000000000
+2.000000000
+0.900000000
 
 -- !logical_op1 --
 1
@@ -40,14 +40,14 @@
 \N
 
 -- !logical_op4 --
-0.3
-1.5
-0.3
+0.300000000
+1.500000000
+0.300000000
 
 -- !logical_op5 --
-0.3
-1.5
-0.3
+0.300000000
+1.500000000
+0.300000000
 
 -- !logical_op1 --
 1
@@ -65,14 +65,14 @@
 \N
 
 -- !logical_op4 --
-1.4
-1.4
-0.3
+1.400000000
+1.400000000
+0.300000000
 
 -- !logical_op5 --
-1.4
-1.4
-0.3
+1.400000000
+1.400000000
+0.300000000
 
 -- !logical_op1 --
 0
@@ -90,34 +90,34 @@
 \N
 
 -- !logical_op4 --
-1.8
-0.7
-0.6
+1.800000000
+0.700000000
+0.600000000
 
 -- !logical_op5 --
-1.8
-0.7
-0.6
+1.800000000
+0.700000000
+0.600000000
 
 -- !logical_op1 --
-1.123
-1.5
-1.325
+1.123000000
+1.500000000
+1.325000000
 
 -- !logical_op2 --
-1.123
-1.5
-1.325
+1.123000000
+1.500000000
+1.325000000
 
 -- !logical_op4 --
-1.323
-0.3
-0.925
+1.323000000
+0.300000000
+0.925000000
 
 -- !logical_op5 --
-1.323
-0.3
-0.925
+1.323000000
+0.300000000
+0.925000000
 
 -- !logical_op6 --
 0
diff --git a/regression-test/data/query_p0/union/test_union.out b/regression-test/data/query_p0/union/test_union.out
index 29a892fb78..302eb78140 100644
--- a/regression-test/data/query_p0/union/test_union.out
+++ b/regression-test/data/query_p0/union/test_union.out
@@ -1,74 +1,44 @@
 -- This file is automatically generated. You should know what you did if you want to edit this
--- !select --
-\N	\N
-1	1989
-10	1991
-11	1989
-12	32767
-13	-32767
-14	255
-15	1992
-1986	1001
-1989	1001
-1989	1002
-2	1986
-3	1989
-4	1991
-5	1985
-6	32767
-7	-32767
-8	255
-9	1991
-
--- !select --
-1985	1
-1986	2
-1989	2
-1991	1
-1992	1
-255	1
-32767	1
-
 -- !union1 --
-123.123	0.1	true
+123.123000000	0.1	true
 
 -- !union2 --
-123.123	0.1	true	world
+123.123000000	0.1	true	world
 
 -- !union3 --
 
 -- !union4 --
--654.654	0.0
--0.123	987456.123
-0	3.141592653
-0.666	-987.001
-123.123	0.1
-243.325	-0.0
-604587	0.1
+-654.654000000	0.0
+-0.123000000	987456.123
+0E-9	3.141592653
+0.666000000	-987.001
+123.123000000	0.1
+243.325000000	-0.0
+604587.000000000	0.1
 
 -- !union5 --
 false	1	1989	1001	11011902	123.123	true	1989-03-21	1989-03-21T13:00	wangjuoo4	0.1	6.333	string12345	170141183460469231731687303715884105727
-false	2	1986	1001	11011903	1243.5	false	1901-12-31	1989-03-21T13:00	wangynnsf	20.268	789.25	string12345	-170141183460469231731687303715884105727
+false	2	1986	1001	11011903	1243.500	false	1901-12-31	1989-03-21T13:00	wangynnsf	20.268	789.25	string12345	-170141183460469231731687303715884105727
 false	3	1989	1002	11011905	24453.325	false	2012-03-14	2000-01-01T00:00	yunlj8@nk	78945.0	3654.0	string12345	0
 
 -- !union6 --
 false	1	1989	1001	11011902	123.123	true	1989-03-21	1989-03-21T13:00	wangjuoo4	0.1	6.333	string12345	170141183460469231731687303715884105727
 false	1	1989	1001	11011902	123.123	true	1989-03-21	1989-03-21T13:00	wangjuoo4	0.1	6.333	string12345	170141183460469231731687303715884105727
-false	2	1986	1001	11011903	1243.5	false	1901-12-31	1989-03-21T13:00	wangynnsf	20.268	789.25	string12345	-170141183460469231731687303715884105727
-false	2	1986	1001	11011903	1243.5	false	1901-12-31	1989-03-21T13:00	wangynnsf	20.268	789.25	string12345	-170141183460469231731687303715884105727
+false	2	1986	1001	11011903	1243.500	false	1901-12-31	1989-03-21T13:00	wangynnsf	20.268	789.25	string12345	-170141183460469231731687303715884105727
+false	2	1986	1001	11011903	1243.500	false	1901-12-31	1989-03-21T13:00	wangynnsf	20.268	789.25	string12345	-170141183460469231731687303715884105727
 
 -- !union7 --
 false	1	1989	1001	11011902	123.123	true	1989-03-21	1989-03-21T13:00	wangjuoo4	0.1	6.333	string12345	170141183460469231731687303715884105727
-false	2	1986	1001	11011903	1243.5	false	1901-12-31	1989-03-21T13:00	wangynnsf	20.268	789.25	string12345	-170141183460469231731687303715884105727
+false	2	1986	1001	11011903	1243.500	false	1901-12-31	1989-03-21T13:00	wangynnsf	20.268	789.25	string12345	-170141183460469231731687303715884105727
 false	3	1989	1002	11011905	24453.325	false	2012-03-14	2000-01-01T00:00	yunlj8@nk	78945.0	3654.0	string12345	0
 
 -- !union8 --
 1	1989	1001	11011902	123.123	true	wangjuoo4	0.1	6.333	1989-03-21	1989-03-21T13:00
 1	1989	1001	11011902	123.123	true	wangjuoo4	0.1	6.333	1989-03-21	1989-03-21T13:00
 1	1989	1001	11011902	123.123	true	wangjuoo4	0.1	6.333	1989-03-21	1989-03-21T13:00
-2	1986	1001	11011903	1243.5	false	wangynnsf	20.268	789.25	1901-12-31	1989-03-21T13:00
-2	1986	1001	11011903	1243.5	false	wangynnsf	20.268	789.25	1901-12-31	1989-03-21T13:00
-2	1986	1001	11011903	1243.5	false	wangynnsf	20.268	789.25	1901-12-31	1989-03-21T13:00
+2	1986	1001	11011903	1243.500	false	wangynnsf	20.268	789.25	1901-12-31	1989-03-21T13:00
+2	1986	1001	11011903	1243.500	false	wangynnsf	20.268	789.25	1901-12-31	1989-03-21T13:00
+2	1986	1001	11011903	1243.500	false	wangynnsf	20.268	789.25	1901-12-31	1989-03-21T13:00
 3	1989	1002	11011905	24453.325	false	yunlj8@nk	78945.0	3654.0	2012-03-14	2000-01-01T00:00
 3	1989	1002	11011905	24453.325	false	yunlj8@nk	78945.0	3654.0	2012-03-14	2000-01-01T00:00
 3	1989	1002	11011905	24453.325	false	yunlj8@nk	78945.0	3654.0	2012-03-14	2000-01-01T00:00
@@ -76,8 +46,8 @@ false	3	1989	1002	11011905	24453.325	false	2012-03-14	2000-01-01T00:00	yunlj8@nk
 4	1991	3021	-11011907	243243.325	false	yanvjldjlll	2.06	-0.001	3124-10-10	2015-03-13T10:30
 5	1985	5014	-11011903	243.325	true	du3lnvl	-0.0	-365.0	2015-01-01	2015-03-13T12:36:38
 5	1985	5014	-11011903	243.325	true	du3lnvl	-0.0	-365.0	2015-01-01	2015-03-13T12:36:38
-6	32767	3021	123456	604587	true	yanavnd	0.1	80699.0	2014-11-11	2015-03-13T12:36:38
-6	32767	3021	123456	604587	true	yanavnd	0.1	80699.0	2014-11-11	2015-03-13T12:36:38
+6	32767	3021	123456	604587.000	true	yanavnd	0.1	80699.0	2014-11-11	2015-03-13T12:36:38
+6	32767	3021	123456	604587.000	true	yanavnd	0.1	80699.0	2014-11-11	2015-03-13T12:36:38
 7	-32767	1002	7210457	3.141	false	jiw3n4	0.0	6058.0	1988-03-21	1901-01-01T00:00
 8	255	2147483647	11011920	-0.123	true	wangjuoo5	987456.123	12.14	1989-03-21	9999-11-11T12:12
 8	255	2147483647	11011920	-0.123	true	wangjuoo5	987456.123	12.14	1989-03-21	9999-11-11T12:12
@@ -88,10 +58,10 @@ false	3	1989	1002	11011905	24453.325	false	2012-03-14	2000-01-01T00:00	yunlj8@nk
 11	1989	25699	-9223372036854775807	0.666	true	yunlj8@nk	-987.001	4.336	2015-04-02	1989-03-21T13:11
 12	32767	-2147483647	9223372036854775807	243.325	false	lifsno	-564.898	3.1415927	1991-08-11	2013-04-02T15:16:52
 13	-32767	2147483647	-9223372036854775807	100.001	false	wenlsfnl	123.456	3.1415927	2015-04-02	2015-04-02T00:00
-14	255	103	11011902	0	false	 	3.141592654	2.036	2015-04-02	2015-04-02T00:00
-14	255	103	11011902	0	false	 	3.141592654	2.036	2015-04-02	2015-04-02T00:00
-15	1992	3021	11011920	0	true		3.141592653	20.456	9999-12-12	2015-04-02T00:00
-15	1992	3021	11011920	0	true		3.141592653	20.456	9999-12-12	2015-04-02T00:00
+14	255	103	11011902	0.000	false	 	3.141592654	2.036	2015-04-02	2015-04-02T00:00
+14	255	103	11011902	0.000	false	 	3.141592654	2.036	2015-04-02	2015-04-02T00:00
+15	1992	3021	11011920	0.000	true		3.141592653	20.456	9999-12-12	2015-04-02T00:00
+15	1992	3021	11011920	0.000	true		3.141592653	20.456	9999-12-12	2015-04-02T00:00
 
 -- !union9 --
 1	1989	1001	11011902	123.123	true	wangjuoo4	0.1	6.333	1989-03-21	1989-03-21T13:00
@@ -100,11 +70,11 @@ false	3	1989	1002	11011905	24453.325	false	2012-03-14	2000-01-01T00:00	yunlj8@nk
 
 -- !union10 --
 1	1989	1001	11011902	123.123	true	wangjuoo4	0.1	6.333	1989-03-21	1989-03-21T13:00
-2	1986	1001	11011903	1243.5	false	wangynnsf	20.268	789.25	1901-12-31	1989-03-21T13:00
+2	1986	1001	11011903	1243.500	false	wangynnsf	20.268	789.25	1901-12-31	1989-03-21T13:00
 3	1989	1002	11011905	24453.325	false	yunlj8@nk	78945.0	3654.0	2012-03-14	2000-01-01T00:00
 4	1991	3021	-11011907	243243.325	false	yanvjldjlll	2.06	-0.001	3124-10-10	2015-03-13T10:30
 5	1985	5014	-11011903	243.325	true	du3lnvl	-0.0	-365.0	2015-01-01	2015-03-13T12:36:38
-6	32767	3021	123456	604587	true	yanavnd	0.1	80699.0	2014-11-11	2015-03-13T12:36:38
+6	32767	3021	123456	604587.000	true	yanavnd	0.1	80699.0	2014-11-11	2015-03-13T12:36:38
 7	-32767	1002	7210457	3.141	false	jiw3n4	0.0	6058.0	1988-03-21	1901-01-01T00:00
 8	255	2147483647	11011920	-0.123	true	wangjuoo5	987456.123	12.14	1989-03-21	9999-11-11T12:12
 9	1991	-2147483647	11011902	-654.654	true	wangjuoo4	0.0	69.123	1991-08-11	1989-03-21T13:11
@@ -112,22 +82,22 @@ false	3	1989	1002	11011905	24453.325	false	2012-03-14	2000-01-01T00:00	yunlj8@nk
 11	1989	25699	-9223372036854775807	0.666	true	yunlj8@nk	-987.001	4.336	2015-04-02	1989-03-21T13:11
 12	32767	-2147483647	9223372036854775807	243.325	false	lifsno	-564.898	3.1415927	1991-08-11	2013-04-02T15:16:52
 13	-32767	2147483647	-9223372036854775807	100.001	false	wenlsfnl	123.456	3.1415927	2015-04-02	2015-04-02T00:00
-14	255	103	11011902	0	false	 	3.141592654	2.036	2015-04-02	2015-04-02T00:00
-15	1992	3021	11011920	0	true		3.141592653	20.456	9999-12-12	2015-04-02T00:00
+14	255	103	11011902	0.000	false	 	3.141592654	2.036	2015-04-02	2015-04-02T00:00
+15	1992	3021	11011920	0.000	true		3.141592653	20.456	9999-12-12	2015-04-02T00:00
 
 -- !union11 --
 1	1989	1001	11011902	123.123	true	wangjuoo4	0.1	6.333	1989-03-21	1989-03-21T13:00
 1	1989	1001	11011902	123.123	true	wangjuoo4	0.1	6.333	1989-03-21	1989-03-21T13:00
-2	1986	1001	11011903	1243.5	false	wangynnsf	20.268	789.25	1901-12-31	1989-03-21T13:00
-2	1986	1001	11011903	1243.5	false	wangynnsf	20.268	789.25	1901-12-31	1989-03-21T13:00
+2	1986	1001	11011903	1243.500	false	wangynnsf	20.268	789.25	1901-12-31	1989-03-21T13:00
+2	1986	1001	11011903	1243.500	false	wangynnsf	20.268	789.25	1901-12-31	1989-03-21T13:00
 3	1989	1002	11011905	24453.325	false	yunlj8@nk	78945.0	3654.0	2012-03-14	2000-01-01T00:00
 3	1989	1002	11011905	24453.325	false	yunlj8@nk	78945.0	3654.0	2012-03-14	2000-01-01T00:00
 4	1991	3021	-11011907	243243.325	false	yanvjldjlll	2.06	-0.001	3124-10-10	2015-03-13T10:30
 4	1991	3021	-11011907	243243.325	false	yanvjldjlll	2.06	-0.001	3124-10-10	2015-03-13T10:30
 5	1985	5014	-11011903	243.325	true	du3lnvl	-0.0	-365.0	2015-01-01	2015-03-13T12:36:38
 5	1985	5014	-11011903	243.325	true	du3lnvl	-0.0	-365.0	2015-01-01	2015-03-13T12:36:38
-6	32767	3021	123456	604587	true	yanavnd	0.1	80699.0	2014-11-11	2015-03-13T12:36:38
-6	32767	3021	123456	604587	true	yanavnd	0.1	80699.0	2014-11-11	2015-03-13T12:36:38
+6	32767	3021	123456	604587.000	true	yanavnd	0.1	80699.0	2014-11-11	2015-03-13T12:36:38
+6	32767	3021	123456	604587.000	true	yanavnd	0.1	80699.0	2014-11-11	2015-03-13T12:36:38
 7	-32767	1002	7210457	3.141	false	jiw3n4	0.0	6058.0	1988-03-21	1901-01-01T00:00
 8	255	2147483647	11011920	-0.123	true	wangjuoo5	987456.123	12.14	1989-03-21	9999-11-11T12:12
 8	255	2147483647	11011920	-0.123	true	wangjuoo5	987456.123	12.14	1989-03-21	9999-11-11T12:12
@@ -138,10 +108,10 @@ false	3	1989	1002	11011905	24453.325	false	2012-03-14	2000-01-01T00:00	yunlj8@nk
 11	1989	25699	-9223372036854775807	0.666	true	yunlj8@nk	-987.001	4.336	2015-04-02	1989-03-21T13:11
 12	32767	-2147483647	9223372036854775807	243.325	false	lifsno	-564.898	3.1415927	1991-08-11	2013-04-02T15:16:52
 13	-32767	2147483647	-9223372036854775807	100.001	false	wenlsfnl	123.456	3.1415927	2015-04-02	2015-04-02T00:00
-14	255	103	11011902	0	false	 	3.141592654	2.036	2015-04-02	2015-04-02T00:00
-14	255	103	11011902	0	false	 	3.141592654	2.036	2015-04-02	2015-04-02T00:00
-15	1992	3021	11011920	0	true		3.141592653	20.456	9999-12-12	2015-04-02T00:00
-15	1992	3021	11011920	0	true		3.141592653	20.456	9999-12-12	2015-04-02T00:00
+14	255	103	11011902	0.000	false	 	3.141592654	2.036	2015-04-02	2015-04-02T00:00
+14	255	103	11011902	0.000	false	 	3.141592654	2.036	2015-04-02	2015-04-02T00:00
+15	1992	3021	11011920	0.000	true		3.141592653	20.456	9999-12-12	2015-04-02T00:00
+15	1992	3021	11011920	0.000	true		3.141592653	20.456	9999-12-12	2015-04-02T00:00
 
 -- !union12 --
 1	1989	1001	11011902	123.123	true	wangjuoo4	0.1	6.333	1989-03-21	1989-03-21T13:00
@@ -159,42 +129,42 @@ false	3	1989	1002	11011905	24453.325	false	2012-03-14	2000-01-01T00:00	yunlj8@nk
 2	3
 
 -- !union17 --
-1	a	\N	10
-1	a	\N	10
-2	b	\N	20
+1	a	\N	10.0
+1	a	\N	10.0
+2	b	\N	20.0
 
 -- !union18 --
 16
 
 -- !union19 --
-1	123.123	true	wangjuoo4
-10	10	hello	world
-20	20	wangjuoo4	beautiful
+1	123.123000000	true	wangjuoo4
+10	10.000000000	hello	world
+20	20.000000000	wangjuoo4	beautiful
 255	3.141592654	false	 
-255	987456.123	true	wangjuoo5
-1985	0	true	du3lnvl
-1986	20.268	false	wangynnsf
-1989	-987.001	true	yunlj8@nk
-1989	0.1	true	wangjuoo4
-1989	78945	false	yunlj8@nk
-1991	-123456.54	false	wangynnsf
-1991	0	true	wangjuoo4
-1991	2.06	false	yanvjldjlll
+255	987456.123000000	true	wangjuoo5
+1985	0E-9	true	du3lnvl
+1986	20.268000000	false	wangynnsf
+1989	-987.001000000	true	yunlj8@nk
+1989	0.100000000	true	wangjuoo4
+1989	78945.000000000	false	yunlj8@nk
+1991	-123456.540000000	false	wangynnsf
+1991	0E-9	true	wangjuoo4
+1991	2.060000000	false	yanvjldjlll
 1992	3.141592653	true	
-32767	-564.898	false	lifsno
-32767	0.1	true	yanavnd
+32767	-564.898000000	false	lifsno
+32767	0.100000000	true	yanavnd
 
 -- !union20 --
 1	1989	1001	11011902	123.123
-2	1986	1001	11011903	1243.5
+2	1986	1001	11011903	1243.500
 3	1989	1002	11011905	24453.325
 4	1991	3021	-11011907	243243.325
 
 -- !union21 --
 1	1989	1001	11011902	123.123
 1	1989	1001	11011902	123.123
-2	1986	1001	11011903	1243.5
-2	1986	1001	11011903	1243.5
+2	1986	1001	11011903	1243.500
+2	1986	1001	11011903	1243.500
 3	1989	1002	11011905	24453.325
 3	1989	1002	11011905	24453.325
 4	1991	3021	-11011907	243243.325
@@ -224,66 +194,66 @@ false	3	1989	1002	11011905	24453.325	false	2012-03-14	2000-01-01T00:00	yunlj8@nk
 5	true	du3lnvl	-0.0	-365.0	2015-01-01
 
 -- !union24 --
-1	2	3	4	3.14	hello	world	0	1.1	1989-03-21	1989-03-21T13:00
-1	1989	1001	11011902	123.123	true	wangjuoo4	0.1	6.333000192	1989-03-21	1989-03-21T13:00
-1	1989	1001	11011902	123.123	true	wangjuoo4	0.1	6.333000192	1989-03-21	1989-03-21T13:00
-2	1986	1001	11011903	1243.5	false	wangynnsf	20.268	789.249982464	1901-12-31	1989-03-21T13:00
-2	1986	1001	11011903	1243.5	false	wangynnsf	20.268	789.249982464	1901-12-31	1989-03-21T13:00
-3	1989	1002	11011905	24453.325	false	yunlj8@nk	78945	3654.000050176	2012-03-14	2000-01-01T00:00
-3	1989	1002	11011905	24453.325	false	yunlj8@nk	78945	3654.000050176	2012-03-14	2000-01-01T00:00
-4	1991	3021	-11011907	243243.325	false	yanvjldjlll	2.06	-0.001	3124-10-10	2015-03-13T10:30
-5	1985	5014	-11011903	243.325	true	du3lnvl	0	-364.999999488	2015-01-01	2015-03-13T12:36:38
-6	32767	3021	123456	604587	true	yanavnd	0.1	80698.99616256	2014-11-11	2015-03-13T12:36:38
-7	-32767	1002	7210457	3.141	false	jiw3n4	0	6057.999990784	1988-03-21	1901-01-01T00:00
-8	255	2147483647	11011920	-0.123	true	wangjuoo5	987456.123	12.140000256	1989-03-21	9999-11-11T12:12
-10	1991	5014	9223372036854775807	-258.369	false	wangynnsf	-123456.54	0.235	2015-04-02	2013-04-02T15:16:52
-11	1989	25699	-9223372036854775807	0.666	true	yunlj8@nk	-987.001	4.336	2015-04-02	1989-03-21T13:11
-13	-32767	2147483647	-9223372036854775807	100.001	false	wenlsfnl	123.456	3.141592832	2015-04-02	2015-04-02T00:00
-14	255	103	11011902	0	false	 	3.141592654	2.036	2015-04-02	2015-04-02T00:00
-15	1992	3021	11011920	0	true		3.141592653	20.455999488	9999-12-12	2015-04-02T00:00
+1	2	3	4	3.140	hello	world	0E-9	1.100000000	1989-03-21	1989-03-21T13:00
+1	1989	1001	11011902	123.123	true	wangjuoo4	0.100000000	6.333000192	1989-03-21	1989-03-21T13:00
+1	1989	1001	11011902	123.123	true	wangjuoo4	0.100000000	6.333000192	1989-03-21	1989-03-21T13:00
+2	1986	1001	11011903	1243.500	false	wangynnsf	20.268000000	789.249982464	1901-12-31	1989-03-21T13:00
+2	1986	1001	11011903	1243.500	false	wangynnsf	20.268000000	789.249982464	1901-12-31	1989-03-21T13:00
+3	1989	1002	11011905	24453.325	false	yunlj8@nk	78945.000000000	3654.000050176	2012-03-14	2000-01-01T00:00
+3	1989	1002	11011905	24453.325	false	yunlj8@nk	78945.000000000	3654.000050176	2012-03-14	2000-01-01T00:00
+4	1991	3021	-11011907	243243.325	false	yanvjldjlll	2.060000000	-0.001000000	3124-10-10	2015-03-13T10:30
+5	1985	5014	-11011903	243.325	true	du3lnvl	0E-9	-364.999999488	2015-01-01	2015-03-13T12:36:38
+6	32767	3021	123456	604587.000	true	yanavnd	0.100000000	80698.996162560	2014-11-11	2015-03-13T12:36:38
+7	-32767	1002	7210457	3.141	false	jiw3n4	0E-9	6057.999990784	1988-03-21	1901-01-01T00:00
+8	255	2147483647	11011920	-0.123	true	wangjuoo5	987456.123000000	12.140000256	1989-03-21	9999-11-11T12:12
+10	1991	5014	9223372036854775807	-258.369	false	wangynnsf	-123456.540000000	0.235000000	2015-04-02	2013-04-02T15:16:52
+11	1989	25699	-9223372036854775807	0.666	true	yunlj8@nk	-987.001000000	4.336000000	2015-04-02	1989-03-21T13:11
+13	-32767	2147483647	-9223372036854775807	100.001	false	wenlsfnl	123.456000000	3.141592832	2015-04-02	2015-04-02T00:00
+14	255	103	11011902	0.000	false	 	3.141592654	2.036000000	2015-04-02	2015-04-02T00:00
+15	1992	3021	11011920	0.000	true		3.141592653	20.455999488	9999-12-12	2015-04-02T00:00
 
 -- !union25 --
-1	2	3	4	3.14	hello	world	0	1.1	1989-03-21	1989-03-21T13:00
-1	1989	1001	11011902	123.123	true	wangjuoo4	0.1	6.333000192	1989-03-21	1989-03-21T13:00
-2	1986	1001	11011903	1243.5	false	wangynnsf	20.268	789.249982464	1901-12-31	1989-03-21T13:00
-3	1989	1002	11011905	24453.325	false	yunlj8@nk	78945	3654.000050176	2012-03-14	2000-01-01T00:00
-4	1991	3021	-11011907	243243.325	false	yanvjldjlll	2.06	-0.001	3124-10-10	2015-03-13T10:30
-5	1985	5014	-11011903	243.325	true	du3lnvl	0	-364.999999488	2015-01-01	2015-03-13T12:36:38
-6	32767	3021	123456	604587	true	yanavnd	0.1	80698.99616256	2014-11-11	2015-03-13T12:36:38
-7	-32767	1002	7210457	3.141	false	jiw3n4	0	6057.999990784	1988-03-21	1901-01-01T00:00
-8	255	2147483647	11011920	-0.123	true	wangjuoo5	987456.123	12.140000256	1989-03-21	9999-11-11T12:12
-10	1991	5014	9223372036854775807	-258.369	false	wangynnsf	-123456.54	0.235	2015-04-02	2013-04-02T15:16:52
-11	1989	25699	-9223372036854775807	0.666	true	yunlj8@nk	-987.001	4.336	2015-04-02	1989-03-21T13:11
-13	-32767	2147483647	-9223372036854775807	100.001	false	wenlsfnl	123.456	3.141592832	2015-04-02	2015-04-02T00:00
-14	255	103	11011902	0	false	 	3.141592654	2.036	2015-04-02	2015-04-02T00:00
-15	1992	3021	11011920	0	true		3.141592653	20.455999488	9999-12-12	2015-04-02T00:00
+1	2	3	4	3.140	hello	world	0E-9	1.100000000	1989-03-21	1989-03-21T13:00
+1	1989	1001	11011902	123.123	true	wangjuoo4	0.100000000	6.333000192	1989-03-21	1989-03-21T13:00
+2	1986	1001	11011903	1243.500	false	wangynnsf	20.268000000	789.249982464	1901-12-31	1989-03-21T13:00
+3	1989	1002	11011905	24453.325	false	yunlj8@nk	78945.000000000	3654.000050176	2012-03-14	2000-01-01T00:00
+4	1991	3021	-11011907	243243.325	false	yanvjldjlll	2.060000000	-0.001000000	3124-10-10	2015-03-13T10:30
+5	1985	5014	-11011903	243.325	true	du3lnvl	0E-9	-364.999999488	2015-01-01	2015-03-13T12:36:38
+6	32767	3021	123456	604587.000	true	yanavnd	0.100000000	80698.996162560	2014-11-11	2015-03-13T12:36:38
+7	-32767	1002	7210457	3.141	false	jiw3n4	0E-9	6057.999990784	1988-03-21	1901-01-01T00:00
+8	255	2147483647	11011920	-0.123	true	wangjuoo5	987456.123000000	12.140000256	1989-03-21	9999-11-11T12:12
+10	1991	5014	9223372036854775807	-258.369	false	wangynnsf	-123456.540000000	0.235000000	2015-04-02	2013-04-02T15:16:52
+11	1989	25699	-9223372036854775807	0.666	true	yunlj8@nk	-987.001000000	4.336000000	2015-04-02	1989-03-21T13:11
+13	-32767	2147483647	-9223372036854775807	100.001	false	wenlsfnl	123.456000000	3.141592832	2015-04-02	2015-04-02T00:00
+14	255	103	11011902	0.000	false	 	3.141592654	2.036000000	2015-04-02	2015-04-02T00:00
+15	1992	3021	11011920	0.000	true		3.141592653	20.455999488	9999-12-12	2015-04-02T00:00
 
 -- !union26 --
-0.0001	1E-7
-1	2
-1.01	2
+0	1E-7
+1	2.0000000
+1	2.0000000
 
 -- !union27 --
 1	2
 hell0	
 
 -- !union28 --
-1	2
+1	2.00000
 
 -- !union29 --
-1	2
+1	2.00000
 
 -- !union30 --
-1	2
-1	2
-1	2
+1	2.00000
+1	2.00000
+1	2.00000
 
 -- !union31 --
-1	2
-1	2
+1	2.00000
+1	2.00000
 
 -- !union32 --
-1	2
+1	2.00000
 
 -- !union33 --
 2016-07-01
diff --git a/regression-test/data/tpcds_sf1_p1/sql/q58.out b/regression-test/data/tpcds_sf1_p1/sql/q58.out
index fc6b60f5f2..c83e736233 100644
--- a/regression-test/data/tpcds_sf1_p1/sql/q58.out
+++ b/regression-test/data/tpcds_sf1_p1/sql/q58.out
@@ -1,6 +1,6 @@
 -- This file is automatically generated. You should know what you did if you want to edit this
 -- !q58 --
-AAAAAAAACNGBAAAA	1900.15	11.1499568	1950.92	11.4478719	1829.52	10.7355046	1893.53
-AAAAAAAAIDOAAAAA	6605.22	11.5748247	6078.33	10.6515156	6338.25	11.106993	6340.6
-AAAAAAAAJMFCAAAA	3608.52	11.4503737	3590.47	11.3930983	3305.82	10.4898613	3501.603333333
+AAAAAAAACNGBAAAA	1900.150000000	11.14	1950.920000000	11.44	1829.520000000	10.73	1893.530000000
+AAAAAAAAIDOAAAAA	6605.220000000	11.57	6078.330000000	10.65	6338.250000000	11.10	6340.600000000
+AAAAAAAAJMFCAAAA	3608.520000000	11.45	3590.470000000	11.39	3305.820000000	10.48	3501.603333333
 
diff --git a/regression-test/data/tpcds_sf1_p1/sql/q83.out b/regression-test/data/tpcds_sf1_p1/sql/q83.out
index a4577094bc..821a2547c5 100644
--- a/regression-test/data/tpcds_sf1_p1/sql/q83.out
+++ b/regression-test/data/tpcds_sf1_p1/sql/q83.out
@@ -1,24 +1,24 @@
 -- This file is automatically generated. You should know what you did if you want to edit this
 -- !q83 --
-AAAAAAAAANECAAAA	34	18.579235	9	4.9180328	18	9.8360656	20.333333333
-AAAAAAAACAOBAAAA	27	11.6883117	38	16.4502165	12	5.1948052	25.666666667
-AAAAAAAACBNDAAAA	38	11.6207951	52	15.9021407	19	5.8103976	36.333333333
-AAAAAAAACCGAAAAA	30	15.1515152	18	9.0909091	18	9.0909091	22
-AAAAAAAACKBDAAAA	32	12.6984127	22	8.7301587	30	11.9047619	28
-AAAAAAAACOIBAAAA	31	8.0729167	29	7.5520833	68	17.7083333	42.666666667
-AAAAAAAAEBLDAAAA	9	3.7037037	34	13.9917695	38	15.6378601	27
-AAAAAAAAELFDAAAA	54	20.2247191	4	1.4981273	31	11.6104869	29.666666667
-AAAAAAAAFEBAAAAA	19	15.4471545	17	13.8211382	5	4.0650407	13.666666667
-AAAAAAAAFODDAAAA	30	11.4942529	7	2.6819923	50	19.1570881	29
-AAAAAAAAGLMCAAAA	37	26.8115942	3	2.173913	6	4.3478261	15.333333333
-AAAAAAAAHAGDAAAA	16	8.3333333	15	7.8125	33	17.1875	21.333333333
-AAAAAAAAHCDEAAAA	41	19.2488263	1	0.4694836	29	13.6150235	23.666666667
-AAAAAAAALAEBAAAA	19	6.959707	21	7.6923077	51	18.6813187	30.333333333
-AAAAAAAAMBDEAAAA	26	6.372549	66	16.1764706	44	10.7843137	45.333333333
-AAAAAAAAMBGBAAAA	5	5.0505051	1	1.010101	27	27.2727273	11
-AAAAAAAAMDODAAAA	17	10.8974359	32	20.5128205	3	1.9230769	17.333333333
-AAAAAAAAMOIAAAAA	84	25.4545455	12	3.6363636	14	4.2424242	36.666666667
-AAAAAAAANMAAAAAA	12	25	1	2.0833333	3	6.25	5.333333333
-AAAAAAAAODOAAAAA	5	4.3859649	3	2.6315789	30	26.3157895	12.666666667
-AAAAAAAAPJOCAAAA	29	15.3439153	21	11.1111111	13	6.8783069	21
+AAAAAAAAANECAAAA	34	18.57	9	4.91	18	9.83	20.333333333
+AAAAAAAACAOBAAAA	27	11.68	38	16.45	12	5.19	25.666666667
+AAAAAAAACBNDAAAA	38	11.62	52	15.90	19	5.81	36.333333333
+AAAAAAAACCGAAAAA	30	15.15	18	9.09	18	9.09	22.000000000
+AAAAAAAACKBDAAAA	32	12.69	22	8.73	30	11.90	28.000000000
+AAAAAAAACOIBAAAA	31	8.07	29	7.55	68	17.70	42.666666667
+AAAAAAAAEBLDAAAA	9	3.70	34	13.99	38	15.63	27.000000000
+AAAAAAAAELFDAAAA	54	20.22	4	1.49	31	11.61	29.666666667
+AAAAAAAAFEBAAAAA	19	15.44	17	13.82	5	4.06	13.666666667
+AAAAAAAAFODDAAAA	30	11.49	7	2.68	50	19.15	29.000000000
+AAAAAAAAGLMCAAAA	37	26.81	3	2.17	6	4.34	15.333333333
+AAAAAAAAHAGDAAAA	16	8.33	15	7.81	33	17.18	21.333333333
+AAAAAAAAHCDEAAAA	41	19.24	1	0.46	29	13.61	23.666666667
+AAAAAAAALAEBAAAA	19	6.95	21	7.69	51	18.68	30.333333333
+AAAAAAAAMBDEAAAA	26	6.37	66	16.17	44	10.78	45.333333333
+AAAAAAAAMBGBAAAA	5	5.05	1	1.01	27	27.27	11.000000000
+AAAAAAAAMDODAAAA	17	10.89	32	20.51	3	1.92	17.333333333
+AAAAAAAAMOIAAAAA	84	25.45	12	3.63	14	4.24	36.666666667
+AAAAAAAANMAAAAAA	12	25.00	1	2.08	3	6.25	5.333333333
+AAAAAAAAODOAAAAA	5	4.38	3	2.63	30	26.31	12.666666667
+AAAAAAAAPJOCAAAA	29	15.34	21	11.11	13	6.87	21.000000000
 
diff --git a/regression-test/data/tpcds_sf1_unique_p1/sql/q58.out b/regression-test/data/tpcds_sf1_unique_p1/sql/q58.out
index fc6b60f5f2..c83e736233 100644
--- a/regression-test/data/tpcds_sf1_unique_p1/sql/q58.out
+++ b/regression-test/data/tpcds_sf1_unique_p1/sql/q58.out
@@ -1,6 +1,6 @@
 -- This file is automatically generated. You should know what you did if you want to edit this
 -- !q58 --
-AAAAAAAACNGBAAAA	1900.15	11.1499568	1950.92	11.4478719	1829.52	10.7355046	1893.53
-AAAAAAAAIDOAAAAA	6605.22	11.5748247	6078.33	10.6515156	6338.25	11.106993	6340.6
-AAAAAAAAJMFCAAAA	3608.52	11.4503737	3590.47	11.3930983	3305.82	10.4898613	3501.603333333
+AAAAAAAACNGBAAAA	1900.150000000	11.14	1950.920000000	11.44	1829.520000000	10.73	1893.530000000
+AAAAAAAAIDOAAAAA	6605.220000000	11.57	6078.330000000	10.65	6338.250000000	11.10	6340.600000000
+AAAAAAAAJMFCAAAA	3608.520000000	11.45	3590.470000000	11.39	3305.820000000	10.48	3501.603333333
 
diff --git a/regression-test/data/tpcds_sf1_unique_p1/sql/q83.out b/regression-test/data/tpcds_sf1_unique_p1/sql/q83.out
index a4577094bc..821a2547c5 100644
--- a/regression-test/data/tpcds_sf1_unique_p1/sql/q83.out
+++ b/regression-test/data/tpcds_sf1_unique_p1/sql/q83.out
@@ -1,24 +1,24 @@
 -- This file is automatically generated. You should know what you did if you want to edit this
 -- !q83 --
-AAAAAAAAANECAAAA	34	18.579235	9	4.9180328	18	9.8360656	20.333333333
-AAAAAAAACAOBAAAA	27	11.6883117	38	16.4502165	12	5.1948052	25.666666667
-AAAAAAAACBNDAAAA	38	11.6207951	52	15.9021407	19	5.8103976	36.333333333
-AAAAAAAACCGAAAAA	30	15.1515152	18	9.0909091	18	9.0909091	22
-AAAAAAAACKBDAAAA	32	12.6984127	22	8.7301587	30	11.9047619	28
-AAAAAAAACOIBAAAA	31	8.0729167	29	7.5520833	68	17.7083333	42.666666667
-AAAAAAAAEBLDAAAA	9	3.7037037	34	13.9917695	38	15.6378601	27
-AAAAAAAAELFDAAAA	54	20.2247191	4	1.4981273	31	11.6104869	29.666666667
-AAAAAAAAFEBAAAAA	19	15.4471545	17	13.8211382	5	4.0650407	13.666666667
-AAAAAAAAFODDAAAA	30	11.4942529	7	2.6819923	50	19.1570881	29
-AAAAAAAAGLMCAAAA	37	26.8115942	3	2.173913	6	4.3478261	15.333333333
-AAAAAAAAHAGDAAAA	16	8.3333333	15	7.8125	33	17.1875	21.333333333
-AAAAAAAAHCDEAAAA	41	19.2488263	1	0.4694836	29	13.6150235	23.666666667
-AAAAAAAALAEBAAAA	19	6.959707	21	7.6923077	51	18.6813187	30.333333333
-AAAAAAAAMBDEAAAA	26	6.372549	66	16.1764706	44	10.7843137	45.333333333
-AAAAAAAAMBGBAAAA	5	5.0505051	1	1.010101	27	27.2727273	11
-AAAAAAAAMDODAAAA	17	10.8974359	32	20.5128205	3	1.9230769	17.333333333
-AAAAAAAAMOIAAAAA	84	25.4545455	12	3.6363636	14	4.2424242	36.666666667
-AAAAAAAANMAAAAAA	12	25	1	2.0833333	3	6.25	5.333333333
-AAAAAAAAODOAAAAA	5	4.3859649	3	2.6315789	30	26.3157895	12.666666667
-AAAAAAAAPJOCAAAA	29	15.3439153	21	11.1111111	13	6.8783069	21
+AAAAAAAAANECAAAA	34	18.57	9	4.91	18	9.83	20.333333333
+AAAAAAAACAOBAAAA	27	11.68	38	16.45	12	5.19	25.666666667
+AAAAAAAACBNDAAAA	38	11.62	52	15.90	19	5.81	36.333333333
+AAAAAAAACCGAAAAA	30	15.15	18	9.09	18	9.09	22.000000000
+AAAAAAAACKBDAAAA	32	12.69	22	8.73	30	11.90	28.000000000
+AAAAAAAACOIBAAAA	31	8.07	29	7.55	68	17.70	42.666666667
+AAAAAAAAEBLDAAAA	9	3.70	34	13.99	38	15.63	27.000000000
+AAAAAAAAELFDAAAA	54	20.22	4	1.49	31	11.61	29.666666667
+AAAAAAAAFEBAAAAA	19	15.44	17	13.82	5	4.06	13.666666667
+AAAAAAAAFODDAAAA	30	11.49	7	2.68	50	19.15	29.000000000
+AAAAAAAAGLMCAAAA	37	26.81	3	2.17	6	4.34	15.333333333
+AAAAAAAAHAGDAAAA	16	8.33	15	7.81	33	17.18	21.333333333
+AAAAAAAAHCDEAAAA	41	19.24	1	0.46	29	13.61	23.666666667
+AAAAAAAALAEBAAAA	19	6.95	21	7.69	51	18.68	30.333333333
+AAAAAAAAMBDEAAAA	26	6.37	66	16.17	44	10.78	45.333333333
+AAAAAAAAMBGBAAAA	5	5.05	1	1.01	27	27.27	11.000000000
+AAAAAAAAMDODAAAA	17	10.89	32	20.51	3	1.92	17.333333333
+AAAAAAAAMOIAAAAA	84	25.45	12	3.63	14	4.24	36.666666667
+AAAAAAAANMAAAAAA	12	25.00	1	2.08	3	6.25	5.333333333
+AAAAAAAAODOAAAAA	5	4.38	3	2.63	30	26.31	12.666666667
+AAAAAAAAPJOCAAAA	29	15.34	21	11.11	13	6.87	21.000000000
 
diff --git a/regression-test/suites/data_model_p0/duplicate/storage/test_dup_tab_decimal.groovy b/regression-test/suites/data_model_p0/duplicate/storage/test_dup_tab_decimal.groovy
index 385ac5d39a..0cbd2ae04c 100644
--- a/regression-test/suites/data_model_p0/duplicate/storage/test_dup_tab_decimal.groovy
+++ b/regression-test/suites/data_model_p0/duplicate/storage/test_dup_tab_decimal.groovy
@@ -48,74 +48,74 @@ PROPERTIES (
     // query decimal
     test {
         sql "select siteid from ${table1} order by siteid"
-        result([[1.1],[1.1],[2.1],[3.1],[4.1]])
+        result([[1.10000],[1.10000],[2.10000],[3.10000],[4.10000]])
     }
     test {
         sql "select siteid,citycode from ${table1} order by siteid,citycode"
-        result([[1.1,1.2],[1.1,2.2],[2.1,2.2],[3.1,3.2],[4.1,4.2]])
+        result([[1.10000,1.20000],[1.10000,2.20000],[2.10000,2.20000],[3.10000,3.20000],[4.10000,4.20000]])
     }
 
     // pred is decimal
     test {
         sql "select siteid from ${table1} where siteid=4.1"
-        result([[4.1]])
+        result([[4.10000]])
     }
     test {
         sql "select siteid from ${table1} where siteid=1.1"
-        result([[1.1],[1.1]])
+        result([[1.10000],[1.10000]])
     }
 
     // pred not key
     test {
         sql "select citycode from ${table1} where citycode=2.2"
-        result([[2.2],[2.2]])
+        result([[2.20000],[2.20000]])
     }
     test {
         sql "select citycode from ${table1} where citycode=4.2"
-        result([[4.2]])
+        result([[4.20000]])
     }
 
     // pred column not same with read column
     test {
         sql "select citycode from ${table1} where siteid=1.1 order by citycode"
-        result([[1.2],[2.2]])
+        result([[1.20000],[2.20000]])
     }
 
     test {
         sql "select citycode from ${table1} where siteid=4.1 order by citycode"
-        result([[4.2]])
+        result([[4.20000]])
     }
 
     // pred column not same with read column;pred is not key
     test {
         sql "select siteid from ${table1} where citycode=2.2 order by siteid"
-        result([[1.1],[2.1]])
+        result([[1.10000],[2.10000]])
     }
 
     test {
         sql "select siteid from ${table1} where citycode=4.2 order by siteid"
-        result([[4.1]])
+        result([[4.10000]])
     }
 
     // int pred
     test {
         sql "select siteid from ${table1} where siteid in(4.1)"
-        result([[4.1]])
+        result([[4.10000]])
     }
 
     test {
         sql "select * from ${table1} where siteid in(4.1)"
-        result([[4.1,4.2,4.3,4.4]])
+        result([[4.10000,4.20000,4.30000,4.40000]])
     }
 
     test {
         sql "select userid from ${table1} where userid in(2.3)"
-        result([[2.3],[2.3]])
+        result([[2.30000],[2.30000]])
     }
 
     test {
         sql "select userid from ${table1} where userid not in(2.3) order by userid"
-        result([[1.3],[3.3],[4.3]])
+        result([[1.30000],[3.30000],[4.30000]])
     }
 
     sql "drop table if exists ${table1}"
diff --git a/regression-test/suites/data_model_p0/duplicate/storage/test_dup_tab_decimal_nullable.groovy b/regression-test/suites/data_model_p0/duplicate/storage/test_dup_tab_decimal_nullable.groovy
index a858ef7ea1..78e0abf0eb 100644
--- a/regression-test/suites/data_model_p0/duplicate/storage/test_dup_tab_decimal_nullable.groovy
+++ b/regression-test/suites/data_model_p0/duplicate/storage/test_dup_tab_decimal_nullable.groovy
@@ -53,7 +53,7 @@ PROPERTIES (
     // query decimal
     test {
         sql "select siteid from ${table1} order by siteid"
-        result([[null],[1.1],[1.1],[2.1],[3.1],[4.1]])
+        result([[null],[1.10000],[1.10000],[2.10000],[3.10000],[4.10000]])
     }
     // FIXME(wb) this may failed because of regression framework
 //    test {
@@ -64,54 +64,54 @@ PROPERTIES (
     // pred is decimal
     test {
         sql "select siteid from ${table1} where siteid=4.1"
-        result([[4.1]])
+        result([[4.10000]])
     }
     test {
         sql "select siteid from ${table1} where siteid=1.1"
-        result([[1.1],[1.1]])
+        result([[1.10000],[1.10000]])
     }
 
     // pred not key
     test {
         sql "select citycode from ${table1} where citycode=2.2"
-        result([[2.2],[2.2]])
+        result([[2.20000],[2.20000]])
     }
     test {
         sql "select citycode from ${table1} where citycode=4.2"
-        result([[4.2]])
+        result([[4.20000]])
     }
 
     // pred column not same with read column
     test {
         sql "select citycode from ${table1} where siteid=4.1"
-        result([[4.2]])
+        result([[4.20000]])
     }
 
     test {
         sql "select citycode from ${table1} where siteid=1.1 order by citycode"
-        result([[1.2],[2.2]])
+        result([[1.20000],[2.20000]])
     }
 
     // pred column not same with read column;pred is not key
     test {
         sql "select siteid from ${table1} where citycode=2.2 order by siteid"
-        result([[1.1],[2.1]])
+        result([[1.10000],[2.10000]])
     }
 
     test {
         sql "select siteid from ${table1} where citycode=4.2 order by siteid"
-        result([[4.1]])
+        result([[4.10000]])
     }
 
     // int pred
     test {
         sql "select userid from ${table1} where userid in(2.3)"
-        result([[2.3],[2.3]])
+        result([[2.30000],[2.30000]])
     }
 
     test {
         sql "select userid from ${table1} where userid not in(2.3) order by userid"
-        result([[1.3],[3.3],[4.3]])
+        result([[1.30000],[3.30000],[4.30000]])
     }
 
     // is null
@@ -122,7 +122,7 @@ PROPERTIES (
     // is not null
     test {
         sql "select userid from  ${table1} where userid is not null order by userid"
-        result([[1.3],[2.3],[2.3],[3.3],[4.3]])
+        result([[1.30000],[2.30000],[2.30000],[3.30000],[4.30000]])
     }
 
     sql "drop table if exists ${table1}"
diff --git a/regression-test/suites/query_p0/union/test_union.groovy b/regression-test/suites/query_p0/union/test_union.groovy
index 003a4b4775..314885630e 100644
--- a/regression-test/suites/query_p0/union/test_union.groovy
+++ b/regression-test/suites/query_p0/union/test_union.groovy
@@ -146,8 +146,6 @@ suite("test_union") {
              union distinct (select 1.00000000, 2.00000) order by 1, 2"""
     def res2 = sql"""select cast(1 as decimal), cast(2 as decimal) union distinct select 1.0, 2.0 
              union distinct (select 1.00000000, 2.00000) order by 1, 2"""
-    check2_doris(res1, res2)
-
 
     // test_union_multi
     List sub_sql = ["(select k1, k2, k3, k4, k5, k6, k7, k8, k9, k10, k11 from baseall where k1 % 3 = 0)"] * 10


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