You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by yi...@apache.org on 2022/07/26 14:36:18 UTC

[doris] branch master updated: [feature-wip] (datetimev2) support `cast` between datetimev2 with different scales (#11198)

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

yiguolei 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 d67029c830 [feature-wip] (datetimev2) support `cast` between datetimev2 with different scales (#11198)
d67029c830 is described below

commit d67029c83009d363f4e8af235e8ff4b2198cdd78
Author: Gabriel <ga...@gmail.com>
AuthorDate: Tue Jul 26 22:36:13 2022 +0800

    [feature-wip] (datetimev2) support `cast` between datetimev2 with different scales (#11198)
    
    * [feature-wip] (datetimev2) support `cast` between datetimev2 with different scale
---
 be/src/vec/data_types/data_type_factory.cpp        |  6 +-
 be/src/vec/data_types/data_type_time_v2.h          | 19 ++++++
 be/src/vec/functions/function_cast.h               | 21 +++++--
 be/src/vec/io/io_helper.h                          |  8 +--
 be/src/vec/runtime/vdatetime_value.cpp             | 23 +++++---
 be/src/vec/runtime/vdatetime_value.h               |  2 +-
 be/test/vec/runtime/vdatetime_value_test.cpp       | 64 ++++++++++++++++++++
 .../org/apache/doris/analysis/DateLiteral.java     | 69 +++++++++++++++-------
 .../java/org/apache/doris/catalog/ScalarType.java  |  1 +
 .../doris/common/proc/PartitionsProcDir.java       |  4 +-
 .../apache/doris/common/proc/RollupProcDir.java    |  4 +-
 .../doris/common/proc/SchemaChangeProcDir.java     |  4 +-
 .../apache/doris/common/util/PropertyAnalyzer.java |  2 +-
 .../java/org/apache/doris/load/DeleteHandler.java  | 10 +++-
 .../org/apache/doris/qe/cache/PartitionRange.java  |  2 +-
 .../org/apache/doris/analysis/DateLiteralTest.java |  6 +-
 .../org/apache/doris/planner/QueryPlanTest.java    |  8 +--
 .../doris/rewrite/RewriteDateLiteralRuleTest.java  | 28 ++++-----
 18 files changed, 210 insertions(+), 71 deletions(-)

diff --git a/be/src/vec/data_types/data_type_factory.cpp b/be/src/vec/data_types/data_type_factory.cpp
index 0025f86ec0..249d008205 100644
--- a/be/src/vec/data_types/data_type_factory.cpp
+++ b/be/src/vec/data_types/data_type_factory.cpp
@@ -87,7 +87,7 @@ DataTypePtr DataTypeFactory::create_data_type(const TypeDescriptor& col_desc, bo
         nested = std::make_shared<vectorized::DataTypeDateV2>();
         break;
     case TYPE_DATETIMEV2:
-        nested = std::make_shared<vectorized::DataTypeDateTimeV2>();
+        nested = std::make_shared<vectorized::DataTypeDateTimeV2>(col_desc.scale);
         break;
     case TYPE_DATETIME:
         nested = std::make_shared<vectorized::DataTypeDateTime>();
@@ -168,7 +168,7 @@ DataTypePtr DataTypeFactory::_create_primitive_data_type(const FieldType& type,
         result = std::make_shared<vectorized::DataTypeDateV2>();
         break;
     case OLAP_FIELD_TYPE_DATETIMEV2:
-        result = std::make_shared<vectorized::DataTypeDateTimeV2>();
+        result = std::make_shared<vectorized::DataTypeDateTimeV2>(scale);
         break;
     case OLAP_FIELD_TYPE_DATETIME:
         result = std::make_shared<vectorized::DataTypeDateTime>();
@@ -252,7 +252,7 @@ DataTypePtr DataTypeFactory::create_data_type(const PColumnMeta& pcolumn) {
         nested = std::make_shared<DataTypeDateV2>();
         break;
     case PGenericType::DATETIMEV2:
-        nested = std::make_shared<DataTypeDateTimeV2>();
+        nested = std::make_shared<DataTypeDateTimeV2>(pcolumn.decimal_param().scale());
         break;
     case PGenericType::DATETIME:
         nested = std::make_shared<DataTypeDateTime>();
diff --git a/be/src/vec/data_types/data_type_time_v2.h b/be/src/vec/data_types/data_type_time_v2.h
index 13d54d1490..0efff481e6 100644
--- a/be/src/vec/data_types/data_type_time_v2.h
+++ b/be/src/vec/data_types/data_type_time_v2.h
@@ -47,6 +47,15 @@ public:
 
 class DataTypeDateTimeV2 final : public DataTypeNumberBase<UInt64> {
 public:
+    static constexpr bool is_parametric = true;
+
+    DataTypeDateTimeV2(UInt32 scale = 0) : scale_(scale) {
+        if (UNLIKELY(scale > 6)) {
+            LOG(FATAL) << fmt::format("Scale {} is out of bounds", scale);
+        }
+    }
+
+    DataTypeDateTimeV2(const DataTypeDateTimeV2& rhs) : scale_(rhs.scale_) {}
     TypeIndex get_type_id() const override { return TypeIndex::DateTimeV2; }
     const char* get_family_name() const override { return "DateTimeV2"; }
     std::string do_get_name() const override { return "DateTimeV2"; }
@@ -60,9 +69,19 @@ public:
 
     MutableColumnPtr create_column() const override;
 
+    const UInt32 get_scale() const { return scale_; }
+
     static void cast_to_date(const UInt64 from, Int64& to);
     static void cast_to_date_time(const UInt64 from, Int64& to);
     static void cast_to_date_v2(const UInt64 from, UInt32& to);
+
+private:
+    UInt32 scale_;
 };
 
+template <typename DataType>
+constexpr bool IsDataTypeDateTimeV2 = false;
+template <>
+inline constexpr bool IsDataTypeDateTimeV2<DataTypeDateTimeV2> = true;
+
 } // namespace doris::vectorized
diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h
index be83d31b44..5494056fd8 100644
--- a/be/src/vec/functions/function_cast.h
+++ b/be/src/vec/functions/function_cast.h
@@ -163,8 +163,8 @@ struct ConvertImpl {
                                              IsDateV2Type<ToDataType>) {
                             DataTypeDateTimeV2::cast_to_date_v2(vec_from[i], vec_to[i]);
                         } else {
-                            // TODO: conversion between datetimev2 with different scales
-                            vec_to[i] = vec_from[i];
+                            UInt32 scale = additions;
+                            vec_to[i] = vec_from[i] / std::pow(10, 6 - scale);
                         }
                     } else if constexpr (IsTimeType<ToDataType>) {
                         if constexpr (IsDateTimeType<ToDataType> && IsDateV2Type<FromDataType>) {
@@ -420,7 +420,8 @@ bool try_parse_impl(typename DataType::FieldType& x, ReadBuffer& rb, const DateL
     }
 
     if constexpr (IsDateTimeV2Type<DataType>) {
-        return try_read_datetime_v2_text(x, rb);
+        UInt32 scale = additions;
+        return try_read_datetime_v2_text(x, rb, scale);
     }
 
     if constexpr (std::is_floating_point_v<typename DataType::FieldType>) {
@@ -669,9 +670,16 @@ private:
 
                     ret_status = ConvertImpl<LeftDataType, RightDataType, Name>::execute(
                             block, arguments, result, input_rows_count, scale);
-                } else
+                } else if constexpr (IsDataTypeDateTimeV2<RightDataType>) {
+                    const ColumnWithTypeAndName& scale_column = block.get_by_position(result);
+                    auto type =
+                            check_and_get_data_type<DataTypeDateTimeV2>(scale_column.type.get());
+                    ret_status = ConvertImpl<LeftDataType, RightDataType, Name>::execute(
+                            block, arguments, result, input_rows_count, type->get_scale());
+                } else {
                     ret_status = ConvertImpl<LeftDataType, RightDataType, Name>::execute(
                             block, arguments, result, input_rows_count);
+                }
                 return true;
             };
 
@@ -894,6 +902,11 @@ struct ConvertThroughParsing {
             if constexpr (IsDataTypeDecimal<ToDataType>) {
                 parsed = try_parse_impl<ToDataType>(vec_to[i], read_buffer, local_time_zone,
                                                     vec_to.get_scale());
+            } else if constexpr (IsDataTypeDateTimeV2<ToDataType>) {
+                auto type = check_and_get_data_type<DataTypeDateTimeV2>(
+                        block.get_by_position(result).type.get());
+                parsed = try_parse_impl<ToDataType>(vec_to[i], read_buffer, local_time_zone,
+                                                    type->get_scale());
             } else {
                 parsed = try_parse_impl<ToDataType>(vec_to[i], read_buffer, local_time_zone);
             }
diff --git a/be/src/vec/io/io_helper.h b/be/src/vec/io/io_helper.h
index 00cddc22e9..64cca34005 100644
--- a/be/src/vec/io/io_helper.h
+++ b/be/src/vec/io/io_helper.h
@@ -293,10 +293,10 @@ bool read_date_v2_text_impl(T& x, ReadBuffer& buf) {
 }
 
 template <typename T>
-bool read_datetime_v2_text_impl(T& x, ReadBuffer& buf) {
+bool read_datetime_v2_text_impl(T& x, ReadBuffer& buf, UInt32 scale = -1) {
     static_assert(std::is_same_v<UInt64, T>);
     auto dv = binary_cast<UInt64, DateV2Value<DateTimeV2ValueType>>(x);
-    auto ans = dv.from_date_str(buf.position(), buf.count());
+    auto ans = dv.from_date_str(buf.position(), buf.count(), scale);
 
     // only to match the is_all_read() check to prevent return null
     buf.position() = buf.end();
@@ -374,7 +374,7 @@ bool try_read_date_v2_text(T& x, ReadBuffer& in) {
 }
 
 template <typename T>
-bool try_read_datetime_v2_text(T& x, ReadBuffer& in) {
-    return read_datetime_v2_text_impl<T>(x, in);
+bool try_read_datetime_v2_text(T& x, ReadBuffer& in, UInt32 scale) {
+    return read_datetime_v2_text_impl<T>(x, in, scale);
 }
 } // namespace doris::vectorized
diff --git a/be/src/vec/runtime/vdatetime_value.cpp b/be/src/vec/runtime/vdatetime_value.cpp
index a48661b6d5..9056e44c46 100644
--- a/be/src/vec/runtime/vdatetime_value.cpp
+++ b/be/src/vec/runtime/vdatetime_value.cpp
@@ -1684,7 +1684,7 @@ bool DateV2Value<T>::is_invalid(uint32_t year, uint32_t month, uint32_t day, uin
 // YYYY-MM-DD HH-MM-DD.FFFFFF AM in default format
 // 0    1  2  3  4  5  6      7
 template <typename T>
-bool DateV2Value<T>::from_date_str(const char* date_str, int len) {
+bool DateV2Value<T>::from_date_str(const char* date_str, int len, int scale) {
     const char* ptr = date_str;
     const char* end = date_str + len;
     // ONLY 2, 6 can follow by a sapce
@@ -1732,6 +1732,11 @@ bool DateV2Value<T>::from_date_str(const char* date_str, int len) {
         if (field_idx == 6) {
             // Microsecond
             temp_val *= std::pow(10, 6 - (end - start));
+            if constexpr (is_datetime) {
+                if (scale >= 0) {
+                    temp_val /= std::pow(10, 6 - scale);
+                }
+            }
         }
         // Imposible
         if (temp_val > 999999L) {
@@ -2258,15 +2263,15 @@ int32_t DateV2Value<T>::to_buffer(char* buffer, int scale) const {
         /* Second */
         *buffer++ = (char)('0' + (date_v2_value_.second_ / 10));
         *buffer++ = (char)('0' + (date_v2_value_.second_ % 10));
-        if (scale != 0) {
+        if (scale != 0 && date_v2_value_.microsecond_ > 0) {
             *buffer++ = '.';
-        }
-        /* Microsecond */
-        uint32_t ms = date_v2_value_.microsecond_;
-        int ms_width = scale == -1 ? 6 : std::min(6, scale);
-        for (int i = 0; i < ms_width; i++) {
-            *buffer++ = (char)('0' + (ms / std::pow(10, 5 - i)));
-            ms %= (uint32_t)std::pow(10, 5 - i);
+            /* Microsecond */
+            uint32_t ms = date_v2_value_.microsecond_;
+            int ms_width = scale == -1 ? 6 : std::min(6, scale);
+            for (int i = 0; i < ms_width; i++) {
+                *buffer++ = (char)('0' + (ms / std::pow(10, 5 - i)));
+                ms %= (uint32_t)std::pow(10, 5 - i);
+            }
         }
     }
     return buffer - start;
diff --git a/be/src/vec/runtime/vdatetime_value.h b/be/src/vec/runtime/vdatetime_value.h
index 7cf696b8c7..5b6663c796 100644
--- a/be/src/vec/runtime/vdatetime_value.h
+++ b/be/src/vec/runtime/vdatetime_value.h
@@ -801,7 +801,7 @@ public:
     // 'YYMMDD', 'YYYYMMDD', 'YYMMDDHHMMSS', 'YYYYMMDDHHMMSS'
     // 'YY-MM-DD', 'YYYY-MM-DD', 'YY-MM-DD HH.MM.SS'
     // 'YYYYMMDDTHHMMSS'
-    bool from_date_str(const char* str, int len);
+    bool from_date_str(const char* str, int len, int scale = -1);
 
     // Convert this value to string
     // this will check type to decide which format to convert
diff --git a/be/test/vec/runtime/vdatetime_value_test.cpp b/be/test/vec/runtime/vdatetime_value_test.cpp
index 809705dead..312b0ed423 100644
--- a/be/test/vec/runtime/vdatetime_value_test.cpp
+++ b/be/test/vec/runtime/vdatetime_value_test.cpp
@@ -483,4 +483,68 @@ TEST(VDateTimeValueTest, date_diff_test) {
     }
 }
 
+TEST(VDateTimeValueTest, date_v2_to_string_test) {
+    uint16_t year = 2022;
+    uint8_t month = 5;
+    uint8_t day = 24;
+    uint8_t hour = 23;
+    uint8_t minute = 50;
+    uint8_t second = 50;
+    uint32_t ms = 555000;
+
+    {
+        DateV2Value<DateV2ValueType> date_v2;
+        date_v2.set_time(year, month, day, 0, 0, 0, 0);
+
+        char buf[30];
+        int len = date_v2.to_buffer(buf);
+        EXPECT_TRUE(std::string(buf, len) == std::string("2022-05-24"));
+    }
+
+    {
+        DateV2Value<DateTimeV2ValueType> date_v2;
+        date_v2.set_time(year, month, day, hour, minute, second, ms);
+
+        char buf[30];
+        int len = date_v2.to_buffer(buf);
+        EXPECT_TRUE(std::string(buf, len) == std::string("2022-05-24 23:50:50.555000"));
+    }
+
+    {
+        DateV2Value<DateTimeV2ValueType> date_v2;
+        date_v2.set_time(year, month, day, hour, minute, second, ms);
+
+        char buf[30];
+        int len = date_v2.to_buffer(buf, 3);
+        EXPECT_TRUE(std::string(buf, len) == std::string("2022-05-24 23:50:50.555"));
+    }
+
+    {
+        DateV2Value<DateTimeV2ValueType> date_v2;
+        date_v2.set_time(year, month, day, hour, minute, second, ms);
+
+        char buf[30];
+        int len = date_v2.to_buffer(buf, 2);
+        EXPECT_TRUE(std::string(buf, len) == std::string("2022-05-24 23:50:50.55"));
+    }
+
+    {
+        DateV2Value<DateTimeV2ValueType> date_v2;
+        date_v2.set_time(year, month, day, hour, minute, second, ms);
+
+        char buf[30];
+        int len = date_v2.to_buffer(buf, 6);
+        EXPECT_TRUE(std::string(buf, len) == std::string("2022-05-24 23:50:50.555000"));
+    }
+
+    {
+        DateV2Value<DateTimeV2ValueType> date_v2;
+        date_v2.set_time(year, month, day, hour, minute, second, 0);
+
+        char buf[30];
+        int len = date_v2.to_buffer(buf);
+        EXPECT_TRUE(std::string(buf, len) == std::string("2022-05-24 23:50:50"));
+    }
+}
+
 } // namespace doris::vectorized
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java
index 52b2029e04..2a08cb6e75 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java
@@ -31,6 +31,7 @@ import org.apache.doris.thrift.TExprNode;
 import org.apache.doris.thrift.TExprNodeType;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -46,10 +47,12 @@ import java.time.ZoneId;
 import java.time.ZonedDateTime;
 import java.time.format.DateTimeFormatter;
 import java.time.format.DateTimeFormatterBuilder;
+import java.time.format.DateTimeParseException;
 import java.time.format.TextStyle;
 import java.time.temporal.ChronoField;
 import java.time.temporal.TemporalAccessor;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.TimeZone;
@@ -74,6 +77,7 @@ public class DateLiteral extends LiteralExpr {
     private static DateTimeFormatter DATE_TIME_FORMATTER = null;
     private static DateTimeFormatter DATE_TIME_FORMATTER_TO_MICRO_SECOND = null;
     private static DateTimeFormatter DATE_FORMATTER = null;
+    private static List<DateTimeFormatter> formatterList = null;
     /*
      *  The datekey type is widely used in data warehouses
      *  For example, 20121229 means '2012-12-29'
@@ -99,8 +103,18 @@ public class DateLiteral extends LiteralExpr {
             DATETIMEKEY_FORMATTER = formatBuilder("%Y%m%d%H%i%s").toFormatter();
             DATE_TIME_FORMATTER_TO_MICRO_SECOND = new DateTimeFormatterBuilder()
                     .appendPattern("uuuu-MM-dd HH:mm:ss")
-                    .appendFraction(ChronoField.NANO_OF_SECOND, 0, 6, true)
+                    .appendFraction(ChronoField.MICRO_OF_SECOND, 0, 6, true)
                     .toFormatter();
+            formatterList = Lists.newArrayList(
+                    formatBuilder("%Y%m%d").appendLiteral('T').appendPattern("HHmmss")
+                            .appendFraction(ChronoField.MICRO_OF_SECOND, 0, 6, true).toFormatter(),
+                    formatBuilder("%Y%m%d").appendLiteral('T').appendPattern("HHmmss")
+                            .appendFraction(ChronoField.MICRO_OF_SECOND, 0, 6, false).toFormatter(),
+                    formatBuilder("%Y%m%d%H%i%s")
+                            .appendFraction(ChronoField.MICRO_OF_SECOND, 0, 6, true).toFormatter(),
+                    formatBuilder("%Y%m%d%H%i%s")
+                            .appendFraction(ChronoField.MICRO_OF_SECOND, 0, 6, false).toFormatter(),
+                    DATETIMEKEY_FORMATTER, DATEKEY_FORMATTER);
         } catch (AnalysisException e) {
             LOG.error("invalid date format", e);
             System.exit(-1);
@@ -321,13 +335,22 @@ public class DateLiteral extends LiteralExpr {
     private void init(String s, Type type) throws AnalysisException {
         try {
             Preconditions.checkArgument(type.isDateType());
-            TemporalAccessor dateTime;
-            if (s.length() == DATEKEY_LENGTH && !s.contains("-")) {
+            TemporalAccessor dateTime = null;
+            boolean parsed = false;
+            if (!s.contains("-")) {
                 // handle format like 20210106, but should not handle 2021-1-6
-                dateTime = DATEKEY_FORMATTER.parse(s);
-            } else if (s.length() == DATETIMEKEY_LENGTH && !s.contains("-")) {
-                // handle format like 20210106, but should not handle 2021-1-6
-                dateTime = DATETIMEKEY_FORMATTER.parse(s);
+                for (DateTimeFormatter formatter : formatterList) {
+                    try {
+                        dateTime = formatter.parse(s);
+                        parsed = true;
+                        break;
+                    } catch (DateTimeParseException ex) {
+                        // ignore
+                    }
+                }
+                if (!parsed) {
+                    throw new AnalysisException("Invalid date value: " + s);
+                }
             } else {
                 String[] datePart = s.contains(" ") ? s.split(" ")[0].split("-") : s.split("-");
                 DateTimeFormatterBuilder builder = new DateTimeFormatterBuilder();
@@ -380,7 +403,7 @@ public class DateLiteral extends LiteralExpr {
                             builder.appendPattern(String.join("", Collections.nCopies(timePart[i].contains(".")
                                     ? timePart[i].split("\\.")[0].length() : timePart[i].length(), "s")));
                             if (timePart[i].contains(".")) {
-                                builder.appendFraction(ChronoField.NANO_OF_SECOND, 0, 6, true);
+                                builder.appendFraction(ChronoField.MICRO_OF_SECOND, 0, 6, true);
                             }
                             break;
                         default:
@@ -392,8 +415,10 @@ public class DateLiteral extends LiteralExpr {
                 }
                 DateTimeFormatter formatter = builder.toFormatter();
                 dateTime = formatter.parse(s);
+                parsed = true;
             }
 
+            Preconditions.checkArgument(parsed);
             year = getOrDefault(dateTime, ChronoField.YEAR, 0);
             month = getOrDefault(dateTime, ChronoField.MONTH_OF_YEAR, 0);
             day = getOrDefault(dateTime, ChronoField.DAY_OF_MONTH, 0);
@@ -402,10 +427,9 @@ public class DateLiteral extends LiteralExpr {
             second = getOrDefault(dateTime, ChronoField.SECOND_OF_MINUTE, 0);
             microsecond = getOrDefault(dateTime, ChronoField.MICRO_OF_SECOND, 0);
             if (type.isDatetimeV2()) {
-                this.type = ScalarType.createDatetimeV2Type(6);
-            } else {
-                this.type = type;
+                this.roundFloor(((ScalarType) type).getScalarScale());
             }
+            this.type = type;
         } catch (Exception ex) {
             throw new AnalysisException("date literal [" + s + "] is invalid: " + ex.getMessage());
         }
@@ -513,14 +537,14 @@ public class DateLiteral extends LiteralExpr {
         long remain = Double.valueOf(microsecond % (Math.pow(10, 6 - newScale))).longValue();
         if (remain != 0) {
             microsecond = Double.valueOf((microsecond + (Math.pow(10, 6 - newScale)))
-                    / (Math.pow(10, 6 - newScale))).longValue();
+                    / (Math.pow(10, 6 - newScale)) * (Math.pow(10, 6 - newScale))).longValue();
         }
         type = ScalarType.createDatetimeV2Type(newScale);
     }
 
     public void roundFloor(int newScale) {
-        Preconditions.checkArgument(type.isDatetimeV2());
-        microsecond = Double.valueOf(microsecond / (Math.pow(10, 6 - newScale))).longValue();
+        microsecond = Double.valueOf(microsecond / (Math.pow(10, 6 - newScale))
+                * (Math.pow(10, 6 - newScale))).longValue();
         type = ScalarType.createDatetimeV2Type(newScale);
     }
 
@@ -532,8 +556,12 @@ public class DateLiteral extends LiteralExpr {
         if (type == PrimitiveType.DATE || type == PrimitiveType.DATEV2) {
             return String.format("%04d-%02d-%02d", year, month, day);
         } else if (type == PrimitiveType.DATETIMEV2) {
-            return String.format("%04d-%02d-%02d %02d:%02d:%02d.%06d",
-                    year, month, day, hour, minute, second, microsecond);
+            String tmp = String.format("%04d-%02d-%02d %02d:%02d:%02d",
+                    year, month, day, hour, minute, second);
+            if (microsecond == 0) {
+                return tmp;
+            }
+            return tmp + String.format(".%06d", microsecond);
         } else {
             return String.format("%04d-%02d-%02d %02d:%02d:%02d", year, month, day, hour, minute, second);
         }
@@ -673,8 +701,7 @@ public class DateLiteral extends LiteralExpr {
             this.type = Type.DATE;
         } else if (dateLiteralType == DateLiteralType.DATETIMEV2.value()) {
             fromPackedDatetime(in.readLong());
-            int scale = in.readInt();
-            this.type = ScalarType.createDatetimeV2Type(scale);
+            this.type = ScalarType.createDatetimeV2Type(in.readInt());
         } else if (dateLiteralType == DateLiteralType.DATEV2.value()) {
             fromPackedDatetime(in.readLong());
             this.type = Type.DATEV2;
@@ -782,6 +809,7 @@ public class DateLiteral extends LiteralExpr {
                         builder.appendValue(ChronoField.ALIGNED_WEEK_OF_YEAR, 2);
                         break;
                     case 'x':
+                    case 'Y': // %Y Year, numeric, four digits
                         // %x Year for the week, where Monday is the first day of the week,
                         // numeric, four digits; used with %v
                         builder.appendValue(ChronoField.YEAR, 4);
@@ -789,9 +817,6 @@ public class DateLiteral extends LiteralExpr {
                     case 'W': // %W Weekday name (Sunday..Saturday)
                         builder.appendText(ChronoField.DAY_OF_WEEK, TextStyle.FULL);
                         break;
-                    case 'Y': // %Y Year, numeric, four digits
-                        builder.appendPattern("uuuu");
-                        break;
                     case 'y': // %y Year, numeric (two digits)
                         builder.appendValueReduced(ChronoField.YEAR, 2, 2, 1970);
                         break;
@@ -843,7 +868,7 @@ public class DateLiteral extends LiteralExpr {
         final int hour = getOrDefault(accessor, ChronoField.HOUR_OF_DAY, 0);
         final int minute = getOrDefault(accessor, ChronoField.MINUTE_OF_HOUR, 0);
         final int second = getOrDefault(accessor, ChronoField.SECOND_OF_MINUTE, 0);
-        final int microSeconds = getOrDefault(accessor, ChronoField.NANO_OF_SECOND, 0);
+        final int microSeconds = getOrDefault(accessor, ChronoField.MICRO_OF_SECOND, 0);
 
         return LocalDateTime.of(year, month, dayOfMonth, hour, minute, second, microSeconds);
     }
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 6fb59c20c6..f3f3784b6f 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
@@ -78,6 +78,7 @@ public class ScalarType extends Type {
     public static final int MAX_DECIMAL32_PRECISION = 9;
     public static final int MAX_DECIMAL64_PRECISION = 18;
     public static final int MAX_DECIMAL128_PRECISION = 38;
+    public static final int MAX_DATETIMEV2_SCALE = 6;
 
     private static final Logger LOG = LogManager.getLogger(ScalarType.class);
     @SerializedName(value = "type")
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java
index 72665d62a5..14bab261bc 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java
@@ -104,9 +104,11 @@ public class PartitionsProcDir implements ProcDirInterface {
                         type = Type.DATETIME;
                         break;
                     case DATEV2:
-                    case DATETIMEV2:
                         type = Type.DATETIMEV2;
                         break;
+                    case DATETIMEV2:
+                        type = subExpr.getChild(1).getType();
+                        break;
                     default:
                         throw new AnalysisException("Invalid date type: " + subExpr.getChild(1).getType());
                 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/RollupProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/RollupProcDir.java
index c3d52787c4..fb65f5231c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/RollupProcDir.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/RollupProcDir.java
@@ -161,9 +161,11 @@ public class RollupProcDir implements ProcDirInterface {
                     type = Type.DATETIME;
                     break;
                 case DATEV2:
-                case DATETIMEV2:
                     type = Type.DATETIMEV2;
                     break;
+                case DATETIMEV2:
+                    type = subExpr.getChild(1).getType();
+                    break;
                 default:
                     throw new AnalysisException("Invalid date type: " + subExpr.getChild(1).getType());
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/SchemaChangeProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/SchemaChangeProcDir.java
index a7e171bb21..a2ff07f966 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/SchemaChangeProcDir.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/SchemaChangeProcDir.java
@@ -80,9 +80,11 @@ public class SchemaChangeProcDir implements ProcDirInterface {
                     type = Type.DATETIME;
                     break;
                 case DATEV2:
-                case DATETIMEV2:
                     type = Type.DATETIMEV2;
                     break;
+                case DATETIMEV2:
+                    type = subExpr.getChild(1).getType();
+                    break;
                 default:
                     throw new AnalysisException("Invalid date type: " + subExpr.getChild(1).getType());
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
index b7c7369f90..317beb78d0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
@@ -163,7 +163,7 @@ public class PropertyAnalyzer {
             } else if (key.equalsIgnoreCase(PROPERTIES_REMOTE_STORAGE_POLICY)) {
                 remoteStoragePolicy = value;
             } else if (key.equalsIgnoreCase(PROPERTIES_DATA_BASE_TIME)) {
-                DateLiteral dateLiteral = new DateLiteral(value, Type.DATETIME);
+                DateLiteral dateLiteral = new DateLiteral(value, DateLiteral.getDefaultDateType(Type.DATETIME));
                 dataBaseTimeMs = dateLiteral.unixTimestamp(TimeUtils.getTimeZone());
             } else if (!hasStoragePolicy && key.equalsIgnoreCase(PROPERTIES_STORAGE_POLICY)) {
                 if (!Strings.isNullOrEmpty(value)) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java
index 0344bffe89..e2fdabfdab 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java
@@ -37,6 +37,7 @@ import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.PartitionType;
 import org.apache.doris.catalog.PrimitiveType;
 import org.apache.doris.catalog.Replica;
+import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.Tablet;
 import org.apache.doris.catalog.TabletInvertedIndex;
@@ -583,12 +584,17 @@ public class DeleteHandler implements Writable {
                         }
                     } else if (column.getDataType() == PrimitiveType.DATE
                             || column.getDataType() == PrimitiveType.DATETIME
-                            || column.getDataType() == PrimitiveType.DATEV2
-                            || column.getDataType() == PrimitiveType.DATETIMEV2) {
+                            || column.getDataType() == PrimitiveType.DATEV2) {
                         DateLiteral dateLiteral = new DateLiteral(value, Type.fromPrimitiveType(column.getDataType()));
                         value = dateLiteral.getStringValue();
                         binaryPredicate.setChild(1, LiteralExpr.create(value,
                                 Type.fromPrimitiveType(column.getDataType())));
+                    } else if (column.getDataType() == PrimitiveType.DATETIMEV2) {
+                        DateLiteral dateLiteral = new DateLiteral(value,
+                                ScalarType.createDecimalType(ScalarType.MAX_DATETIMEV2_SCALE));
+                        value = dateLiteral.getStringValue();
+                        binaryPredicate.setChild(1, LiteralExpr.create(value,
+                                ScalarType.createDecimalType(ScalarType.MAX_DATETIMEV2_SCALE)));
                     }
                     LiteralExpr.create(value, Type.fromPrimitiveType(column.getDataType()));
                 } catch (AnalysisException e) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/PartitionRange.java b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/PartitionRange.java
index 1c5b5bb4a0..3a4e7dd45b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/PartitionRange.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/PartitionRange.java
@@ -459,7 +459,7 @@ public class PartitionRange {
                 LiteralExpr newLiteral;
                 if (key.keyType == KeyType.DATE) {
                     try {
-                        newLiteral = new DateLiteral(key.toString(), Type.DATE);
+                        newLiteral = new DateLiteral(key.toString(), DateLiteral.getDefaultDateType(Type.DATE));
                     } catch (Exception e) {
                         LOG.warn("Date's format is error {},{}", key.toString(), e);
                         continue;
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/DateLiteralTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/DateLiteralTest.java
index 60cee42cfe..dacce11b67 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/DateLiteralTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/DateLiteralTest.java
@@ -254,16 +254,16 @@ public class DateLiteralTest {
     public void testDateFormatForDatetimeV2() {
         boolean hasException = false;
         try {
-            DateLiteral literal = new DateLiteral("1997-10-7 00:00:00.123456", Type.DATETIMEV2);
+            DateLiteral literal = new DateLiteral("1997-10-7 00:00:00.123456", ScalarType.createDatetimeV2Type(6));
             Assert.assertEquals(1997, literal.getYear());
             Assert.assertEquals(123456, literal.getMicrosecond());
 
-            literal = new DateLiteral("2021-06-1 00:00:00.123456", Type.DATETIMEV2);
+            literal = new DateLiteral("2021-06-1 00:00:00.123456", ScalarType.createDatetimeV2Type(6));
             Assert.assertEquals(2021, literal.getYear());
             Assert.assertEquals(6, literal.getMonth());
             Assert.assertEquals(1, literal.getDay());
 
-            literal = new DateLiteral("2022-6-01 00:00:00.123456", Type.DATETIMEV2);
+            literal = new DateLiteral("2022-6-01 00:00:00.123456", ScalarType.createDatetimeV2Type(6));
             Assert.assertEquals(2022, literal.getYear());
             Assert.assertEquals(6, literal.getMonth());
             Assert.assertEquals(1, literal.getDay());
diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java
index 14d2478317..925a720591 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java
@@ -2012,10 +2012,10 @@ public class QueryPlanTest extends TestWithFeService {
         rewriteDateLiteralRuleTest.testWithIntFormatDate();
         rewriteDateLiteralRuleTest.testWithInvalidFormatDate();
         rewriteDateLiteralRuleTest.testWithStringFormatDate();
-        // rewriteDateLiteralRuleTest.testWithDoubleFormatDateV2();
-        // rewriteDateLiteralRuleTest.testWithIntFormatDateV2();
-        // rewriteDateLiteralRuleTest.testWithInvalidFormatDateV2();
-        // rewriteDateLiteralRuleTest.testWithStringFormatDateV2();
+        rewriteDateLiteralRuleTest.testWithDoubleFormatDateV2();
+        rewriteDateLiteralRuleTest.testWithIntFormatDateV2();
+        rewriteDateLiteralRuleTest.testWithInvalidFormatDateV2();
+        rewriteDateLiteralRuleTest.testWithStringFormatDateV2();
         rewriteDateLiteralRuleTest.after();
     }
     // --end--
diff --git a/fe/fe-core/src/test/java/org/apache/doris/rewrite/RewriteDateLiteralRuleTest.java b/fe/fe-core/src/test/java/org/apache/doris/rewrite/RewriteDateLiteralRuleTest.java
index 9f2d106bd9..40bc9bb2c3 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/rewrite/RewriteDateLiteralRuleTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/rewrite/RewriteDateLiteralRuleTest.java
@@ -64,13 +64,13 @@ public class RewriteDateLiteralRuleTest {
     public void testWithIntFormatDateV2() throws Exception {
         String query = "select * from " + DB_NAME + ".tb2 where k1 > 20210301";
         String planString = dorisAssert.query(query).explainQuery();
-        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 00:00:00.000000'"));
+        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 00:00:00'"));
         query = "select k1 > 20210301 from " + DB_NAME + ".tb2";
         planString = dorisAssert.query(query).explainQuery();
-        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 00:00:00.000000'"));
+        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 00:00:00'"));
         query = "select k1 > 20210301223344 from " + DB_NAME + ".tb2";
         planString = dorisAssert.query(query).explainQuery();
-        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 22:33:44.000000'"));
+        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 22:33:44'"));
     }
 
     public void testWithStringFormatDate() throws Exception {
@@ -122,47 +122,47 @@ public class RewriteDateLiteralRuleTest {
     public void testWithStringFormatDateV2() throws Exception {
         String query = "select * from " + DB_NAME + ".tb2 where k1 > '2021030112334455'";
         String planString = dorisAssert.query(query).explainQuery();
-        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 12:33:44.000000'"));
+        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 12:33:44.550000'"));
 
         query = "select k1 > '20210301' from " + DB_NAME + ".tb2";
         planString = dorisAssert.query(query).explainQuery();
-        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 00:00:00.000000'"));
+        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 00:00:00'"));
 
         query = "select k1 > '20210301233234.34' from " + DB_NAME + ".tb2";
         planString = dorisAssert.query(query).explainQuery();
-        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 23:32:34.000000'"));
+        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 23:32:34.340000'"));
 
         query = "select * from " + DB_NAME + ".tb2 where k1 > '2021-03-01'";
         planString = dorisAssert.query(query).explainQuery();
-        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 00:00:00.000000'"));
+        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 00:00:00'"));
 
         query = "select k1 > '2021-03-01 11:22:33' from " + DB_NAME + ".tb2";
         planString = dorisAssert.query(query).explainQuery();
         Assert.assertTrue(planString.contains("`k1` > '2021-03-01 11:22:33'"));
 
-        query = "select k1 > '2021-03-01  16:22:33' from " + DB_NAME + ".tb2";
+        query = "select k1 > '2021-03-01 16:22:33' from " + DB_NAME + ".tb2";
         planString = dorisAssert.query(query).explainQuery();
         Assert.assertTrue(planString.contains("`k1` > '2021-03-01 16:22:33'"));
 
         query = "select k1 > '2021-03-01 11:22' from " + DB_NAME + ".tb2";
         planString = dorisAssert.query(query).explainQuery();
-        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 11:22:00.000000'"));
+        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 11:22:00'"));
 
         query = "select k1 > '20210301T221133' from " + DB_NAME + ".tb2";
         planString = dorisAssert.query(query).explainQuery();
-        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 22:11:33.000000'"));
+        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 22:11:33'"));
 
-        query = "select k1 > '2021-03-01dd 11:22' from " + DB_NAME + ".tb2";
+        query = "select k1 > '2021-03-01 11:22' from " + DB_NAME + ".tb2";
         planString = dorisAssert.query(query).explainQuery();
-        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 00:00:00.000000'"));
+        Assert.assertTrue(planString.contains("`k1` > '2021-03-01 11:22:00'"));
 
         query = "select k1 > '80-03-01 11:22' from " + DB_NAME + ".tb2";
         planString = dorisAssert.query(query).explainQuery();
-        Assert.assertTrue(planString.contains("`k1` > '1980-03-01 11:22:00.000000'"));
+        Assert.assertTrue(planString.contains("`k1` > '1980-03-01 11:22:00'"));
 
         query = "select k1 > '12-03-01 11:22' from " + DB_NAME + ".tb2";
         planString = dorisAssert.query(query).explainQuery();
-        Assert.assertTrue(planString.contains("`k1` > '2012-03-01 11:22:00.000000'"));
+        Assert.assertTrue(planString.contains("`k1` > '2012-03-01 11:22:00'"));
     }
 
     public void testWithDoubleFormatDate() throws Exception {


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