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/23 08:08:04 UTC

[doris] branch master updated: [feature-wip] support datetimev2 (#11085)

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 babab5d535 [feature-wip] support datetimev2 (#11085)
babab5d535 is described below

commit babab5d535d27f58949c3c5f22a6ca759b697a31
Author: Gabriel <ga...@gmail.com>
AuthorDate: Sat Jul 23 16:07:59 2022 +0800

    [feature-wip] support datetimev2 (#11085)
---
 be/src/exec/olap_common.h                          |  28 +-
 be/src/exec/tablet_info.cpp                        |  11 +-
 be/src/exec/tablet_sink.cpp                        |   1 +
 be/src/exprs/anyval_util.cpp                       |   9 +-
 be/src/exprs/anyval_util.h                         |  33 +-
 be/src/exprs/binary_predicate.cpp                  |  21 +
 be/src/exprs/bloomfilter_predicate.h               |  29 +-
 be/src/exprs/cast_functions.cpp                    | 138 +++++-
 be/src/exprs/cast_functions.h                      |  29 ++
 be/src/exprs/expr.cpp                              |  13 +-
 be/src/exprs/expr.h                                |  15 +-
 be/src/exprs/expr_context.cpp                      |  20 +-
 be/src/exprs/expr_context.h                        |   1 +
 be/src/exprs/expr_value.h                          |   3 +-
 be/src/exprs/literal.cpp                           |  13 +-
 be/src/exprs/literal.h                             |   1 +
 be/src/exprs/runtime_filter.cpp                    |  46 +-
 be/src/exprs/scalar_fn_call.cpp                    |  13 +-
 be/src/exprs/scalar_fn_call.h                      |   1 +
 be/src/exprs/slot_ref.cpp                          |  21 +-
 be/src/exprs/slot_ref.h                            |   1 +
 be/src/olap/aggregate_func.cpp                     |   5 +
 be/src/olap/bloom_filter_predicate.cpp             |   3 +-
 be/src/olap/bloom_filter_predicate.h               |   3 +-
 be/src/olap/column_vector.cpp                      |   5 +
 be/src/olap/delete_handler.cpp                     |   1 +
 be/src/olap/field.h                                |   4 +-
 be/src/olap/key_coder.cpp                          |   1 +
 be/src/olap/key_coder.h                            |  34 ++
 be/src/olap/reader.cpp                             |  18 +
 be/src/olap/row_block2.cpp                         |  10 +
 .../olap/rowset/segment_v2/bitmap_index_writer.cpp |   3 +
 be/src/olap/rowset/segment_v2/encoding_info.cpp    |  18 +
 be/src/olap/rowset/segment_v2/segment_iterator.cpp |   3 +-
 be/src/olap/schema.cpp                             |   3 +
 be/src/olap/tablet_schema.cpp                      |   7 +
 be/src/olap/types.h                                | 113 ++++-
 be/src/olap/utils.cpp                              |   8 +-
 be/src/runtime/datetime_value.h                    |  10 +-
 be/src/runtime/primitive_type.cpp                  |   4 +-
 be/src/runtime/primitive_type.h                    |   6 +-
 be/src/runtime/raw_value.cpp                       |  59 ++-
 be/src/runtime/raw_value.h                         |  53 ++-
 be/src/runtime/type_limit.h                        |  28 +-
 be/src/runtime/types.cpp                           |   8 +-
 be/src/runtime/types.h                             |   3 +-
 be/src/udf/udf.h                                   |  26 ++
 be/src/util/binary_cast.hpp                        |  31 +-
 be/src/util/date_func.cpp                          |   8 +
 be/src/util/date_func.h                            |   1 +
 be/src/util/symbols_util.cpp                       |   4 +-
 .../aggregate_function_min_max.cpp                 |   4 +
 .../aggregate_function_window.h                    |   4 +
 be/src/vec/columns/column.h                        |   3 +
 be/src/vec/columns/column_nullable.h               |   1 +
 be/src/vec/columns/columns_number.h                |   4 +-
 be/src/vec/columns/predicate_column.h              |  11 +-
 be/src/vec/core/call_on_type_index.h               |   3 +
 be/src/vec/core/types.h                            |   1 +
 be/src/vec/data_types/data_type.cpp                |   2 +
 be/src/vec/data_types/data_type.h                  |   5 +-
 be/src/vec/data_types/data_type_date_time.h        |   5 +
 be/src/vec/data_types/data_type_factory.cpp        |   8 +
 be/src/vec/data_types/data_type_factory.hpp        |   1 +
 be/src/vec/data_types/data_type_time_v2.cpp        |  66 ++-
 be/src/vec/data_types/data_type_time_v2.h          |  25 +-
 be/src/vec/data_types/get_least_supertype.cpp      |  14 +-
 be/src/vec/exec/join/vhash_join_node.cpp           |   1 +
 be/src/vec/exec/vaggregation_node.cpp              |   1 +
 be/src/vec/exec/volap_scan_node.cpp                |  40 +-
 be/src/vec/exec/vschema_scan_node.cpp              |   6 +
 be/src/vec/exec/vset_operation_node.cpp            |   1 +
 be/src/vec/exprs/vliteral.cpp                      |  11 +-
 .../vec/functions/array/function_array_element.h   |   4 +
 be/src/vec/functions/array/function_array_index.h  |   6 +
 be/src/vec/functions/date_time_transforms.h        | 127 +++--
 be/src/vec/functions/function.h                    |  16 +-
 be/src/vec/functions/function_cast.h               |  93 +++-
 .../function_date_or_datetime_computation.cpp      | 189 +++++---
 .../function_date_or_datetime_computation.h        | 147 ++++--
 .../function_date_or_datetime_to_string.cpp        |  13 +-
 .../function_datetime_string_to_string.cpp         |   6 +-
 be/src/vec/functions/function_timestamp.cpp        |  11 +-
 be/src/vec/functions/time_of_function.cpp          |  43 +-
 be/src/vec/functions/to_time_function.cpp          |  79 +++-
 be/src/vec/io/io_helper.h                          |  21 +-
 be/src/vec/olap/olap_data_convertor.cpp            | 133 ++++--
 be/src/vec/olap/olap_data_convertor.h              |  97 +++-
 be/src/vec/runtime/vdatetime_value.cpp             | 515 +++++++++++++++++----
 be/src/vec/runtime/vdatetime_value.h               | 488 ++++++++++++++-----
 be/src/vec/runtime/vfile_result_writer.cpp         |  13 +-
 be/src/vec/sink/vmysql_result_writer.cpp           |  26 +-
 be/src/vec/sink/vmysql_result_writer.h             |   2 +-
 be/src/vec/sink/vmysql_table_writer.cpp            |  17 +-
 be/src/vec/utils/arrow_column_to_doris_column.cpp  |  53 ++-
 be/src/vec/utils/template_helpers.hpp              |   3 +-
 be/test/olap/delta_writer_test.cpp                 |  18 +-
 be/test/olap/in_list_predicate_test.cpp            |  10 +-
 be/test/vec/core/block_test.cpp                    |  12 +-
 be/test/vec/exprs/vexpr_test.cpp                   |   2 +-
 be/test/vec/function/function_test_util.cpp        |   6 +-
 be/test/vec/runtime/vdatetime_value_test.cpp       | 271 ++++++++++-
 .../java/org/apache/doris/analysis/Analyzer.java   |   2 +
 .../org/apache/doris/analysis/BinaryPredicate.java |  23 +-
 .../java/org/apache/doris/analysis/CastExpr.java   |   3 +
 .../org/apache/doris/analysis/DateLiteral.java     |  29 +-
 .../java/org/apache/doris/catalog/Function.java    |   6 +-
 .../java/org/apache/doris/catalog/FunctionSet.java |  13 -
 .../java/org/apache/doris/catalog/ScalarType.java  |  28 +-
 .../doris/rewrite/RewriteBinaryPredicatesRule.java |  60 +--
 .../RoundLiteralInBinaryPredicatesRule.java        | 145 ++++++
 gensrc/proto/internal_service.proto                |   7 +-
 112 files changed, 3179 insertions(+), 702 deletions(-)

diff --git a/be/src/exec/olap_common.h b/be/src/exec/olap_common.h
index d2aed8db68..fb303de5f6 100644
--- a/be/src/exec/olap_common.h
+++ b/be/src/exec/olap_common.h
@@ -30,6 +30,7 @@
 #include "runtime/primitive_type.h"
 #include "runtime/type_limit.h"
 #include "vec/io/io_helper.h"
+#include "vec/runtime/vdatetime_value.h"
 
 namespace doris {
 
@@ -51,6 +52,15 @@ std::string cast_to_string(T value, int scale) {
         return std::to_string(static_cast<int>(value));
     } else if constexpr (primitive_type == TYPE_LARGEINT) {
         return vectorized::int128_to_string(value);
+    } else if constexpr (primitive_type == TYPE_DATETIMEV2) {
+        doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> datetimev2_val =
+                static_cast<doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>>(
+                        value);
+        char buf[30];
+        datetimev2_val.to_string(buf);
+        std::stringstream ss;
+        ss << buf;
+        return ss.str();
     } else {
         return boost::lexical_cast<std::string>(value);
     }
@@ -352,14 +362,16 @@ private:
     bool _is_convertible;
 };
 
-using ColumnValueRangeType = std::variant<
-        ColumnValueRange<TYPE_TINYINT>, ColumnValueRange<TYPE_SMALLINT>, ColumnValueRange<TYPE_INT>,
-        ColumnValueRange<TYPE_BIGINT>, ColumnValueRange<TYPE_LARGEINT>, ColumnValueRange<TYPE_CHAR>,
-        ColumnValueRange<TYPE_VARCHAR>, ColumnValueRange<TYPE_STRING>, ColumnValueRange<TYPE_DATE>,
-        ColumnValueRange<TYPE_DATEV2>, ColumnValueRange<TYPE_DATETIME>,
-        ColumnValueRange<TYPE_DECIMALV2>, ColumnValueRange<TYPE_BOOLEAN>,
-        ColumnValueRange<TYPE_HLL>, ColumnValueRange<TYPE_DECIMAL32>,
-        ColumnValueRange<TYPE_DECIMAL64>, ColumnValueRange<TYPE_DECIMAL128>>;
+using ColumnValueRangeType =
+        std::variant<ColumnValueRange<TYPE_TINYINT>, ColumnValueRange<TYPE_SMALLINT>,
+                     ColumnValueRange<TYPE_INT>, ColumnValueRange<TYPE_BIGINT>,
+                     ColumnValueRange<TYPE_LARGEINT>, ColumnValueRange<TYPE_CHAR>,
+                     ColumnValueRange<TYPE_VARCHAR>, ColumnValueRange<TYPE_STRING>,
+                     ColumnValueRange<TYPE_DATE>, ColumnValueRange<TYPE_DATEV2>,
+                     ColumnValueRange<TYPE_DATETIME>, ColumnValueRange<TYPE_DATETIMEV2>,
+                     ColumnValueRange<TYPE_DECIMALV2>, ColumnValueRange<TYPE_BOOLEAN>,
+                     ColumnValueRange<TYPE_HLL>, ColumnValueRange<TYPE_DECIMAL32>,
+                     ColumnValueRange<TYPE_DECIMAL64>, ColumnValueRange<TYPE_DECIMAL128>>;
 
 template <PrimitiveType primitive_type>
 const typename ColumnValueRange<primitive_type>::CppType
diff --git a/be/src/exec/tablet_info.cpp b/be/src/exec/tablet_info.cpp
index cdaa2a9a61..0fd5e24032 100644
--- a/be/src/exec/tablet_info.cpp
+++ b/be/src/exec/tablet_info.cpp
@@ -584,7 +584,16 @@ Status VOlapTablePartitionParam::_create_partition_key(const TExprNode& t_expr,
     switch (t_expr.node_type) {
     case TExprNodeType::DATE_LITERAL: {
         if (TypeDescriptor::from_thrift(t_expr.type).is_date_v2_type()) {
-            vectorized::DateV2Value dt;
+            vectorized::DateV2Value<doris::vectorized::DateV2ValueType> dt;
+            if (!dt.from_date_str(t_expr.date_literal.value.c_str(),
+                                  t_expr.date_literal.value.size())) {
+                std::stringstream ss;
+                ss << "invalid date literal in partition column, date=" << t_expr.date_literal;
+                return Status::InternalError(ss.str());
+            }
+            column->insert_data(reinterpret_cast<const char*>(&dt), 0);
+        } else if (TypeDescriptor::from_thrift(t_expr.type).is_datetime_v2_type()) {
+            vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> dt;
             if (!dt.from_date_str(t_expr.date_literal.value.c_str(),
                                   t_expr.date_literal.value.size())) {
                 std::stringstream ss;
diff --git a/be/src/exec/tablet_sink.cpp b/be/src/exec/tablet_sink.cpp
index 7c67a060ed..6254d906bd 100644
--- a/be/src/exec/tablet_sink.cpp
+++ b/be/src/exec/tablet_sink.cpp
@@ -778,6 +778,7 @@ Status OlapTableSink::prepare(RuntimeState* state) {
         case TYPE_DATE:
         case TYPE_DATETIME:
         case TYPE_DATEV2:
+        case TYPE_DATETIMEV2:
         case TYPE_HLL:
         case TYPE_OBJECT:
         case TYPE_STRING:
diff --git a/be/src/exprs/anyval_util.cpp b/be/src/exprs/anyval_util.cpp
index a53031fc12..1099c1297e 100644
--- a/be/src/exprs/anyval_util.cpp
+++ b/be/src/exprs/anyval_util.cpp
@@ -38,6 +38,7 @@ using doris_udf::DateTimeVal;
 using doris_udf::StringVal;
 using doris_udf::AnyVal;
 using doris_udf::DateV2Val;
+using doris_udf::DateTimeV2Val;
 
 Status allocate_any_val(RuntimeState* state, MemPool* pool, const TypeDescriptor& type,
                         const std::string& mem_limit_exceeded_msg, AnyVal** result) {
@@ -109,8 +110,10 @@ AnyVal* create_any_val(ObjectPool* pool, const TypeDescriptor& type) {
     case TYPE_DATEV2:
         return pool->add(new DateV2Val);
 
-    case TYPE_DATETIME:
     case TYPE_DATETIMEV2:
+        return pool->add(new DateTimeV2Val);
+
+    case TYPE_DATETIME:
         return pool->add(new DateTimeVal);
 
     case TYPE_ARRAY:
@@ -154,12 +157,14 @@ FunctionContext::TypeDesc AnyValUtil::column_type_to_type_desc(const TypeDescrip
         out.type = FunctionContext::TYPE_DATE;
         break;
     case TYPE_DATETIME:
-    case TYPE_DATETIMEV2:
         out.type = FunctionContext::TYPE_DATETIME;
         break;
     case TYPE_DATEV2:
         out.type = FunctionContext::TYPE_DATEV2;
         break;
+    case TYPE_DATETIMEV2:
+        out.type = FunctionContext::TYPE_DATETIMEV2;
+        break;
     case TYPE_DECIMAL32:
         out.type = FunctionContext::TYPE_DECIMAL32;
         out.precision = type.precision;
diff --git a/be/src/exprs/anyval_util.h b/be/src/exprs/anyval_util.h
index 207d610f25..d30565e0b9 100644
--- a/be/src/exprs/anyval_util.h
+++ b/be/src/exprs/anyval_util.h
@@ -185,7 +185,14 @@ public:
             return val;
         } else if constexpr (std::is_same_v<Val, DateV2Val>) {
             DateV2Val val;
-            type_limit<doris::vectorized::DateV2Value>::min().to_datev2_val(&val);
+            type_limit<doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>>::min()
+                    .to_datev2_val(&val);
+            return val;
+        } else if constexpr (std::is_same_v<Val, DateTimeV2Val>) {
+            DateTimeV2Val val;
+            type_limit<
+                    doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>>::min()
+                    .to_datetimev2_val(&val);
             return val;
         } else {
             return Val(type_limit<decltype(std::declval<Val>().val)>::min());
@@ -212,7 +219,14 @@ public:
             return val;
         } else if constexpr (std::is_same_v<Val, DateV2Val>) {
             DateV2Val val;
-            type_limit<doris::vectorized::DateV2Value>::max().to_datev2_val(&val);
+            type_limit<doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>>::max()
+                    .to_datev2_val(&val);
+            return val;
+        } else if constexpr (std::is_same_v<Val, DateTimeV2Val>) {
+            DateTimeV2Val val;
+            type_limit<
+                    doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>>::max()
+                    .to_datetimev2_val(&val);
             return val;
         } else {
             return Val(type_limit<decltype(std::declval<Val>().val)>::max());
@@ -256,6 +270,8 @@ public:
 
         case TYPE_DATEV2:
             return sizeof(doris_udf::DateV2Val);
+        case TYPE_DATETIMEV2:
+            return sizeof(doris_udf::DateTimeV2Val);
         case TYPE_DATE:
         case TYPE_DATETIME:
             return sizeof(doris_udf::DateTimeVal);
@@ -303,6 +319,8 @@ public:
             return alignof(DateTimeVal);
         case TYPE_DATEV2:
             return alignof(DateV2Val);
+        case TYPE_DATETIMEV2:
+            return alignof(DateTimeV2Val);
         case TYPE_DECIMALV2:
             return alignof(DecimalV2Val);
         case TYPE_ARRAY:
@@ -425,8 +443,15 @@ public:
                     reinterpret_cast<doris_udf::DateTimeVal*>(dst));
 
         case TYPE_DATEV2:
-            reinterpret_cast<const doris::vectorized::DateV2Value*>(slot)->to_datev2_val(
-                    reinterpret_cast<doris_udf::DateV2Val*>(dst));
+            reinterpret_cast<
+                    const doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*>(slot)
+                    ->to_datev2_val(reinterpret_cast<doris_udf::DateV2Val*>(dst));
+            return;
+        case TYPE_DATETIMEV2:
+            reinterpret_cast<
+                    const doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>*>(
+                    slot)
+                    ->to_datev2_val(reinterpret_cast<doris_udf::DateV2Val*>(dst));
             return;
         case TYPE_ARRAY:
             reinterpret_cast<const CollectionValue*>(slot)->to_collection_val(
diff --git a/be/src/exprs/binary_predicate.cpp b/be/src/exprs/binary_predicate.cpp
index 533f2a6d82..f067577166 100644
--- a/be/src/exprs/binary_predicate.cpp
+++ b/be/src/exprs/binary_predicate.cpp
@@ -37,9 +37,11 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) {
             return new EqSmallIntValPred(node);
         case TPrimitiveType::INT:
         case TPrimitiveType::DECIMAL32:
+        case TPrimitiveType::DATEV2:
             return new EqIntValPred(node);
         case TPrimitiveType::BIGINT:
         case TPrimitiveType::DECIMAL64:
+        case TPrimitiveType::DATETIMEV2:
             return new EqBigIntValPred(node);
         case TPrimitiveType::LARGEINT:
         case TPrimitiveType::DECIMAL128:
@@ -58,6 +60,7 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) {
         case TPrimitiveType::DECIMALV2:
             return new EqDecimalV2ValPred(node);
         default:
+            DCHECK(false) << "Invalid type: " << node.child_type;
             return nullptr;
         }
     }
@@ -71,9 +74,11 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) {
             return new NeSmallIntValPred(node);
         case TPrimitiveType::INT:
         case TPrimitiveType::DECIMAL32:
+        case TPrimitiveType::DATEV2:
             return new NeIntValPred(node);
         case TPrimitiveType::BIGINT:
         case TPrimitiveType::DECIMAL64:
+        case TPrimitiveType::DATETIMEV2:
             return new NeBigIntValPred(node);
         case TPrimitiveType::LARGEINT:
         case TPrimitiveType::DECIMAL128:
@@ -92,6 +97,7 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) {
         case TPrimitiveType::DECIMALV2:
             return new NeDecimalV2ValPred(node);
         default:
+            DCHECK(false) << "Invalid type!";
             return nullptr;
         }
     }
@@ -105,9 +111,11 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) {
             return new LtSmallIntValPred(node);
         case TPrimitiveType::INT:
         case TPrimitiveType::DECIMAL32:
+        case TPrimitiveType::DATEV2:
             return new LtIntValPred(node);
         case TPrimitiveType::BIGINT:
         case TPrimitiveType::DECIMAL64:
+        case TPrimitiveType::DATETIMEV2:
             return new LtBigIntValPred(node);
         case TPrimitiveType::LARGEINT:
         case TPrimitiveType::DECIMAL128:
@@ -126,6 +134,7 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) {
         case TPrimitiveType::DECIMALV2:
             return new LtDecimalV2ValPred(node);
         default:
+            DCHECK(false) << "Invalid type!";
             return nullptr;
         }
     }
@@ -139,9 +148,11 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) {
             return new LeSmallIntValPred(node);
         case TPrimitiveType::INT:
         case TPrimitiveType::DECIMAL32:
+        case TPrimitiveType::DATEV2:
             return new LeIntValPred(node);
         case TPrimitiveType::BIGINT:
         case TPrimitiveType::DECIMAL64:
+        case TPrimitiveType::DATETIMEV2:
             return new LeBigIntValPred(node);
         case TPrimitiveType::LARGEINT:
         case TPrimitiveType::DECIMAL128:
@@ -160,6 +171,7 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) {
         case TPrimitiveType::DECIMALV2:
             return new LeDecimalV2ValPred(node);
         default:
+            DCHECK(false) << "Invalid type!";
             return nullptr;
         }
     }
@@ -173,9 +185,11 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) {
             return new GtSmallIntValPred(node);
         case TPrimitiveType::INT:
         case TPrimitiveType::DECIMAL32:
+        case TPrimitiveType::DATEV2:
             return new GtIntValPred(node);
         case TPrimitiveType::BIGINT:
         case TPrimitiveType::DECIMAL64:
+        case TPrimitiveType::DATETIMEV2:
             return new GtBigIntValPred(node);
         case TPrimitiveType::LARGEINT:
         case TPrimitiveType::DECIMAL128:
@@ -194,6 +208,7 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) {
         case TPrimitiveType::DECIMALV2:
             return new GtDecimalV2ValPred(node);
         default:
+            DCHECK(false) << "Invalid type!";
             return nullptr;
         }
     }
@@ -207,9 +222,11 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) {
             return new GeSmallIntValPred(node);
         case TPrimitiveType::INT:
         case TPrimitiveType::DECIMAL32:
+        case TPrimitiveType::DATEV2:
             return new GeIntValPred(node);
         case TPrimitiveType::BIGINT:
         case TPrimitiveType::DECIMAL64:
+        case TPrimitiveType::DATETIMEV2:
             return new GeBigIntValPred(node);
         case TPrimitiveType::LARGEINT:
         case TPrimitiveType::DECIMAL128:
@@ -228,6 +245,7 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) {
         case TPrimitiveType::DECIMALV2:
             return new GeDecimalV2ValPred(node);
         default:
+            DCHECK(false) << "Invalid type!";
             return nullptr;
         }
     }
@@ -241,9 +259,11 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) {
             return new EqForNullSmallIntValPred(node);
         case TPrimitiveType::INT:
         case TPrimitiveType::DECIMAL32:
+        case TPrimitiveType::DATEV2:
             return new EqForNullIntValPred(node);
         case TPrimitiveType::BIGINT:
         case TPrimitiveType::DECIMAL64:
+        case TPrimitiveType::DATETIMEV2:
             return new EqForNullBigIntValPred(node);
         case TPrimitiveType::LARGEINT:
         case TPrimitiveType::DECIMAL128:
@@ -262,6 +282,7 @@ Expr* BinaryPredicate::from_thrift(const TExprNode& node) {
         case TPrimitiveType::DECIMALV2:
             return new EqForNullDecimalV2ValPred(node);
         default:
+            DCHECK(false) << "Invalid type!";
             return nullptr;
         }
     }
diff --git a/be/src/exprs/bloomfilter_predicate.h b/be/src/exprs/bloomfilter_predicate.h
index 479d193fdc..7f420a2d8e 100644
--- a/be/src/exprs/bloomfilter_predicate.h
+++ b/be/src/exprs/bloomfilter_predicate.h
@@ -238,11 +238,29 @@ struct DateFindOp : public CommonFindOp<DateTimeValue, BloomFilterAdaptor> {
 };
 
 template <class BloomFilterAdaptor>
-struct DateV2FindOp : public CommonFindOp<doris::vectorized::DateV2Value, BloomFilterAdaptor> {
+struct DateV2FindOp
+        : public CommonFindOp<doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>,
+                              BloomFilterAdaptor> {
     bool find_olap_engine(const BloomFilterAdaptor& bloom_filter, const void* data) const {
-        doris::vectorized::DateV2Value value;
+        doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> value;
         value.from_date(*reinterpret_cast<const uint32_t*>(data));
-        return bloom_filter.test(Slice((char*)&value, sizeof(doris::vectorized::DateV2Value)));
+        return bloom_filter.test(
+                Slice((char*)&value,
+                      sizeof(doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>)));
+    }
+};
+
+template <class BloomFilterAdaptor>
+struct DateTimeV2FindOp
+        : public CommonFindOp<
+                  doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>,
+                  BloomFilterAdaptor> {
+    bool find_olap_engine(const BloomFilterAdaptor& bloom_filter, const void* data) const {
+        doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> value;
+        value.from_datetime(*reinterpret_cast<const uint64_t*>(data));
+        return bloom_filter.test(Slice(
+                (char*)&value,
+                sizeof(doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>)));
     }
 };
 
@@ -278,6 +296,11 @@ struct BloomFilterTypeTraits<TYPE_DATEV2, BloomFilterAdaptor> {
     using FindOp = DateV2FindOp<BloomFilterAdaptor>;
 };
 
+template <class BloomFilterAdaptor>
+struct BloomFilterTypeTraits<TYPE_DATETIMEV2, BloomFilterAdaptor> {
+    using FindOp = DateTimeV2FindOp<BloomFilterAdaptor>;
+};
+
 template <class BloomFilterAdaptor>
 struct BloomFilterTypeTraits<TYPE_DATETIME, BloomFilterAdaptor> {
     using FindOp = DateTimeFindOp<BloomFilterAdaptor>;
diff --git a/be/src/exprs/cast_functions.cpp b/be/src/exprs/cast_functions.cpp
index 16a2b47a6a..2b6d4c8a08 100644
--- a/be/src/exprs/cast_functions.cpp
+++ b/be/src/exprs/cast_functions.cpp
@@ -34,6 +34,15 @@
 #include "vec/data_types/data_type_decimal.h"
 #include "vec/runtime/vdatetime_value.h"
 
+namespace doris::vectorized {
+template <>
+void doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>::convert_date_v2_to_dt(
+        doris::DateTimeValue* dt);
+template <>
+void doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>::convert_date_v2_to_dt(
+        doris::DateTimeValue* dt);
+} // namespace doris::vectorized
+
 namespace doris {
 
 void CastFunctions::init() {}
@@ -614,7 +623,22 @@ DateTimeVal CastFunctions::cast_to_date_val(FunctionContext* ctx, const DateV2Va
     if (val.is_null) {
         return DateTimeVal::null();
     }
-    vectorized::DateV2Value datev2_val = vectorized::DateV2Value::from_datev2_val(val);
+    vectorized::DateV2Value<doris::vectorized::DateV2ValueType> datev2_val =
+            vectorized::DateV2Value<doris::vectorized::DateV2ValueType>::from_datev2_val(val);
+    DateTimeValue datetime_value;
+    datev2_val.convert_date_v2_to_dt(&datetime_value);
+    DateTimeVal result;
+    datetime_value.to_datetime_val(&result);
+    return result;
+}
+
+DateTimeVal CastFunctions::cast_to_date_val(FunctionContext* ctx, const DateTimeV2Val& val) {
+    if (val.is_null) {
+        return DateTimeVal::null();
+    }
+    vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> datev2_val =
+            vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>::from_datetimev2_val(
+                    val);
     DateTimeValue datetime_value;
     datev2_val.convert_date_v2_to_dt(&datetime_value);
     DateTimeVal result;
@@ -626,7 +650,23 @@ DateTimeVal CastFunctions::cast_to_datetime_val(FunctionContext* ctx, const Date
     if (val.is_null) {
         return DateTimeVal::null();
     }
-    vectorized::DateV2Value datev2_val = vectorized::DateV2Value::from_datev2_val(val);
+    vectorized::DateV2Value<doris::vectorized::DateV2ValueType> datev2_val =
+            vectorized::DateV2Value<doris::vectorized::DateV2ValueType>::from_datev2_val(val);
+    DateTimeValue datetime_value;
+    datev2_val.convert_date_v2_to_dt(&datetime_value);
+    datetime_value.set_type(TYPE_DATETIME);
+    DateTimeVal result;
+    datetime_value.to_datetime_val(&result);
+    return result;
+}
+
+DateTimeVal CastFunctions::cast_to_datetime_val(FunctionContext* ctx, const DateTimeV2Val& val) {
+    if (val.is_null) {
+        return DateTimeVal::null();
+    }
+    vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> datev2_val =
+            vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>::from_datetimev2_val(
+                    val);
     DateTimeValue datetime_value;
     datev2_val.convert_date_v2_to_dt(&datetime_value);
     datetime_value.set_type(TYPE_DATETIME);
@@ -638,7 +678,7 @@ DateTimeVal CastFunctions::cast_to_datetime_val(FunctionContext* ctx, const Date
 #define CAST_TO_DATEV2(from_type)                                                             \
     DateV2Val CastFunctions::cast_to_datev2_val(FunctionContext* ctx, const from_type& val) { \
         if (val.is_null) return DateV2Val::null();                                            \
-        doris::vectorized::DateV2Value date_value;                                            \
+        doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> date_value;        \
         if (!date_value.from_date_int64(val.val)) return DateV2Val::null();                   \
         DateV2Val result;                                                                     \
         date_value.to_datev2_val(&result);                                                    \
@@ -656,6 +696,28 @@ DateTimeVal CastFunctions::cast_to_datetime_val(FunctionContext* ctx, const Date
 
 CAST_NUMERIC_TYPES_TO_DATEV2();
 
+#define CAST_TO_DATETIMEV2(from_type)                                                      \
+    DateTimeV2Val CastFunctions::cast_to_datetimev2_val(FunctionContext* ctx,              \
+                                                        const from_type& val) {            \
+        if (val.is_null) return DateTimeV2Val::null();                                     \
+        doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> date_value; \
+        if (!date_value.from_date_int64(val.val)) return DateTimeV2Val::null();            \
+        DateTimeV2Val result;                                                              \
+        date_value.to_datetimev2_val(&result);                                             \
+        return result;                                                                     \
+    }
+
+#define CAST_NUMERIC_TYPES_TO_DATETIMEV2() \
+    CAST_TO_DATETIMEV2(TinyIntVal);        \
+    CAST_TO_DATETIMEV2(SmallIntVal);       \
+    CAST_TO_DATETIMEV2(IntVal);            \
+    CAST_TO_DATETIMEV2(BigIntVal);         \
+    CAST_TO_DATETIMEV2(LargeIntVal);       \
+    CAST_TO_DATETIMEV2(FloatVal);          \
+    CAST_TO_DATETIMEV2(DoubleVal);
+
+CAST_NUMERIC_TYPES_TO_DATETIMEV2();
+
 DateV2Val CastFunctions::cast_to_datev2_val(FunctionContext* ctx, const DateV2Val& val) {
     if (val.is_null) {
         return DateV2Val::null();
@@ -667,7 +729,7 @@ DateV2Val CastFunctions::cast_to_datev2_val(FunctionContext* ctx, const StringVa
     if (val.is_null) {
         return DateV2Val::null();
     }
-    doris::vectorized::DateV2Value date_value;
+    doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> date_value;
     if (!date_value.from_date_str((char*)val.ptr, val.len)) {
         return DateV2Val::null();
     }
@@ -683,13 +745,79 @@ DateV2Val CastFunctions::cast_to_datev2_val(FunctionContext* ctx, const DateTime
     }
     vectorized::VecDateTimeValue date_value = vectorized::VecDateTimeValue::from_datetime_val(val);
 
-    doris::vectorized::DateV2Value datev2_val;
+    doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> datev2_val;
     datev2_val.from_date(date_value.to_date_v2());
     DateV2Val result;
     datev2_val.to_datev2_val(&result);
     return result;
 }
 
+DateV2Val CastFunctions::cast_to_datev2_val(FunctionContext* ctx, const DateTimeV2Val& val) {
+    if (val.is_null) {
+        return DateV2Val::null();
+    }
+    doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> datetime_value =
+            vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>::from_datetimev2_val(
+                    val);
+
+    doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> datev2_val;
+    datev2_val.set_date_uint32(datetime_value.to_date_int_val() >>
+                               doris::vectorized::TIME_PART_LENGTH);
+    DateV2Val result;
+    datev2_val.to_datev2_val(&result);
+    return result;
+}
+
+DateTimeV2Val CastFunctions::cast_to_datetimev2_val(FunctionContext* ctx,
+                                                    const DateTimeV2Val& val) {
+    if (val.is_null) {
+        return DateTimeV2Val::null();
+    }
+    return val;
+}
+
+DateTimeV2Val CastFunctions::cast_to_datetimev2_val(FunctionContext* ctx, const StringVal& val) {
+    if (val.is_null) {
+        return DateTimeV2Val::null();
+    }
+    doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> date_value;
+    if (!date_value.from_date_str((char*)val.ptr, val.len)) {
+        return DateTimeV2Val::null();
+    }
+    // Return null if 'val' did not parse
+    DateTimeV2Val result;
+    date_value.to_datetimev2_val(&result);
+    return result;
+}
+
+DateTimeV2Val CastFunctions::cast_to_datetimev2_val(FunctionContext* ctx, const DateTimeVal& val) {
+    if (val.is_null) {
+        return DateTimeV2Val::null();
+    }
+    vectorized::VecDateTimeValue date_value = vectorized::VecDateTimeValue::from_datetime_val(val);
+
+    doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> datev2_val;
+    datev2_val.from_date(date_value.to_datetime_v2());
+    DateTimeV2Val result;
+    datev2_val.to_datetimev2_val(&result);
+    return result;
+}
+
+DateTimeV2Val CastFunctions::cast_to_datetimev2_val(FunctionContext* ctx, const DateV2Val& val) {
+    if (val.is_null) {
+        return DateTimeV2Val::null();
+    }
+    doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> date_value =
+            vectorized::DateV2Value<doris::vectorized::DateV2ValueType>::from_datev2_val(val);
+
+    doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> datetimev2_val;
+    datetimev2_val.set_datetime_uint64((uint64_t)date_value.to_date_int_val()
+                                       << doris::vectorized::TIME_PART_LENGTH);
+    DateTimeV2Val result;
+    datetimev2_val.to_datetimev2_val(&result);
+    return result;
+}
+
 CollectionVal CastFunctions::cast_to_array_val(FunctionContext* context, const StringVal& val) {
     CollectionVal array_val;
     Status status = ArrayParser::parse(array_val, context, val);
diff --git a/be/src/exprs/cast_functions.h b/be/src/exprs/cast_functions.h
index 5beecf8e62..9281b13293 100644
--- a/be/src/exprs/cast_functions.h
+++ b/be/src/exprs/cast_functions.h
@@ -130,6 +130,8 @@ public:
     static DateTimeVal cast_to_datetime_val(FunctionContext* context, const StringVal& val);
     static DateTimeVal cast_to_datetime_val(FunctionContext* context,
                                             const doris_udf::DateV2Val& val);
+    static DateTimeVal cast_to_datetime_val(FunctionContext* context,
+                                            const doris_udf::DateTimeV2Val& val);
 
     static DateTimeVal cast_to_date_val(FunctionContext* context, const TinyIntVal& val);
     static DateTimeVal cast_to_date_val(FunctionContext* context, const SmallIntVal& val);
@@ -141,6 +143,8 @@ public:
     static DateTimeVal cast_to_date_val(FunctionContext* context, const DateTimeVal& val);
     static DateTimeVal cast_to_date_val(FunctionContext* context, const StringVal& val);
     static DateTimeVal cast_to_date_val(FunctionContext* context, const doris_udf::DateV2Val& val);
+    static DateTimeVal cast_to_date_val(FunctionContext* context,
+                                        const doris_udf::DateTimeV2Val& val);
 
     static doris_udf::DateV2Val cast_to_datev2_val(FunctionContext* context, const TinyIntVal& val);
     static doris_udf::DateV2Val cast_to_datev2_val(FunctionContext* context,
@@ -156,6 +160,31 @@ public:
     static doris_udf::DateV2Val cast_to_datev2_val(FunctionContext* context, const StringVal& val);
     static doris_udf::DateV2Val cast_to_datev2_val(FunctionContext* context,
                                                    const DateTimeVal& val);
+    static doris_udf::DateV2Val cast_to_datev2_val(FunctionContext* context,
+                                                   const doris_udf::DateTimeV2Val& val);
+
+    static doris_udf::DateTimeV2Val cast_to_datetimev2_val(FunctionContext* context,
+                                                           const TinyIntVal& val);
+    static doris_udf::DateTimeV2Val cast_to_datetimev2_val(FunctionContext* context,
+                                                           const SmallIntVal& val);
+    static doris_udf::DateTimeV2Val cast_to_datetimev2_val(FunctionContext* context,
+                                                           const IntVal& val);
+    static doris_udf::DateTimeV2Val cast_to_datetimev2_val(FunctionContext* context,
+                                                           const BigIntVal& val);
+    static doris_udf::DateTimeV2Val cast_to_datetimev2_val(FunctionContext* context,
+                                                           const LargeIntVal& val);
+    static doris_udf::DateTimeV2Val cast_to_datetimev2_val(FunctionContext* context,
+                                                           const FloatVal& val);
+    static doris_udf::DateTimeV2Val cast_to_datetimev2_val(FunctionContext* context,
+                                                           const DoubleVal& val);
+    static doris_udf::DateTimeV2Val cast_to_datetimev2_val(FunctionContext* context,
+                                                           const doris_udf::DateV2Val& val);
+    static doris_udf::DateTimeV2Val cast_to_datetimev2_val(FunctionContext* context,
+                                                           const StringVal& val);
+    static doris_udf::DateTimeV2Val cast_to_datetimev2_val(FunctionContext* context,
+                                                           const DateTimeVal& val);
+    static doris_udf::DateTimeV2Val cast_to_datetimev2_val(FunctionContext* context,
+                                                           const doris_udf::DateTimeV2Val& val);
 
 #define DECLARE_CAST_TO_DECIMAL(width)                                                             \
     static Decimal##width##Val cast_to_decimal##width##_val(FunctionContext*, const TinyIntVal&);  \
diff --git a/be/src/exprs/expr.cpp b/be/src/exprs/expr.cpp
index 40224094ce..f113733597 100644
--- a/be/src/exprs/expr.cpp
+++ b/be/src/exprs/expr.cpp
@@ -696,8 +696,7 @@ doris_udf::AnyVal* Expr::get_const_val(ExprContext* context) {
         break;
     }
     case TYPE_DATE:
-    case TYPE_DATETIME:
-    case TYPE_DATETIMEV2: {
+    case TYPE_DATETIME: {
         _constant_val.reset(new DateTimeVal(get_datetime_val(context, nullptr)));
         break;
     }
@@ -707,6 +706,11 @@ doris_udf::AnyVal* Expr::get_const_val(ExprContext* context) {
         break;
     }
 
+    case TYPE_DATETIMEV2: {
+        _constant_val.reset(new DateTimeV2Val(get_datetimev2_val(context, nullptr)));
+        break;
+    }
+
     case TYPE_DECIMALV2: {
         _constant_val.reset(new DecimalV2Val(get_decimalv2_val(context, nullptr)));
         break;
@@ -821,6 +825,11 @@ DateV2Val Expr::get_datev2_val(ExprContext* context, TupleRow* row) {
     return val;
 }
 
+DateTimeV2Val Expr::get_datetimev2_val(ExprContext* context, TupleRow* row) {
+    DateTimeV2Val val;
+    return val;
+}
+
 DecimalV2Val Expr::get_decimalv2_val(ExprContext* context, TupleRow* row) {
     DecimalV2Val val;
     return val;
diff --git a/be/src/exprs/expr.h b/be/src/exprs/expr.h
index 2beb57c732..f0c802b4a0 100644
--- a/be/src/exprs/expr.h
+++ b/be/src/exprs/expr.h
@@ -106,6 +106,7 @@ public:
     virtual DateTimeVal get_datetime_val(ExprContext* context, TupleRow*);
     virtual DecimalV2Val get_decimalv2_val(ExprContext* context, TupleRow*);
     virtual DateV2Val get_datev2_val(ExprContext* context, TupleRow*);
+    virtual DateTimeV2Val get_datetimev2_val(ExprContext* context, TupleRow*);
     virtual CollectionVal get_array_val(ExprContext* context, TupleRow*);
 
     virtual Decimal32Val get_decimal32_val(ExprContext* context, TupleRow*);
@@ -516,7 +517,8 @@ Status create_texpr_literal_node(const void* data, TExprNode* node, int precisio
             (*node).__set_type(create_type_desc(PrimitiveType::TYPE_TIME));
         }
     } else if constexpr (T == TYPE_DATEV2) {
-        auto origin_value = reinterpret_cast<const doris::vectorized::DateV2Value*>(data);
+        auto origin_value = reinterpret_cast<
+                const doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*>(data);
         TDateLiteral date_literal;
         char convert_buffer[30];
         origin_value->to_string(convert_buffer);
@@ -524,6 +526,17 @@ Status create_texpr_literal_node(const void* data, TExprNode* node, int precisio
         (*node).__set_date_literal(date_literal);
         (*node).__set_node_type(TExprNodeType::DATE_LITERAL);
         (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DATEV2));
+    } else if constexpr (T == TYPE_DATETIMEV2) {
+        auto origin_value = reinterpret_cast<
+                const doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>*>(
+                data);
+        TDateLiteral date_literal;
+        char convert_buffer[30];
+        origin_value->to_string(convert_buffer);
+        date_literal.__set_value(convert_buffer);
+        (*node).__set_date_literal(date_literal);
+        (*node).__set_node_type(TExprNodeType::DATE_LITERAL);
+        (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DATETIMEV2));
     } else if constexpr (T == TYPE_DECIMALV2) {
         auto origin_value = reinterpret_cast<const DecimalV2Value*>(data);
         (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL);
diff --git a/be/src/exprs/expr_context.cpp b/be/src/exprs/expr_context.cpp
index 8cd8fe7c1c..a76fbb936e 100644
--- a/be/src/exprs/expr_context.cpp
+++ b/be/src/exprs/expr_context.cpp
@@ -246,8 +246,7 @@ void* ExprContext::get_value(Expr* e, TupleRow* row, int precision, int scale) {
         return &_result.string_val;
     }
     case TYPE_DATE:
-    case TYPE_DATETIME:
-    case TYPE_DATETIMEV2: {
+    case TYPE_DATETIME: {
         doris_udf::DateTimeVal v = e->get_datetime_val(this, row);
         if (v.is_null) {
             return nullptr;
@@ -260,9 +259,20 @@ void* ExprContext::get_value(Expr* e, TupleRow* row, int precision, int scale) {
         if (v.is_null) {
             return nullptr;
         }
-        _result.datev2_val = doris::vectorized::DateV2Value::from_datev2_val(v);
+        _result.datev2_val =
+                doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>::from_datev2_val(
+                        v);
         return &_result.datev2_val;
     }
+    case TYPE_DATETIMEV2: {
+        doris_udf::DateTimeV2Val v = e->get_datetimev2_val(this, row);
+        if (v.is_null) {
+            return nullptr;
+        }
+        _result.datetimev2_val = doris::vectorized::DateV2Value<
+                doris::vectorized::DateTimeV2ValueType>::from_datetimev2_val(v);
+        return &_result.datetimev2_val;
+    }
     case TYPE_DECIMALV2: {
         DecimalV2Val v = e->get_decimalv2_val(this, row);
         if (v.is_null) {
@@ -371,6 +381,10 @@ DateV2Val ExprContext::get_datev2_val(TupleRow* row) {
     return _root->get_datev2_val(this, row);
 }
 
+DateTimeV2Val ExprContext::get_datetimev2_val(TupleRow* row) {
+    return _root->get_datetimev2_val(this, row);
+}
+
 DecimalV2Val ExprContext::get_decimalv2_val(TupleRow* row) {
     return _root->get_decimalv2_val(this, row);
 }
diff --git a/be/src/exprs/expr_context.h b/be/src/exprs/expr_context.h
index 362aa53877..81cda05361 100644
--- a/be/src/exprs/expr_context.h
+++ b/be/src/exprs/expr_context.h
@@ -124,6 +124,7 @@ public:
     // ArrayVal GetArrayVal(TupleRow* row);
     DateTimeVal get_datetime_val(TupleRow* row);
     DateV2Val get_datev2_val(TupleRow* row);
+    DateTimeV2Val get_datetimev2_val(TupleRow* row);
     DecimalV2Val get_decimalv2_val(TupleRow* row);
 
     /// Frees all local allocations made by fn_contexts_. This can be called when result
diff --git a/be/src/exprs/expr_value.h b/be/src/exprs/expr_value.h
index 12cc1dba6f..d38dadd45e 100644
--- a/be/src/exprs/expr_value.h
+++ b/be/src/exprs/expr_value.h
@@ -45,7 +45,8 @@ struct ExprValue {
     std::string string_data;
     StringValue string_val;
     DateTimeValue datetime_val;
-    doris::vectorized::DateV2Value datev2_val;
+    doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> datev2_val;
+    doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> datetimev2_val;
     DecimalV2Value decimalv2_val;
     CollectionValue array_val;
 
diff --git a/be/src/exprs/literal.cpp b/be/src/exprs/literal.cpp
index 83db5f6ae0..e107831739 100644
--- a/be/src/exprs/literal.cpp
+++ b/be/src/exprs/literal.cpp
@@ -81,7 +81,6 @@ Literal::Literal(const TExprNode& node) : Expr(node) {
         break;
     case TYPE_DATE:
     case TYPE_DATETIME:
-    case TYPE_DATETIMEV2:
         _value.datetime_val.from_date_str(node.date_literal.value.c_str(),
                                           node.date_literal.value.size());
         break;
@@ -89,6 +88,10 @@ Literal::Literal(const TExprNode& node) : Expr(node) {
         _value.datev2_val.from_date_str(node.date_literal.value.c_str(),
                                         node.date_literal.value.size());
         break;
+    case TYPE_DATETIMEV2:
+        _value.datetimev2_val.from_date_str(node.date_literal.value.c_str(),
+                                            node.date_literal.value.size());
+        break;
     case TYPE_CHAR:
     case TYPE_VARCHAR:
     case TYPE_STRING:
@@ -117,8 +120,8 @@ Literal::Literal(const TExprNode& node) : Expr(node) {
         break;
     }
     default:
+        DCHECK(false) << "Invalid type: " << _type.debug_string();
         break;
-        // DCHECK(false) << "Invalid type: " << TypeToString(_type.type);
     }
 }
 
@@ -219,6 +222,12 @@ DateV2Val Literal::get_datev2_val(ExprContext* context, TupleRow* row) {
     return dt_val;
 }
 
+DateTimeV2Val Literal::get_datetimev2_val(ExprContext* context, TupleRow* row) {
+    DateTimeV2Val dt_val;
+    _value.datetimev2_val.to_datetimev2_val(&dt_val);
+    return dt_val;
+}
+
 StringVal Literal::get_string_val(ExprContext* context, TupleRow* row) {
     DCHECK(_type.is_string_type()) << _type;
     StringVal str_val;
diff --git a/be/src/exprs/literal.h b/be/src/exprs/literal.h
index 9637b3038c..9a6f7ae793 100644
--- a/be/src/exprs/literal.h
+++ b/be/src/exprs/literal.h
@@ -45,6 +45,7 @@ public:
     virtual DecimalV2Val get_decimalv2_val(ExprContext* context, TupleRow*) override;
     virtual DateTimeVal get_datetime_val(ExprContext* context, TupleRow*) override;
     virtual DateV2Val get_datev2_val(ExprContext* context, TupleRow*) override;
+    virtual DateTimeV2Val get_datetimev2_val(ExprContext* context, TupleRow*) override;
     virtual StringVal get_string_val(ExprContext* context, TupleRow* row) override;
     virtual CollectionVal get_array_val(ExprContext* context, TupleRow*) override;
     virtual Decimal32Val get_decimal32_val(ExprContext* context, TupleRow*) override;
diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp
index f6c142767f..2695e31f25 100644
--- a/be/src/exprs/runtime_filter.cpp
+++ b/be/src/exprs/runtime_filter.cpp
@@ -75,9 +75,8 @@ TExprNodeType::type get_expr_node_type(PrimitiveType type) {
         return TExprNodeType::DECIMAL_LITERAL;
 
     case TYPE_DATETIME:
-        return TExprNodeType::DATE_LITERAL;
-
     case TYPE_DATEV2:
+    case TYPE_DATETIMEV2:
         return TExprNodeType::DATE_LITERAL;
 
     case TYPE_CHAR:
@@ -117,6 +116,8 @@ PColumnType to_proto(PrimitiveType type) {
         return PColumnType::COLUMN_TYPE_DATE;
     case TYPE_DATEV2:
         return PColumnType::COLUMN_TYPE_DATEV2;
+    case TYPE_DATETIMEV2:
+        return PColumnType::COLUMN_TYPE_DATETIMEV2;
     case TYPE_DATETIME:
         return PColumnType::COLUMN_TYPE_DATETIME;
     case TYPE_DECIMALV2:
@@ -164,6 +165,8 @@ PrimitiveType to_primitive_type(PColumnType type) {
         return TYPE_DATE;
     case PColumnType::COLUMN_TYPE_DATEV2:
         return TYPE_DATEV2;
+    case PColumnType::COLUMN_TYPE_DATETIMEV2:
+        return TYPE_DATETIMEV2;
     case PColumnType::COLUMN_TYPE_DATETIME:
         return TYPE_DATETIME;
     case PColumnType::COLUMN_TYPE_DECIMALV2:
@@ -259,6 +262,10 @@ Status create_literal(ObjectPool* pool, const TypeDescriptor& type, const void*
         create_texpr_literal_node<TYPE_DATEV2>(data, &node);
         break;
     }
+    case TYPE_DATETIMEV2: {
+        create_texpr_literal_node<TYPE_DATETIMEV2>(data, &node);
+        break;
+    }
     case TYPE_DATE: {
         create_texpr_literal_node<TYPE_DATE>(data, &node);
         break;
@@ -726,6 +733,14 @@ public:
             });
             break;
         }
+        case TYPE_DATETIMEV2: {
+            batch_assign(in_filter, [](std::unique_ptr<HybridSetBase>& set, PColumnValue& column,
+                                       ObjectPool* pool) {
+                auto date_v2_val = column.longval();
+                set->insert(&date_v2_val);
+            });
+            break;
+        }
         case TYPE_DATETIME:
         case TYPE_DATE: {
             batch_assign(in_filter, [](std::unique_ptr<HybridSetBase>& set, PColumnValue& column,
@@ -863,6 +878,11 @@ public:
             int32_t max_val = minmax_filter->max_val().intval();
             return _minmax_func->assign(&min_val, &max_val);
         }
+        case TYPE_DATETIMEV2: {
+            int64_t min_val = minmax_filter->min_val().longval();
+            int64_t max_val = minmax_filter->max_val().longval();
+            return _minmax_func->assign(&min_val, &max_val);
+        }
         case TYPE_DATETIME:
         case TYPE_DATE: {
             auto& min_val_ref = minmax_filter->min_val().stringval();
@@ -1362,12 +1382,25 @@ void IRuntimeFilter::to_protobuf(PInFilter* filter) {
         return;
     }
     case TYPE_DATEV2: {
-        batch_copy<doris::vectorized::DateV2Value>(
-                filter, it, [](PColumnValue* column, const doris::vectorized::DateV2Value* value) {
+        batch_copy<doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>>(
+                filter, it,
+                [](PColumnValue* column,
+                   const doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*
+                           value) {
                     column->set_intval(*reinterpret_cast<const int32_t*>(value));
                 });
         return;
     }
+    case TYPE_DATETIMEV2: {
+        batch_copy<doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>>(
+                filter, it,
+                [](PColumnValue* column,
+                   const doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>*
+                           value) {
+                    column->set_longval(*reinterpret_cast<const int64_t*>(value));
+                });
+        return;
+    }
     case TYPE_DATE:
     case TYPE_DATETIME: {
         batch_copy<DateTimeValue>(filter, it, [](PColumnValue* column, const DateTimeValue* value) {
@@ -1473,6 +1506,11 @@ void IRuntimeFilter::to_protobuf(PMinMaxFilter* filter) {
         filter->mutable_max_val()->set_intval(*reinterpret_cast<const int32_t*>(max_data));
         return;
     }
+    case TYPE_DATETIMEV2: {
+        filter->mutable_min_val()->set_longval(*reinterpret_cast<const int64_t*>(min_data));
+        filter->mutable_max_val()->set_longval(*reinterpret_cast<const int64_t*>(max_data));
+        return;
+    }
     case TYPE_DATE:
     case TYPE_DATETIME: {
         char convert_buffer[30];
diff --git a/be/src/exprs/scalar_fn_call.cpp b/be/src/exprs/scalar_fn_call.cpp
index 86140761d4..bb710b81b7 100644
--- a/be/src/exprs/scalar_fn_call.cpp
+++ b/be/src/exprs/scalar_fn_call.cpp
@@ -356,6 +356,7 @@ typedef DoubleVal (*DoubleWrapper)(ExprContext*, TupleRow*);
 typedef StringVal (*StringWrapper)(ExprContext*, TupleRow*);
 typedef DateTimeVal (*DatetimeWrapper)(ExprContext*, TupleRow*);
 typedef DateV2Val (*DateV2Wrapper)(ExprContext*, TupleRow*);
+typedef DateTimeV2Val (*DateTimeV2Wrapper)(ExprContext*, TupleRow*);
 typedef DecimalV2Val (*DecimalV2Wrapper)(ExprContext*, TupleRow*);
 typedef Decimal32Val (*Decimal32Wrapper)(ExprContext*, TupleRow*);
 typedef Decimal64Val (*Decimal64Wrapper)(ExprContext*, TupleRow*);
@@ -455,7 +456,7 @@ StringVal ScalarFnCall::get_string_val(ExprContext* context, TupleRow* row) {
 }
 
 DateTimeVal ScalarFnCall::get_datetime_val(ExprContext* context, TupleRow* row) {
-    DCHECK(_type.is_date_type() || _type.is_date_v2_type());
+    DCHECK(_type.is_date_type());
     DCHECK(context != nullptr);
     if (_scalar_fn_wrapper == nullptr) {
         return interpret_eval<DateTimeVal>(context, row);
@@ -474,6 +475,16 @@ DateV2Val ScalarFnCall::get_datev2_val(ExprContext* context, TupleRow* row) {
     return fn(context, row);
 }
 
+DateTimeV2Val ScalarFnCall::get_datetimev2_val(ExprContext* context, TupleRow* row) {
+    DCHECK(_type.is_datetime_v2_type());
+    DCHECK(context != nullptr);
+    if (_scalar_fn_wrapper == nullptr) {
+        return interpret_eval<DateTimeV2Val>(context, row);
+    }
+    DateTimeV2Wrapper fn = reinterpret_cast<DateTimeV2Wrapper>(_scalar_fn_wrapper);
+    return fn(context, row);
+}
+
 DecimalV2Val ScalarFnCall::get_decimalv2_val(ExprContext* context, TupleRow* row) {
     DCHECK_EQ(_type.type, TYPE_DECIMALV2);
     DCHECK(context != nullptr);
diff --git a/be/src/exprs/scalar_fn_call.h b/be/src/exprs/scalar_fn_call.h
index 0e4f297c57..46b633366c 100644
--- a/be/src/exprs/scalar_fn_call.h
+++ b/be/src/exprs/scalar_fn_call.h
@@ -85,6 +85,7 @@ protected:
     virtual doris_udf::StringVal get_string_val(ExprContext* context, TupleRow*) override;
     virtual doris_udf::DateTimeVal get_datetime_val(ExprContext* context, TupleRow*) override;
     virtual doris_udf::DateV2Val get_datev2_val(ExprContext* context, TupleRow*) override;
+    virtual doris_udf::DateTimeV2Val get_datetimev2_val(ExprContext* context, TupleRow*) override;
     virtual doris_udf::DecimalV2Val get_decimalv2_val(ExprContext* context, TupleRow*) override;
     virtual CollectionVal get_array_val(ExprContext* context, TupleRow*) override;
 
diff --git a/be/src/exprs/slot_ref.cpp b/be/src/exprs/slot_ref.cpp
index d6893eedce..ecc749404d 100644
--- a/be/src/exprs/slot_ref.cpp
+++ b/be/src/exprs/slot_ref.cpp
@@ -226,18 +226,33 @@ DateTimeVal SlotRef::get_datetime_val(ExprContext* context, TupleRow* row) {
 }
 
 DateV2Val SlotRef::get_datev2_val(ExprContext* context, TupleRow* row) {
-    DCHECK(_type.is_date_type());
+    DCHECK(_type.is_date_v2_type());
     Tuple* t = row->get_tuple(_tuple_idx);
     if (t == nullptr || t->is_null(_null_indicator_offset)) {
         return DateV2Val::null();
     }
-    doris::vectorized::DateV2Value* tv =
-            reinterpret_cast<doris::vectorized::DateV2Value*>(t->get_slot(_slot_offset));
+    doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>* tv =
+            reinterpret_cast<doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*>(
+                    t->get_slot(_slot_offset));
     DateV2Val result;
     tv->to_datev2_val(&result);
     return result;
 }
 
+DateTimeV2Val SlotRef::get_datetimev2_val(ExprContext* context, TupleRow* row) {
+    DCHECK(_type.is_datetime_v2_type());
+    Tuple* t = row->get_tuple(_tuple_idx);
+    if (t == nullptr || t->is_null(_null_indicator_offset)) {
+        return DateTimeV2Val::null();
+    }
+    doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>* tv = reinterpret_cast<
+            doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>*>(
+            t->get_slot(_slot_offset));
+    DateTimeV2Val result;
+    tv->to_datetimev2_val(&result);
+    return result;
+}
+
 DecimalV2Val SlotRef::get_decimalv2_val(ExprContext* context, TupleRow* row) {
     DCHECK_EQ(_type.type, TYPE_DECIMALV2);
     Tuple* t = row->get_tuple(_tuple_idx);
diff --git a/be/src/exprs/slot_ref.h b/be/src/exprs/slot_ref.h
index 009b841e56..406ed1f3f5 100644
--- a/be/src/exprs/slot_ref.h
+++ b/be/src/exprs/slot_ref.h
@@ -70,6 +70,7 @@ public:
     virtual doris_udf::StringVal get_string_val(ExprContext* context, TupleRow*) override;
     virtual doris_udf::DateTimeVal get_datetime_val(ExprContext* context, TupleRow*) override;
     virtual doris_udf::DateV2Val get_datev2_val(ExprContext* context, TupleRow*) override;
+    virtual doris_udf::DateTimeV2Val get_datetimev2_val(ExprContext* context, TupleRow*) override;
     virtual doris_udf::DecimalV2Val get_decimalv2_val(ExprContext* context, TupleRow*) override;
     virtual doris_udf::CollectionVal get_array_val(ExprContext* context, TupleRow*) override;
     virtual Decimal32Val get_decimal32_val(ExprContext* context, TupleRow*) override;
diff --git a/be/src/olap/aggregate_func.cpp b/be/src/olap/aggregate_func.cpp
index 36f60933aa..084dbca91f 100644
--- a/be/src/olap/aggregate_func.cpp
+++ b/be/src/olap/aggregate_func.cpp
@@ -104,6 +104,7 @@ AggregateFuncResolver::AggregateFuncResolver() {
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_NONE, OLAP_FIELD_TYPE_DATE>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_NONE, OLAP_FIELD_TYPE_DATETIME>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_NONE, OLAP_FIELD_TYPE_DATEV2>();
+    add_aggregate_mapping<OLAP_FIELD_AGGREGATION_NONE, OLAP_FIELD_TYPE_DATETIMEV2>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_NONE, OLAP_FIELD_TYPE_CHAR>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_NONE, OLAP_FIELD_TYPE_VARCHAR>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_NONE, OLAP_FIELD_TYPE_STRING>();
@@ -161,6 +162,7 @@ AggregateFuncResolver::AggregateFuncResolver() {
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_MIN, OLAP_FIELD_TYPE_DATE>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_MIN, OLAP_FIELD_TYPE_DATETIME>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_MIN, OLAP_FIELD_TYPE_DATEV2>();
+    add_aggregate_mapping<OLAP_FIELD_AGGREGATION_MIN, OLAP_FIELD_TYPE_DATETIMEV2>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_MIN, OLAP_FIELD_TYPE_CHAR>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_MIN, OLAP_FIELD_TYPE_VARCHAR>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_MIN, OLAP_FIELD_TYPE_STRING>();
@@ -180,6 +182,7 @@ AggregateFuncResolver::AggregateFuncResolver() {
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_MAX, OLAP_FIELD_TYPE_DATE>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_MAX, OLAP_FIELD_TYPE_DATETIME>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_MAX, OLAP_FIELD_TYPE_DATEV2>();
+    add_aggregate_mapping<OLAP_FIELD_AGGREGATION_MAX, OLAP_FIELD_TYPE_DATETIMEV2>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_MAX, OLAP_FIELD_TYPE_CHAR>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_MAX, OLAP_FIELD_TYPE_VARCHAR>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_MAX, OLAP_FIELD_TYPE_STRING>();
@@ -213,6 +216,7 @@ AggregateFuncResolver::AggregateFuncResolver() {
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_REPLACE, OLAP_FIELD_TYPE_DATE>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_REPLACE, OLAP_FIELD_TYPE_DATETIME>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_REPLACE, OLAP_FIELD_TYPE_DATEV2>();
+    add_aggregate_mapping<OLAP_FIELD_AGGREGATION_REPLACE, OLAP_FIELD_TYPE_DATETIMEV2>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_REPLACE, OLAP_FIELD_TYPE_CHAR>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_REPLACE, OLAP_FIELD_TYPE_VARCHAR>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_REPLACE, OLAP_FIELD_TYPE_STRING>();
@@ -233,6 +237,7 @@ AggregateFuncResolver::AggregateFuncResolver() {
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_REPLACE_IF_NOT_NULL, OLAP_FIELD_TYPE_DATE>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_REPLACE_IF_NOT_NULL, OLAP_FIELD_TYPE_DATETIME>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_REPLACE_IF_NOT_NULL, OLAP_FIELD_TYPE_DATEV2>();
+    add_aggregate_mapping<OLAP_FIELD_AGGREGATION_REPLACE_IF_NOT_NULL, OLAP_FIELD_TYPE_DATETIMEV2>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_REPLACE_IF_NOT_NULL, OLAP_FIELD_TYPE_CHAR>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_REPLACE_IF_NOT_NULL, OLAP_FIELD_TYPE_VARCHAR>();
     add_aggregate_mapping<OLAP_FIELD_AGGREGATION_REPLACE_IF_NOT_NULL, OLAP_FIELD_TYPE_STRING>();
diff --git a/be/src/olap/bloom_filter_predicate.cpp b/be/src/olap/bloom_filter_predicate.cpp
index a1acb913c4..299e4e1e1c 100644
--- a/be/src/olap/bloom_filter_predicate.cpp
+++ b/be/src/olap/bloom_filter_predicate.cpp
@@ -29,8 +29,9 @@
     M(TYPE_DOUBLE)            \
     M(TYPE_CHAR)              \
     M(TYPE_DATE)              \
-    M(TYPE_DATEV2)            \
     M(TYPE_DATETIME)          \
+    M(TYPE_DATEV2)            \
+    M(TYPE_DATETIMEV2)        \
     M(TYPE_VARCHAR)           \
     M(TYPE_STRING)            \
     M(TYPE_DECIMAL32)         \
diff --git a/be/src/olap/bloom_filter_predicate.h b/be/src/olap/bloom_filter_predicate.h
index b4ed077709..e5833729b4 100644
--- a/be/src/olap/bloom_filter_predicate.h
+++ b/be/src/olap/bloom_filter_predicate.h
@@ -99,7 +99,8 @@ private:
             };
 
             auto get_cell_value = [&tmp_uint24_value](auto& data) {
-                if constexpr (std::is_same_v<std::decay_t<decltype(data)>, uint32_t>) {
+                if constexpr (std::is_same_v<std::decay_t<decltype(data)>, uint32_t> &&
+                              T == PrimitiveType::TYPE_DATE) {
                     memcpy((char*)(&tmp_uint24_value), (char*)(&data), sizeof(uint24_t));
                     return (const char*)&tmp_uint24_value;
                 } else {
diff --git a/be/src/olap/column_vector.cpp b/be/src/olap/column_vector.cpp
index 6d085b7a64..36fd451a21 100644
--- a/be/src/olap/column_vector.cpp
+++ b/be/src/olap/column_vector.cpp
@@ -115,6 +115,11 @@ Status ColumnVectorBatch::create(size_t init_capacity, bool is_nullable, const T
                     new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_DATETIME>::CppType>(
                             type_info, is_nullable));
             break;
+        case OLAP_FIELD_TYPE_DATETIMEV2:
+            local.reset(
+                    new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_DATETIMEV2>::CppType>(
+                            type_info, is_nullable));
+            break;
         case OLAP_FIELD_TYPE_CHAR:
             local.reset(new ScalarColumnVectorBatch<CppTypeTraits<OLAP_FIELD_TYPE_CHAR>::CppType>(
                     type_info, is_nullable));
diff --git a/be/src/olap/delete_handler.cpp b/be/src/olap/delete_handler.cpp
index 358262965b..716c84b504 100644
--- a/be/src/olap/delete_handler.cpp
+++ b/be/src/olap/delete_handler.cpp
@@ -153,6 +153,7 @@ bool DeleteHandler::is_condition_value_valid(const TabletColumn& column,
     case OLAP_FIELD_TYPE_DATE:
     case OLAP_FIELD_TYPE_DATETIME:
     case OLAP_FIELD_TYPE_DATEV2:
+    case OLAP_FIELD_TYPE_DATETIMEV2:
         return valid_datetime(value_str);
     case OLAP_FIELD_TYPE_BOOL:
         return valid_bool(value_str);
diff --git a/be/src/olap/field.h b/be/src/olap/field.h
index dadd483581..99b9bc0665 100644
--- a/be/src/olap/field.h
+++ b/be/src/olap/field.h
@@ -754,7 +754,9 @@ public:
                 [[fallthrough]];
             case OLAP_FIELD_TYPE_DECIMAL64:
                 [[fallthrough]];
-            case OLAP_FIELD_TYPE_DECIMAL128: {
+            case OLAP_FIELD_TYPE_DECIMAL128:
+                [[fallthrough]];
+            case OLAP_FIELD_TYPE_DATETIMEV2: {
                 Field* field = new Field(column);
                 field->set_precision(column.precision());
                 field->set_scale(column.frac());
diff --git a/be/src/olap/key_coder.cpp b/be/src/olap/key_coder.cpp
index ce3a722893..d4fa6bd6c7 100644
--- a/be/src/olap/key_coder.cpp
+++ b/be/src/olap/key_coder.cpp
@@ -74,6 +74,7 @@ private:
         add_mapping<OLAP_FIELD_TYPE_STRING>();
         add_mapping<OLAP_FIELD_TYPE_BOOL>();
         add_mapping<OLAP_FIELD_TYPE_DATEV2>();
+        add_mapping<OLAP_FIELD_TYPE_DATETIMEV2>();
         add_mapping<OLAP_FIELD_TYPE_DECIMAL32>();
         add_mapping<OLAP_FIELD_TYPE_DECIMAL64>();
         add_mapping<OLAP_FIELD_TYPE_DECIMAL128>();
diff --git a/be/src/olap/key_coder.h b/be/src/olap/key_coder.h
index b25aa20dd8..dd762a1b22 100644
--- a/be/src/olap/key_coder.h
+++ b/be/src/olap/key_coder.h
@@ -204,6 +204,40 @@ public:
     }
 };
 
+template <>
+class KeyCoderTraits<OLAP_FIELD_TYPE_DATETIMEV2> {
+public:
+    using CppType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATETIMEV2>::CppType;
+    using UnsignedCppType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATETIMEV2>::UnsignedCppType;
+
+public:
+    static void full_encode_ascending(const void* value, std::string* buf) {
+        UnsignedCppType unsigned_val;
+        memcpy(&unsigned_val, value, sizeof(unsigned_val));
+        // make it bigendian
+        unsigned_val = BigEndian::FromHost64(unsigned_val);
+        buf->append((char*)&unsigned_val, sizeof(unsigned_val));
+    }
+
+    static void encode_ascending(const void* value, size_t index_size, std::string* buf) {
+        full_encode_ascending(value, buf);
+    }
+
+    static Status decode_ascending(Slice* encoded_key, size_t index_size, uint8_t* cell_ptr,
+                                   MemPool* pool) {
+        if (encoded_key->size < sizeof(UnsignedCppType)) {
+            return Status::InvalidArgument(Substitute("Key too short, need=$0 vs real=$1",
+                                                      sizeof(UnsignedCppType), encoded_key->size));
+        }
+        UnsignedCppType unsigned_val;
+        memcpy(&unsigned_val, encoded_key->data, sizeof(UnsignedCppType));
+        unsigned_val = BigEndian::FromHost64(unsigned_val);
+        memcpy(cell_ptr, &unsigned_val, sizeof(UnsignedCppType));
+        encoded_key->remove_prefix(sizeof(UnsignedCppType));
+        return Status::OK();
+    }
+};
+
 template <>
 class KeyCoderTraits<OLAP_FIELD_TYPE_DECIMAL> {
 public:
diff --git a/be/src/olap/reader.cpp b/be/src/olap/reader.cpp
index b1da44702c..2599c66747 100644
--- a/be/src/olap/reader.cpp
+++ b/be/src/olap/reader.cpp
@@ -570,6 +570,11 @@ void TabletReader::_init_conditions_param(const ReaderParams& read_params) {
             predicate = new PREDICATE<uint32_t>(index, value, opposite);                           \
             break;                                                                                 \
         }                                                                                          \
+        case OLAP_FIELD_TYPE_DATETIMEV2: {                                                         \
+            uint64_t value = timestamp_from_datetime_v2(cond);                                     \
+            predicate = new PREDICATE<uint64_t>(index, value, opposite);                           \
+            break;                                                                                 \
+        }                                                                                          \
         case OLAP_FIELD_TYPE_DATETIME: {                                                           \
             uint64_t value = timestamp_from_datetime(cond);                                        \
             predicate = new PREDICATE<uint64_t>(index, value, opposite);                           \
@@ -861,6 +866,19 @@ ColumnPredicate* TabletReader::_parse_to_predicate(const TCondition& condition,
             }
             break;
         }
+        case OLAP_FIELD_TYPE_DATETIMEV2: {
+            phmap::flat_hash_set<uint64_t> values;
+            for (auto& cond_val : condition.condition_values) {
+                uint64_t value = timestamp_from_datetime_v2(cond_val);
+                values.insert(value);
+            }
+            if (condition.condition_op == "*=") {
+                predicate = new InListPredicate<uint64_t>(index, std::move(values), opposite);
+            } else {
+                predicate = new NotInListPredicate<uint64_t>(index, std::move(values), opposite);
+            }
+            break;
+        }
         case OLAP_FIELD_TYPE_DATETIME: {
             phmap::flat_hash_set<uint64_t> values;
             for (auto& cond_val : condition.condition_values) {
diff --git a/be/src/olap/row_block2.cpp b/be/src/olap/row_block2.cpp
index 8f8632e5ea..f647018f58 100644
--- a/be/src/olap/row_block2.cpp
+++ b/be/src/olap/row_block2.cpp
@@ -250,6 +250,11 @@ Status RowBlockV2::_copy_data_to_column(int cid,
         insert_data_directly(cid, column_int);
         break;
     }
+    case OLAP_FIELD_TYPE_DATETIMEV2: {
+        auto column_int = assert_cast<vectorized::ColumnVector<vectorized::UInt64>*>(column);
+        insert_data_directly(cid, column_int);
+        break;
+    }
     case OLAP_FIELD_TYPE_DATETIME: {
         auto column_int = assert_cast<vectorized::ColumnVector<vectorized::Int64>*>(column);
 
@@ -535,6 +540,11 @@ Status RowBlockV2::_append_data_to_column(const ColumnVectorBatch* batch, size_t
         insert_data_directly(batch, column_int, start, len);
         break;
     }
+    case OLAP_FIELD_TYPE_DATETIMEV2: {
+        auto column_int = assert_cast<vectorized::ColumnVector<vectorized::UInt64>*>(column);
+        insert_data_directly(batch, column_int, start, len);
+        break;
+    }
     case OLAP_FIELD_TYPE_DATETIME: {
         auto column_int = assert_cast<vectorized::ColumnVector<vectorized::Int64>*>(column);
 
diff --git a/be/src/olap/rowset/segment_v2/bitmap_index_writer.cpp b/be/src/olap/rowset/segment_v2/bitmap_index_writer.cpp
index 610d9ea3fa..f8bc37a127 100644
--- a/be/src/olap/rowset/segment_v2/bitmap_index_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/bitmap_index_writer.cpp
@@ -223,6 +223,9 @@ Status BitmapIndexWriter::create(const TypeInfo* type_info,
     case OLAP_FIELD_TYPE_DATEV2:
         res->reset(new BitmapIndexWriterImpl<OLAP_FIELD_TYPE_DATEV2>(type_info));
         break;
+    case OLAP_FIELD_TYPE_DATETIMEV2:
+        res->reset(new BitmapIndexWriterImpl<OLAP_FIELD_TYPE_DATETIMEV2>(type_info));
+        break;
     case OLAP_FIELD_TYPE_LARGEINT:
         res->reset(new BitmapIndexWriterImpl<OLAP_FIELD_TYPE_LARGEINT>(type_info));
         break;
diff --git a/be/src/olap/rowset/segment_v2/encoding_info.cpp b/be/src/olap/rowset/segment_v2/encoding_info.cpp
index 1a49da8792..bfe1a43f43 100644
--- a/be/src/olap/rowset/segment_v2/encoding_info.cpp
+++ b/be/src/olap/rowset/segment_v2/encoding_info.cpp
@@ -149,6 +149,20 @@ struct TypeEncodingTraits<OLAP_FIELD_TYPE_DATEV2, FOR_ENCODING,
     }
 };
 
+template <>
+struct TypeEncodingTraits<OLAP_FIELD_TYPE_DATETIMEV2, FOR_ENCODING,
+                          typename CppTypeTraits<OLAP_FIELD_TYPE_DATETIMEV2>::CppType> {
+    static Status create_page_builder(const PageBuilderOptions& opts, PageBuilder** builder) {
+        *builder = new FrameOfReferencePageBuilder<OLAP_FIELD_TYPE_DATETIMEV2>(opts);
+        return Status::OK();
+    }
+    static Status create_page_decoder(const Slice& data, const PageDecoderOptions& opts,
+                                      PageDecoder** decoder) {
+        *decoder = new FrameOfReferencePageDecoder<OLAP_FIELD_TYPE_DATETIMEV2>(data, opts);
+        return Status::OK();
+    }
+};
+
 template <FieldType type, typename CppType>
 struct TypeEncodingTraits<type, FOR_ENCODING, CppType,
                           typename std::enable_if<std::is_integral<CppType>::value>::type> {
@@ -286,6 +300,10 @@ EncodingInfoResolver::EncodingInfoResolver() {
     _add_map<OLAP_FIELD_TYPE_DATEV2, PLAIN_ENCODING>();
     _add_map<OLAP_FIELD_TYPE_DATEV2, FOR_ENCODING, true>();
 
+    _add_map<OLAP_FIELD_TYPE_DATETIMEV2, BIT_SHUFFLE>();
+    _add_map<OLAP_FIELD_TYPE_DATETIMEV2, PLAIN_ENCODING>();
+    _add_map<OLAP_FIELD_TYPE_DATETIMEV2, FOR_ENCODING, true>();
+
     _add_map<OLAP_FIELD_TYPE_DATETIME, BIT_SHUFFLE>();
     _add_map<OLAP_FIELD_TYPE_DATETIME, PLAIN_ENCODING>();
     _add_map<OLAP_FIELD_TYPE_DATETIME, FOR_ENCODING, true>();
diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
index 3ab94f0d9b..8197e543b8 100644
--- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
@@ -809,10 +809,11 @@ void SegmentIterator::_init_current_block(
             if (column_desc->type() == OLAP_FIELD_TYPE_DATE) {
                 current_columns[cid]->set_date_type();
             } else if (column_desc->type() == OLAP_FIELD_TYPE_DATETIME) {
-                // TODO(Gabriel): support datetime v2
                 current_columns[cid]->set_datetime_type();
             } else if (column_desc->type() == OLAP_FIELD_TYPE_DATEV2) {
                 current_columns[cid]->set_date_v2_type();
+            } else if (column_desc->type() == OLAP_FIELD_TYPE_DATETIMEV2) {
+                current_columns[cid]->set_datetime_v2_type();
             } else if (column_desc->type() == OLAP_FIELD_TYPE_DECIMAL) {
                 current_columns[cid]->set_decimalv2_type();
             }
diff --git a/be/src/olap/schema.cpp b/be/src/olap/schema.cpp
index 47fc3bcbdd..7f49735821 100644
--- a/be/src/olap/schema.cpp
+++ b/be/src/olap/schema.cpp
@@ -156,6 +156,9 @@ vectorized::IColumn::MutablePtr Schema::get_predicate_column_ptr(FieldType type)
     case OLAP_FIELD_TYPE_DATEV2:
         return doris::vectorized::PredicateColumnType<uint32_t>::create();
 
+    case OLAP_FIELD_TYPE_DATETIMEV2:
+        return doris::vectorized::PredicateColumnType<uint64_t>::create();
+
     case OLAP_FIELD_TYPE_DATETIME:
         return doris::vectorized::PredicateColumnType<uint64_t>::create();
 
diff --git a/be/src/olap/tablet_schema.cpp b/be/src/olap/tablet_schema.cpp
index cc207369ce..cbeca51192 100644
--- a/be/src/olap/tablet_schema.cpp
+++ b/be/src/olap/tablet_schema.cpp
@@ -61,6 +61,8 @@ FieldType TabletColumn::get_field_type_by_string(const std::string& type_str) {
         type = OLAP_FIELD_TYPE_DATE;
     } else if (0 == upper_type_str.compare("DATEV2")) {
         type = OLAP_FIELD_TYPE_DATEV2;
+    } else if (0 == upper_type_str.compare("DATETIMEV2")) {
+        type = OLAP_FIELD_TYPE_DATETIMEV2;
     } else if (0 == upper_type_str.compare("DATETIME")) {
         type = OLAP_FIELD_TYPE_DATETIME;
     } else if (0 == upper_type_str.compare("DECIMAL32")) {
@@ -180,6 +182,9 @@ std::string TabletColumn::get_string_by_field_type(FieldType type) {
     case OLAP_FIELD_TYPE_DATETIME:
         return "DATETIME";
 
+    case OLAP_FIELD_TYPE_DATETIMEV2:
+        return "DATETIMEV2";
+
     case OLAP_FIELD_TYPE_DECIMAL:
         return "DECIMAL";
 
@@ -276,6 +281,8 @@ uint32_t TabletColumn::get_field_length_by_type(TPrimitiveType::type type, uint3
         return 4;
     case TPrimitiveType::DATETIME:
         return 8;
+    case TPrimitiveType::DATETIMEV2:
+        return 8;
     case TPrimitiveType::FLOAT:
         return 4;
     case TPrimitiveType::DOUBLE:
diff --git a/be/src/olap/types.h b/be/src/olap/types.h
index 2b50ad7785..3418e9a95d 100644
--- a/be/src/olap/types.h
+++ b/be/src/olap/types.h
@@ -1051,6 +1051,11 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATE> : public BaseFieldtypeTraits<OLAP_F
             using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATEV2>::CppType;
             *reinterpret_cast<CppType*>(dest) = *reinterpret_cast<const SrcType*>(src);
             return Status::OK();
+        } else if (src_type->type() == FieldType::OLAP_FIELD_TYPE_DATETIMEV2) {
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATETIMEV2>::CppType;
+            *reinterpret_cast<CppType*>(dest) =
+                    (*reinterpret_cast<const SrcType*>(src) >> doris::vectorized::TIME_PART_LENGTH);
+            return Status::OK();
         }
 
         if (src_type->type() == FieldType::OLAP_FIELD_TYPE_INT) {
@@ -1119,8 +1124,8 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATEV2>
     }
     static std::string to_string(const void* src) {
         CppType tmp = *reinterpret_cast<const CppType*>(src);
-        doris::vectorized::DateV2Value value =
-                binary_cast<CppType, doris::vectorized::DateV2Value>(tmp);
+        doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> value = binary_cast<
+                CppType, doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>>(tmp);
         string format = "%Y-%m-%d";
         string res;
         res.resize(12);
@@ -1142,6 +1147,13 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATEV2>
             return Status::OK();
         }
 
+        if (src_type->type() == FieldType::OLAP_FIELD_TYPE_DATETIMEV2) {
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATETIMEV2>::CppType;
+            SrcType src_value = *reinterpret_cast<const SrcType*>(src);
+            *reinterpret_cast<CppType*>(dest) = src_value >> doris::vectorized::TIME_PART_LENGTH;
+            return Status::OK();
+        }
+
         if (src_type->type() == FieldType::OLAP_FIELD_TYPE_DATE) {
             using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATE>::CppType;
             SrcType value = *reinterpret_cast<const SrcType*>(src);
@@ -1155,7 +1167,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATEV2>
         if (src_type->type() == FieldType::OLAP_FIELD_TYPE_INT) {
             using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_INT>::CppType;
             SrcType src_value = *reinterpret_cast<const SrcType*>(src);
-            doris::vectorized::DateV2Value dt;
+            doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> dt;
             if (!dt.from_date_int64(src_value)) {
                 return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
             }
@@ -1174,7 +1186,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATEV2>
             }
             using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_VARCHAR>::CppType;
             auto src_value = *reinterpret_cast<const SrcType*>(src);
-            doris::vectorized::DateV2Value dt;
+            doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> dt;
             for (const auto& format : DATE_FORMATS) {
                 if (dt.from_date_format_str(format.c_str(), format.length(), src_value.get_data(),
                                             src_value.get_size())) {
@@ -1198,6 +1210,87 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATEV2>
     }
 };
 
+template <>
+struct FieldTypeTraits<OLAP_FIELD_TYPE_DATETIMEV2>
+        : public BaseFieldtypeTraits<OLAP_FIELD_TYPE_DATETIMEV2> {
+    static Status from_string(void* buf, const std::string& scan_key, const int precision,
+                              const int scale) {
+        doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> datetimev2_value;
+        std::string date_format = "%Y-%m-%d %H:%i:%s.%f";
+
+        if (datetimev2_value.from_date_format_str(date_format.data(), date_format.size(),
+                                                  scan_key.data(), scan_key.size())) {
+            *reinterpret_cast<CppType*>(buf) = datetimev2_value.to_date_int_val();
+        } else {
+            *reinterpret_cast<CppType*>(buf) = doris::vectorized::MIN_DATETIME_V2;
+        }
+
+        return Status::OK();
+    }
+    static std::string to_string(const void* src) {
+        CppType tmp = *reinterpret_cast<const CppType*>(src);
+        doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> value =
+                binary_cast<CppType,
+                            doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>>(
+                        tmp);
+        string format = "%Y-%m-%d %H:%i:%s.%f";
+        string res;
+        res.resize(30);
+        res.reserve(30);
+        value.to_format_string(format.c_str(), format.size(), res.data());
+        return res;
+    }
+    static Status convert_from(void* dest, const void* src, const TypeInfo* src_type,
+                               MemPool* mem_pool, size_t variable_len = 0) {
+        doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> datetimev2_value;
+        if (src_type->type() == FieldType::OLAP_FIELD_TYPE_DATETIME) {
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATETIME>::CppType;
+            SrcType src_value = *reinterpret_cast<const SrcType*>(src);
+            //only need part one
+            SrcType part1 = src_value / 1000000L;
+            CppType year = static_cast<CppType>((part1 / 10000L) % 10000);
+            CppType mon = static_cast<CppType>((part1 / 100) % 100);
+            CppType mday = static_cast<CppType>(part1 % 100);
+
+            CppType part2 = src_value % 1000000L;
+            CppType hour = static_cast<int>((part2 / 10000L) % 10000);
+            CppType min = static_cast<int>((part2 / 100) % 100);
+            CppType sec = static_cast<int>(part2 % 100);
+            datetimev2_value.set_time(year, mon, mday, hour, min, sec, 0);
+            *reinterpret_cast<CppType*>(dest) = datetimev2_value.to_date_int_val();
+            return Status::OK();
+        }
+
+        if (src_type->type() == FieldType::OLAP_FIELD_TYPE_DATE) {
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATE>::CppType;
+            SrcType value = *reinterpret_cast<const SrcType*>(src);
+            int day = static_cast<int>(value & 31);
+            int mon = static_cast<int>(value >> 5 & 15);
+            int year = static_cast<int>(value >> 9);
+            datetimev2_value.set_time(year, mon, day, 0, 0, 0, 0);
+            *reinterpret_cast<CppType*>(dest) = datetimev2_value.to_date_int_val();
+            return Status::OK();
+        }
+
+        if (src_type->type() == FieldType::OLAP_FIELD_TYPE_DATEV2) {
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATEV2>::CppType;
+            SrcType value = *reinterpret_cast<const SrcType*>(src);
+            *reinterpret_cast<CppType*>(dest) = (uint64_t)value
+                                                << doris::vectorized::TIME_PART_LENGTH;
+            return Status::OK();
+        }
+        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+    }
+    static void set_to_max(void* buf) {
+        // max is 9999 * 16 * 32 + 12 * 32 + 31;
+        *reinterpret_cast<CppType*>(buf) = doris::vectorized::MAX_DATETIME_V2;
+    }
+    static void set_to_min(void* buf) {
+        // min is 0 * 16 * 32 + 1 * 32 + 1;
+        *reinterpret_cast<CppType*>(buf) = doris::vectorized::MIN_DATETIME_V2;
+    }
+};
+
 template <>
 struct FieldTypeTraits<OLAP_FIELD_TYPE_DATETIME>
         : public BaseFieldtypeTraits<OLAP_FIELD_TYPE_DATETIME> {
@@ -1256,6 +1349,18 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATETIME>
             int year = static_cast<int>((value & 0xFFFFFE00) >> 9);
             *reinterpret_cast<CppType*>(dest) = (year * 10000L + mon * 100L + day) * 1000000;
             return Status::OK();
+        } else if (src_type->type() == FieldType::OLAP_FIELD_TYPE_DATETIMEV2) {
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATEV2>::CppType;
+            auto value = *reinterpret_cast<const SrcType*>(src);
+
+            doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> datetimev2_value;
+            datetimev2_value.from_datetime(value);
+            doris::vectorized::VecDateTimeValue to_value;
+            to_value.set_time(datetimev2_value.year(), datetimev2_value.month(),
+                              datetimev2_value.day(), datetimev2_value.hour(),
+                              datetimev2_value.minute(), datetimev2_value.second());
+            *reinterpret_cast<CppType*>(dest) = to_value.to_datetime_int64();
+            return Status::OK();
         }
         return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
     }
diff --git a/be/src/olap/utils.cpp b/be/src/olap/utils.cpp
index 417d537fd2..9561d70206 100644
--- a/be/src/olap/utils.cpp
+++ b/be/src/olap/utils.cpp
@@ -744,7 +744,7 @@ bool valid_decimal(const string& value_str, const uint32_t precision, const uint
 bool valid_datetime(const string& value_str) {
     const char* datetime_pattern =
             "((?:\\d){4})-((?:\\d){2})-((?:\\d){2})[ ]*"
-            "(((?:\\d){2}):((?:\\d){2}):((?:\\d){2}))?";
+            "(((?:\\d){2}):((?:\\d){2}):((?:\\d){2})([.]*((?:\\d){0,6})))?";
     std::regex e(datetime_pattern);
     std::smatch what;
 
@@ -785,6 +785,12 @@ bool valid_datetime(const string& value_str) {
                 LOG(WARNING) << "invalid second. [second=" << second << "]";
                 return false;
             }
+            if (what[8].length()) {
+                if (what[9].str().size() > 6) {
+                    LOG(WARNING) << "invalid microsecond. [second=" << second << "]";
+                    return false;
+                }
+            }
         }
 
         return true;
diff --git a/be/src/runtime/datetime_value.h b/be/src/runtime/datetime_value.h
index 20caa2a428..91e0f83c62 100644
--- a/be/src/runtime/datetime_value.h
+++ b/be/src/runtime/datetime_value.h
@@ -570,8 +570,14 @@ private:
     friend class UnusedClass;
     friend void doris::vectorized::VecDateTimeValue::convert_vec_dt_to_dt(DateTimeValue* dt);
     friend void doris::vectorized::VecDateTimeValue::convert_dt_to_vec_dt(DateTimeValue* dt);
-    friend void doris::vectorized::DateV2Value::convert_date_v2_to_dt(DateTimeValue* dt);
-    friend void doris::vectorized::DateV2Value::convert_dt_to_date_v2(DateTimeValue* dt);
+    friend void doris::vectorized::DateV2Value<
+            doris::vectorized::DateV2ValueType>::convert_date_v2_to_dt(DateTimeValue* dt);
+    friend void doris::vectorized::DateV2Value<
+            doris::vectorized::DateV2ValueType>::convert_dt_to_date_v2(DateTimeValue* dt);
+    friend void doris::vectorized::DateV2Value<
+            doris::vectorized::DateTimeV2ValueType>::convert_date_v2_to_dt(DateTimeValue* dt);
+    friend void doris::vectorized::DateV2Value<
+            doris::vectorized::DateTimeV2ValueType>::convert_dt_to_date_v2(DateTimeValue* dt);
 
     void from_packed_time(int64_t packed_time) {
         _microsecond = packed_time % (1LL << 24);
diff --git a/be/src/runtime/primitive_type.cpp b/be/src/runtime/primitive_type.cpp
index 8e90123c9a..0ad2654187 100644
--- a/be/src/runtime/primitive_type.cpp
+++ b/be/src/runtime/primitive_type.cpp
@@ -642,6 +642,8 @@ int get_slot_size(PrimitiveType type) {
     case TYPE_DOUBLE:
     case TYPE_TIME:
     case TYPE_DECIMAL64:
+    case TYPE_DATETIMEV2:
+    case TYPE_TIMEV2:
         return 8;
 
     case TYPE_LARGEINT:
@@ -649,8 +651,6 @@ int get_slot_size(PrimitiveType type) {
 
     case TYPE_DATE:
     case TYPE_DATETIME:
-    case TYPE_DATETIMEV2:
-    case TYPE_TIMEV2:
         // This is the size of the slot, the actual size of the data is 12.
         return sizeof(DateTimeValue);
 
diff --git a/be/src/runtime/primitive_type.h b/be/src/runtime/primitive_type.h
index adf1acadd4..973db8f31c 100644
--- a/be/src/runtime/primitive_type.h
+++ b/be/src/runtime/primitive_type.h
@@ -148,12 +148,12 @@ struct PrimitiveTypeTraits<TYPE_DATETIME> {
 };
 template <>
 struct PrimitiveTypeTraits<TYPE_DATETIMEV2> {
-    using CppType = doris::DateTimeValue;
-    using ColumnType = vectorized::ColumnVector<vectorized::DateTime>;
+    using CppType = doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>;
+    using ColumnType = vectorized::ColumnVector<vectorized::DateTimeV2>;
 };
 template <>
 struct PrimitiveTypeTraits<TYPE_DATEV2> {
-    using CppType = doris::vectorized::DateV2Value;
+    using CppType = doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>;
     using ColumnType = vectorized::ColumnVector<vectorized::DateV2>;
 };
 template <>
diff --git a/be/src/runtime/raw_value.cpp b/be/src/runtime/raw_value.cpp
index c86bf53f0c..8040514d84 100644
--- a/be/src/runtime/raw_value.cpp
+++ b/be/src/runtime/raw_value.cpp
@@ -87,7 +87,14 @@ void RawValue::print_value_as_bytes(const void* value, const TypeDescriptor& typ
         break;
 
     case TYPE_DATEV2:
-        stream->write(chars, sizeof(doris::vectorized::DateV2Value));
+        stream->write(chars,
+                      sizeof(doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>));
+        break;
+
+    case TYPE_DATETIMEV2:
+        stream->write(
+                chars,
+                sizeof(doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>));
         break;
 
     case TYPE_DECIMALV2:
@@ -181,7 +188,14 @@ void RawValue::print_value(const void* value, const TypeDescriptor& type, int sc
         break;
 
     case TYPE_DATEV2:
-        *stream << *reinterpret_cast<const doris::vectorized::DateV2Value*>(value);
+        *stream << *reinterpret_cast<
+                const doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*>(value);
+        break;
+
+    case TYPE_DATETIMEV2:
+        *stream << *reinterpret_cast<
+                const doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>*>(
+                value);
         break;
 
     case TYPE_DECIMALV2:
@@ -334,8 +348,18 @@ void RawValue::write(const void* value, void* dst, const TypeDescriptor& type, M
         break;
 
     case TYPE_DATEV2:
-        *reinterpret_cast<doris::vectorized::DateV2Value*>(dst) =
-                *reinterpret_cast<const doris::vectorized::DateV2Value*>(value);
+        *reinterpret_cast<doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*>(
+                dst) =
+                *reinterpret_cast<
+                        const doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*>(
+                        value);
+        break;
+
+    case TYPE_DATETIMEV2:
+        *reinterpret_cast<doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>*>(
+                dst) =
+                *reinterpret_cast<const doris::vectorized::DateV2Value<
+                        doris::vectorized::DateTimeV2ValueType>*>(value);
         break;
 
     case TYPE_DECIMALV2:
@@ -437,8 +461,17 @@ void RawValue::write(const void* value, const TypeDescriptor& type, void* dst, u
         *reinterpret_cast<DateTimeValue*>(dst) = *reinterpret_cast<const DateTimeValue*>(value);
         break;
     case TYPE_DATEV2:
-        *reinterpret_cast<doris::vectorized::DateV2Value*>(dst) =
-                *reinterpret_cast<const doris::vectorized::DateV2Value*>(value);
+        *reinterpret_cast<doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*>(
+                dst) =
+                *reinterpret_cast<
+                        const doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*>(
+                        value);
+        break;
+    case TYPE_DATETIMEV2:
+        *reinterpret_cast<doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>*>(
+                dst) =
+                *reinterpret_cast<const doris::vectorized::DateV2Value<
+                        doris::vectorized::DateTimeV2ValueType>*>(value);
         break;
     case TYPE_VARCHAR:
     case TYPE_CHAR:
@@ -557,8 +590,18 @@ int RawValue::compare(const void* v1, const void* v2, const TypeDescriptor& type
         return *ts_value1 > *ts_value2 ? 1 : (*ts_value1 < *ts_value2 ? -1 : 0);
 
     case TYPE_DATEV2: {
-        auto date_v2_value1 = reinterpret_cast<const doris::vectorized::DateV2Value*>(v1);
-        auto date_v2_value2 = reinterpret_cast<const doris::vectorized::DateV2Value*>(v2);
+        auto date_v2_value1 = reinterpret_cast<
+                const doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*>(v1);
+        auto date_v2_value2 = reinterpret_cast<
+                const doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*>(v2);
+        return *date_v2_value1 > *date_v2_value2 ? 1 : (*date_v2_value1 < *date_v2_value2 ? -1 : 0);
+    }
+
+    case TYPE_DATETIMEV2: {
+        auto date_v2_value1 = reinterpret_cast<
+                const doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>*>(v1);
+        auto date_v2_value2 = reinterpret_cast<
+                const doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>*>(v2);
         return *date_v2_value1 > *date_v2_value2 ? 1 : (*date_v2_value1 < *date_v2_value2 ? -1 : 0);
     }
 
diff --git a/be/src/runtime/raw_value.h b/be/src/runtime/raw_value.h
index a83a64bb1e..7c5f990061 100644
--- a/be/src/runtime/raw_value.h
+++ b/be/src/runtime/raw_value.h
@@ -162,8 +162,16 @@ inline bool RawValue::lt(const void* v1, const void* v2, const TypeDescriptor& t
                *reinterpret_cast<const DateTimeValue*>(v2);
 
     case TYPE_DATEV2:
-        return *reinterpret_cast<const vectorized::DateV2Value*>(v1) <
-               *reinterpret_cast<const vectorized::DateV2Value*>(v2);
+        return *reinterpret_cast<
+                       const vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*>(v1) <
+               *reinterpret_cast<
+                       const vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*>(v2);
+
+    case TYPE_DATETIMEV2:
+        return *reinterpret_cast<
+                       const vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>*>(v1) <
+               *reinterpret_cast<
+                       const vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>*>(v2);
 
     case TYPE_DECIMALV2:
         return reinterpret_cast<const PackedInt128*>(v1)->value <
@@ -226,8 +234,17 @@ inline bool RawValue::eq(const void* v1, const void* v2, const TypeDescriptor& t
                *reinterpret_cast<const DateTimeValue*>(v2);
 
     case TYPE_DATEV2:
-        return *reinterpret_cast<const vectorized::DateV2Value*>(v1) ==
-               *reinterpret_cast<const vectorized::DateV2Value*>(v2);
+        return *reinterpret_cast<
+                       const vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*>(v1) ==
+               *reinterpret_cast<
+                       const vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*>(v2);
+
+    case TYPE_DATETIMEV2:
+        return *reinterpret_cast<
+                       const vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>*>(
+                       v1) ==
+               *reinterpret_cast<
+                       const vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>*>(v2);
 
     case TYPE_DECIMALV2:
         return reinterpret_cast<const PackedInt128*>(v1)->value ==
@@ -301,6 +318,9 @@ inline uint32_t RawValue::get_hash_value(const void* v, const PrimitiveType& typ
     case TYPE_DATEV2:
         return HashUtil::hash(v, 4, seed);
 
+    case TYPE_DATETIMEV2:
+        return HashUtil::hash(v, 8, seed);
+
     case TYPE_DECIMALV2:
         return HashUtil::hash(v, 16, seed);
     case TYPE_DECIMAL32:
@@ -367,6 +387,9 @@ inline uint32_t RawValue::get_hash_value_fvn(const void* v, const PrimitiveType&
     case TYPE_DATEV2:
         return HashUtil::fnv_hash(v, 4, seed);
 
+    case TYPE_DATETIMEV2:
+        return HashUtil::fnv_hash(v, 8, seed);
+
     case TYPE_DECIMALV2:
         return HashUtil::fnv_hash(v, 16, seed);
     case TYPE_DECIMAL32:
@@ -436,7 +459,16 @@ inline uint32_t RawValue::zlib_crc32(const void* v, const TypeDescriptor& type,
         return HashUtil::zlib_crc_hash(buf, len, seed);
     }
     case TYPE_DATEV2: {
-        const vectorized::DateV2Value* date_v2_val = (const vectorized::DateV2Value*)v;
+        const vectorized::DateV2Value<doris::vectorized::DateV2ValueType>* date_v2_val =
+                (const vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*)v;
+        char buf[64];
+        int len = date_v2_val->to_buffer(buf);
+        return HashUtil::zlib_crc_hash(buf, len, seed);
+    }
+
+    case TYPE_DATETIMEV2: {
+        const vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>* date_v2_val =
+                (const vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>*)v;
         char buf[64];
         int len = date_v2_val->to_buffer(buf);
         return HashUtil::zlib_crc_hash(buf, len, seed);
@@ -502,8 +534,17 @@ inline uint32_t RawValue::zlib_crc32(const void* v, size_t len, const TypeDescri
         int len = date_val->to_buffer(buf);
         return HashUtil::zlib_crc_hash(buf, len, seed);
     }
+
     case TYPE_DATEV2: {
-        auto* date_v2_val = (const vectorized::DateV2Value*)v;
+        auto* date_v2_val = (const vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*)v;
+        char buf[64];
+        int date_v2_len = date_v2_val->to_buffer(buf);
+        return HashUtil::zlib_crc_hash(buf, date_v2_len, seed);
+    }
+
+    case TYPE_DATETIMEV2: {
+        auto* date_v2_val =
+                (const vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>*)v;
         char buf[64];
         int date_v2_len = date_v2_val->to_buffer(buf);
         return HashUtil::zlib_crc_hash(buf, date_v2_len, seed);
diff --git a/be/src/runtime/type_limit.h b/be/src/runtime/type_limit.h
index b868e25447..923f2891e9 100644
--- a/be/src/runtime/type_limit.h
+++ b/be/src/runtime/type_limit.h
@@ -48,14 +48,32 @@ struct type_limit<DateTimeValue> {
 };
 
 template <>
-struct type_limit<doris::vectorized::DateV2Value> {
-    static doris::vectorized::DateV2Value min() {
+struct type_limit<doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>> {
+    static doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> min() {
         uint32_t min = doris::vectorized::MIN_DATE_V2;
-        return binary_cast<uint32_t, doris::vectorized::DateV2Value>(min);
+        return binary_cast<uint32_t,
+                           doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>>(min);
     }
-    static doris::vectorized::DateV2Value max() {
+    static doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> max() {
         uint32_t max = doris::vectorized::MAX_DATE_V2;
-        return binary_cast<uint32_t, doris::vectorized::DateV2Value>(max);
+        return binary_cast<uint32_t,
+                           doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>>(max);
+    }
+};
+
+template <>
+struct type_limit<doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>> {
+    static doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> min() {
+        uint64_t min = doris::vectorized::MIN_DATETIME_V2;
+        return binary_cast<uint64_t,
+                           doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>>(
+                min);
+    }
+    static doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> max() {
+        uint64_t max = doris::vectorized::MAX_DATETIME_V2;
+        return binary_cast<uint64_t,
+                           doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>>(
+                max);
     }
 };
 
diff --git a/be/src/runtime/types.cpp b/be/src/runtime/types.cpp
index c5f3eb89b9..32285c48c2 100644
--- a/be/src/runtime/types.cpp
+++ b/be/src/runtime/types.cpp
@@ -38,7 +38,7 @@ TypeDescriptor::TypeDescriptor(const std::vector<TTypeNode>& types, int* idx)
             DCHECK(scalar_type.__isset.len);
             len = scalar_type.len;
         } else if (type == TYPE_DECIMALV2 || type == TYPE_DECIMAL32 || type == TYPE_DECIMAL64 ||
-                   type == TYPE_DECIMAL128) {
+                   type == TYPE_DECIMAL128 || type == TYPE_DATETIMEV2) {
             DCHECK(scalar_type.__isset.precision);
             DCHECK(scalar_type.__isset.scale);
             precision = scalar_type.precision;
@@ -115,7 +115,7 @@ void TypeDescriptor::to_thrift(TTypeDesc* thrift_type) const {
             // DCHECK_NE(len, -1);
             scalar_type.__set_len(len);
         } else if (type == TYPE_DECIMALV2 || type == TYPE_DECIMAL32 || type == TYPE_DECIMAL64 ||
-                   type == TYPE_DECIMAL128) {
+                   type == TYPE_DECIMAL128 || type == TYPE_DATETIMEV2) {
             DCHECK_NE(precision, -1);
             DCHECK_NE(scale, -1);
             scalar_type.__set_precision(precision);
@@ -134,7 +134,7 @@ void TypeDescriptor::to_protobuf(PTypeDesc* ptype) const {
     if (type == TYPE_CHAR || type == TYPE_VARCHAR || type == TYPE_HLL) {
         scalar_type->set_len(len);
     } else if (type == TYPE_DECIMALV2 || type == TYPE_DECIMAL32 || type == TYPE_DECIMAL64 ||
-               type == TYPE_DECIMAL128) {
+               type == TYPE_DECIMAL128 || type == TYPE_DATETIMEV2) {
         DCHECK_NE(precision, -1);
         DCHECK_NE(scale, -1);
         scalar_type->set_precision(precision);
@@ -162,7 +162,7 @@ TypeDescriptor::TypeDescriptor(const google::protobuf::RepeatedPtrField<PTypeNod
             DCHECK(scalar_type.has_len());
             len = scalar_type.len();
         } else if (type == TYPE_DECIMALV2 || type == TYPE_DECIMAL32 || type == TYPE_DECIMAL64 ||
-                   type == TYPE_DECIMAL128) {
+                   type == TYPE_DECIMAL128 || type == TYPE_DATETIMEV2) {
             DCHECK(scalar_type.has_precision());
             DCHECK(scalar_type.has_scale());
             precision = scalar_type.precision();
diff --git a/be/src/runtime/types.h b/be/src/runtime/types.h
index 1eb392ab02..9c20e4092f 100644
--- a/be/src/runtime/types.h
+++ b/be/src/runtime/types.h
@@ -167,7 +167,8 @@ struct TypeDescriptor {
 
     bool is_date_type() const { return type == TYPE_DATE || type == TYPE_DATETIME; }
 
-    bool is_date_v2_type() const { return type == TYPE_DATEV2 || type == TYPE_DATETIMEV2; }
+    bool is_date_v2_type() const { return type == TYPE_DATEV2; }
+    bool is_datetime_v2_type() const { return type == TYPE_DATETIMEV2; }
 
     bool is_decimal_type() const { return (type == TYPE_DECIMALV2); }
 
diff --git a/be/src/udf/udf.h b/be/src/udf/udf.h
index b4916fe7b4..959deac8c4 100644
--- a/be/src/udf/udf.h
+++ b/be/src/udf/udf.h
@@ -714,6 +714,32 @@ struct DateV2Val : public AnyVal {
     bool operator!=(const DateV2Val& other) const { return !(*this == other); }
 };
 
+struct DateTimeV2Val : public AnyVal {
+    uint64_t datetimev2_value;
+
+    DateTimeV2Val() {}
+    DateTimeV2Val(uint64_t val) : datetimev2_value(val) {}
+
+    static DateTimeV2Val null() {
+        DateTimeV2Val result;
+        result.is_null = true;
+        return result;
+    }
+
+    bool operator==(const DateTimeV2Val& other) const {
+        if (is_null && other.is_null) {
+            return true;
+        }
+
+        if (is_null || other.is_null) {
+            return false;
+        }
+
+        return datetimev2_value == other.datetimev2_value;
+    }
+    bool operator!=(const DateTimeV2Val& other) const { return !(*this == other); }
+};
+
 // Note: there is a difference between a nullptr string (is_null == true) and an
 // empty string (len == 0).
 struct StringVal : public AnyVal {
diff --git a/be/src/util/binary_cast.hpp b/be/src/util/binary_cast.hpp
index 013c9e598e..1996946163 100644
--- a/be/src/util/binary_cast.hpp
+++ b/be/src/util/binary_cast.hpp
@@ -64,11 +64,17 @@ union VecDateTimeInt64Union {
 };
 
 union DateV2UInt32Union {
-    doris::vectorized::DateV2Value dt;
+    doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> dt;
     uint32_t ui32;
     ~DateV2UInt32Union() {}
 };
 
+union DateTimeV2UInt64Union {
+    doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> dt;
+    uint64_t ui64;
+    ~DateTimeV2UInt64Union() {}
+};
+
 // similar to reinterpret_cast but won't break strict-aliasing rules
 template <typename From, typename To>
 To binary_cast(From from) {
@@ -87,15 +93,26 @@ To binary_cast(From from) {
     constexpr bool from_decv2_to_i128 = match_v<From, DecimalV2Value, To, __int128_t>;
 
     constexpr bool from_ui32_to_date_v2 =
-            match_v<From, uint32_t, To, doris::vectorized::DateV2Value>;
+            match_v<From, uint32_t, To,
+                    doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>>;
 
     constexpr bool from_date_v2_to_ui32 =
-            match_v<From, doris::vectorized::DateV2Value, To, uint32_t>;
+            match_v<From, doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>, To,
+                    uint32_t>;
+
+    constexpr bool from_ui64_to_datetime_v2 =
+            match_v<From, uint64_t, To,
+                    doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>>;
+
+    constexpr bool from_datetime_v2_to_ui64 =
+            match_v<From, doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>,
+                    To, uint64_t>;
 
     static_assert(from_u64_to_db || from_i64_to_db || from_db_to_i64 || from_db_to_u64 ||
                   from_decv2_to_packed128 || from_i128_to_dt || from_dt_to_i128 ||
                   from_i64_to_vec_dt || from_vec_dt_to_i64 || from_i128_to_decv2 ||
-                  from_decv2_to_i128 || from_ui32_to_date_v2 || from_date_v2_to_ui32);
+                  from_decv2_to_i128 || from_ui32_to_date_v2 || from_date_v2_to_ui32 ||
+                  from_ui64_to_datetime_v2 || from_datetime_v2_to_ui64);
 
     if constexpr (from_u64_to_db) {
         TypeConverter conv;
@@ -132,6 +149,12 @@ To binary_cast(From from) {
     } else if constexpr (from_date_v2_to_ui32) {
         DateV2UInt32Union conv = {.dt = from};
         return conv.ui32;
+    } else if constexpr (from_ui64_to_datetime_v2) {
+        DateTimeV2UInt64Union conv = {.ui64 = from};
+        return conv.dt;
+    } else if constexpr (from_datetime_v2_to_ui64) {
+        DateTimeV2UInt64Union conv = {.dt = from};
+        return conv.ui64;
     } else if constexpr (from_vec_dt_to_i64) {
         VecDateTimeInt64Union conv = {.dt = from};
         return conv.i64;
diff --git a/be/src/util/date_func.cpp b/be/src/util/date_func.cpp
index 7dca6f1200..bebb691afd 100644
--- a/be/src/util/date_func.cpp
+++ b/be/src/util/date_func.cpp
@@ -69,6 +69,14 @@ uint32_t timestamp_from_date_v2(const std::string& date_str) {
 
     return value;
 }
+
+uint64_t timestamp_from_datetime_v2(const std::string& date_str) {
+    doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType> val;
+    std::string date_format = "%Y-%m-%d %H:%i:%s.%f";
+    val.from_date_format_str(date_format.data(), date_format.size(), date_str.data(),
+                             date_str.size());
+    return val.to_date_int_val();
+}
 // refer to https://dev.mysql.com/doc/refman/5.7/en/time.html
 // the time value between '-838:59:59' and '838:59:59'
 int32_t time_to_buffer_from_double(double time, char* buffer) {
diff --git a/be/src/util/date_func.h b/be/src/util/date_func.h
index 507ee9652e..a10bfd45d4 100644
--- a/be/src/util/date_func.h
+++ b/be/src/util/date_func.h
@@ -30,5 +30,6 @@ uint64_t timestamp_from_datetime(const std::string& datetime_str);
 uint24_t timestamp_from_date(const std::string& date_str);
 int32_t time_to_buffer_from_double(double time, char* buffer);
 uint32_t timestamp_from_date_v2(const std::string& date_str);
+uint64_t timestamp_from_datetime_v2(const std::string& date_str);
 
 } // namespace doris
diff --git a/be/src/util/symbols_util.cpp b/be/src/util/symbols_util.cpp
index d60de4224b..06df655082 100644
--- a/be/src/util/symbols_util.cpp
+++ b/be/src/util/symbols_util.cpp
@@ -159,12 +159,14 @@ static void append_any_val_type(int namespace_id, const TypeDescriptor& type,
         break;
     case TYPE_DATE:
     case TYPE_DATETIME:
-    case TYPE_DATETIMEV2:
         append_mangled_token("DateTimeVal", s);
         break;
     case TYPE_DATEV2:
         append_mangled_token("DateV2Val", s);
         break;
+    case TYPE_DATETIMEV2:
+        append_mangled_token("DateTimeV2Val", s);
+        break;
     case TYPE_DECIMALV2:
         append_mangled_token("DecimalV2Val", s);
         break;
diff --git a/be/src/vec/aggregate_functions/aggregate_function_min_max.cpp b/be/src/vec/aggregate_functions/aggregate_function_min_max.cpp
index 43b6717c04..fb48cc18b8 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_min_max.cpp
+++ b/be/src/vec/aggregate_functions/aggregate_function_min_max.cpp
@@ -54,6 +54,10 @@ static IAggregateFunction* create_aggregate_function_single_value(const String&
         return new AggregateFunctionTemplate<Data<SingleValueDataFixed<UInt32>>, false>(
                 argument_type);
     }
+    if (which.idx == TypeIndex::DateTimeV2) {
+        return new AggregateFunctionTemplate<Data<SingleValueDataFixed<UInt64>>, false>(
+                argument_type);
+    }
     if (which.idx == TypeIndex::Decimal32) {
         return new AggregateFunctionTemplate<Data<SingleValueDataDecimal<Decimal32>>, false>(
                 argument_type);
diff --git a/be/src/vec/aggregate_functions/aggregate_function_window.h b/be/src/vec/aggregate_functions/aggregate_function_window.h
index 1cd763c1c3..6e0dba239e 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_window.h
+++ b/be/src/vec/aggregate_functions/aggregate_function_window.h
@@ -557,6 +557,10 @@ static IAggregateFunction* create_function_single_value(const String& name,
         return new AggregateFunctionTemplate<
                 Data<LeadAndLagData<UInt32, result_is_nullable, false, StoreType>>>(argument_types);
     }
+    if (which.is_date_time_v2()) {
+        return new AggregateFunctionTemplate<
+                Data<LeadAndLagData<UInt64, result_is_nullable, false, StoreType>>>(argument_types);
+    }
     if (which.is_string_or_fixed_string()) {
         return new AggregateFunctionTemplate<
                 Data<LeadAndLagData<StringRef, result_is_nullable, true, StoreType>>>(
diff --git a/be/src/vec/columns/column.h b/be/src/vec/columns/column.h
index 4513b12ec4..cdc6fed9bf 100644
--- a/be/src/vec/columns/column.h
+++ b/be/src/vec/columns/column.h
@@ -514,11 +514,13 @@ public:
 
     virtual bool is_date_type() const { return is_date; }
     virtual bool is_date_v2_type() const { return is_date_v2; }
+    virtual bool is_datetime_v2_type() const { return is_datetime_v2; }
     virtual bool is_datetime_type() const { return is_date_time; }
     virtual bool is_decimalv2_type() const { return is_decimalv2; }
 
     virtual void set_date_type() { is_date = true; }
     virtual void set_date_v2_type() { is_date_v2 = true; }
+    virtual void set_datetime_v2_type() { is_datetime_v2 = true; }
     virtual void set_datetime_type() { is_date_time = true; }
     virtual void set_decimalv2_type() { is_decimalv2 = true; }
 
@@ -526,6 +528,7 @@ public:
     bool is_date = false;
     bool is_date_time = false;
     bool is_date_v2 = false;
+    bool is_datetime_v2 = false;
     bool is_decimalv2 = false;
 
 protected:
diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h
index 6bd37d036f..37249a858a 100644
--- a/be/src/vec/columns/column_nullable.h
+++ b/be/src/vec/columns/column_nullable.h
@@ -181,6 +181,7 @@ public:
 
     bool is_date_type() const override { return get_nested_column().is_date_type(); }
     bool is_date_v2_type() const override { return get_nested_column().is_date_v2_type(); }
+    bool is_datetime_v2_type() const override { return get_nested_column().is_datetime_v2_type(); }
     bool is_datetime_type() const override { return get_nested_column().is_datetime_type(); }
     bool is_decimalv2_type() const override { return get_nested_column().is_decimalv2_type(); }
     void set_date_type() override { get_nested_column().set_date_type(); }
diff --git a/be/src/vec/columns/columns_number.h b/be/src/vec/columns/columns_number.h
index af878136d3..2cc96d3f39 100644
--- a/be/src/vec/columns/columns_number.h
+++ b/be/src/vec/columns/columns_number.h
@@ -40,9 +40,11 @@ using ColumnInt64 = ColumnVector<Int64>;
 using ColumnInt128 = ColumnVector<Int128>;
 
 using ColumnDate = ColumnVector<Int64>;
-using ColumnDateV2 = ColumnVector<UInt32>;
 using ColumnDateTime = ColumnVector<Int64>;
 
+using ColumnDateV2 = ColumnVector<UInt32>;
+using ColumnDateTimeV2 = ColumnVector<UInt64>;
+
 using ColumnFloat32 = ColumnVector<Float32>;
 using ColumnFloat64 = ColumnVector<Float64>;
 
diff --git a/be/src/vec/columns/predicate_column.h b/be/src/vec/columns/predicate_column.h
index d99ee9a12c..daa4163f05 100644
--- a/be/src/vec/columns/predicate_column.h
+++ b/be/src/vec/columns/predicate_column.h
@@ -433,8 +433,15 @@ public:
                     reinterpret_cast<vectorized::ColumnVector<doris::vectorized::Float64>*>(
                             col_ptr));
         } else if constexpr (std::is_same_v<T, uint64_t>) {
-            insert_datetime_to_res_column(
-                    sel, sel_size, reinterpret_cast<vectorized::ColumnVector<Int64>*>(col_ptr));
+            if (const vectorized::ColumnVector<UInt64>* date_col =
+                        check_and_get_column<vectorized::ColumnVector<UInt64>>(
+                                const_cast<const IColumn*>(col_ptr))) {
+                insert_default_value_res_column(
+                        sel, sel_size, const_cast<vectorized::ColumnVector<UInt64>*>(date_col));
+            } else {
+                insert_datetime_to_res_column(
+                        sel, sel_size, reinterpret_cast<vectorized::ColumnVector<Int64>*>(col_ptr));
+            }
         } else if constexpr (std::is_same_v<T, uint24_t>) {
             insert_date_to_res_column(sel, sel_size,
                                       reinterpret_cast<vectorized::ColumnVector<Int64>*>(col_ptr));
diff --git a/be/src/vec/core/call_on_type_index.h b/be/src/vec/core/call_on_type_index.h
index d62288611d..0e9dd5468c 100644
--- a/be/src/vec/core/call_on_type_index.h
+++ b/be/src/vec/core/call_on_type_index.h
@@ -160,6 +160,7 @@ inline bool call_on_basic_types(TypeIndex type_num1, TypeIndex type_num2, F&& f)
 
 class DataTypeDate;
 class DataTypeDateV2;
+class DataTypeDateTimeV2;
 class DataTypeDateTime;
 class DataTypeString;
 template <typename T>
@@ -208,6 +209,8 @@ bool call_on_index_and_data_type(TypeIndex number, F&& f) {
         return f(TypePair<DataTypeDate, T>());
     case TypeIndex::DateV2:
         return f(TypePair<DataTypeDateV2, T>());
+    case TypeIndex::DateTimeV2:
+        return f(TypePair<DataTypeDateTimeV2, T>());
     case TypeIndex::DateTime:
         return f(TypePair<DataTypeDateTime, T>());
 
diff --git a/be/src/vec/core/types.h b/be/src/vec/core/types.h
index a9c61bb31e..def50d5e01 100644
--- a/be/src/vec/core/types.h
+++ b/be/src/vec/core/types.h
@@ -259,6 +259,7 @@ struct TypeId<Int128> {
 using Date = Int64;
 using DateTime = Int64;
 using DateV2 = UInt32;
+using DateTimeV2 = UInt64;
 
 /// Own FieldType for Decimal.
 /// It is only a "storage" for decimal. To perform operations, you also have to provide a scale (number of digits after point).
diff --git a/be/src/vec/data_types/data_type.cpp b/be/src/vec/data_types/data_type.cpp
index 4be9a6112a..f04bd0c592 100644
--- a/be/src/vec/data_types/data_type.cpp
+++ b/be/src/vec/data_types/data_type.cpp
@@ -135,6 +135,8 @@ PGenericType_TypeId IDataType::get_pdata_type(const IDataType* data_type) {
         return PGenericType::DATEV2;
     case TypeIndex::DateTime:
         return PGenericType::DATETIME;
+    case TypeIndex::DateTimeV2:
+        return PGenericType::DATETIMEV2;
     case TypeIndex::BitMap:
         return PGenericType::BITMAP;
     case TypeIndex::HLL:
diff --git a/be/src/vec/data_types/data_type.h b/be/src/vec/data_types/data_type.h
index 047c9fa0b6..ccb3d1fce0 100644
--- a/be/src/vec/data_types/data_type.h
+++ b/be/src/vec/data_types/data_type.h
@@ -324,6 +324,9 @@ inline bool is_date(const DataTypePtr& data_type) {
 inline bool is_date_v2(const DataTypePtr& data_type) {
     return WhichDataType(data_type).is_date_v2();
 }
+inline bool is_date_time_v2(const DataTypePtr& data_type) {
+    return WhichDataType(data_type).is_date_time_v2();
+}
 inline bool is_date_or_datetime(const DataTypePtr& data_type) {
     return WhichDataType(data_type).is_date_or_datetime();
 }
@@ -385,7 +388,7 @@ template <typename T>
 inline bool is_columned_as_number(const T& data_type) {
     WhichDataType which(data_type);
     return which.is_int() || which.is_uint() || which.is_float() || which.is_date_or_datetime() ||
-           which.is_uuid() || which.is_date_v2();
+           which.is_uuid() || which.is_date_v2_or_datetime_v2();
 }
 
 template <typename T>
diff --git a/be/src/vec/data_types/data_type_date_time.h b/be/src/vec/data_types/data_type_date_time.h
index ebee94ac70..d97f8bad99 100644
--- a/be/src/vec/data_types/data_type_date_time.h
+++ b/be/src/vec/data_types/data_type_date_time.h
@@ -89,8 +89,13 @@ inline constexpr bool IsDateV2Type<DataTypeDateV2> = true;
 
 template <typename DataType>
 constexpr bool IsDateTimeV2Type = false;
+template <>
+inline constexpr bool IsDateTimeV2Type<DataTypeDateTimeV2> = true;
 
 template <typename DataType>
 constexpr bool IsTimeType = IsDateTimeType<DataType> || IsDateType<DataType>;
 
+template <typename DataType>
+constexpr bool IsTimeV2Type = IsDateTimeV2Type<DataType> || IsDateV2Type<DataType>;
+
 } // namespace doris::vectorized
diff --git a/be/src/vec/data_types/data_type_factory.cpp b/be/src/vec/data_types/data_type_factory.cpp
index c8be27a6ab..0025f86ec0 100644
--- a/be/src/vec/data_types/data_type_factory.cpp
+++ b/be/src/vec/data_types/data_type_factory.cpp
@@ -87,6 +87,8 @@ 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>();
+        break;
     case TYPE_DATETIME:
         nested = std::make_shared<vectorized::DataTypeDateTime>();
         break;
@@ -165,6 +167,9 @@ DataTypePtr DataTypeFactory::_create_primitive_data_type(const FieldType& type,
     case OLAP_FIELD_TYPE_DATEV2:
         result = std::make_shared<vectorized::DataTypeDateV2>();
         break;
+    case OLAP_FIELD_TYPE_DATETIMEV2:
+        result = std::make_shared<vectorized::DataTypeDateTimeV2>();
+        break;
     case OLAP_FIELD_TYPE_DATETIME:
         result = std::make_shared<vectorized::DataTypeDateTime>();
         break;
@@ -246,6 +251,9 @@ DataTypePtr DataTypeFactory::create_data_type(const PColumnMeta& pcolumn) {
     case PGenericType::DATEV2:
         nested = std::make_shared<DataTypeDateV2>();
         break;
+    case PGenericType::DATETIMEV2:
+        nested = std::make_shared<DataTypeDateTimeV2>();
+        break;
     case PGenericType::DATETIME:
         nested = std::make_shared<DataTypeDateTime>();
         break;
diff --git a/be/src/vec/data_types/data_type_factory.hpp b/be/src/vec/data_types/data_type_factory.hpp
index 674bb5515f..37dda1ee54 100644
--- a/be/src/vec/data_types/data_type_factory.hpp
+++ b/be/src/vec/data_types/data_type_factory.hpp
@@ -65,6 +65,7 @@ public:
                     {"Date", std::make_shared<DataTypeDate>()},
                     {"DateV2", std::make_shared<DataTypeDateV2>()},
                     {"DateTime", std::make_shared<DataTypeDateTime>()},
+                    {"DateTimeV2", std::make_shared<DataTypeDateTimeV2>()},
                     {"String", std::make_shared<DataTypeString>()},
                     {"Decimal", std::make_shared<DataTypeDecimal<Decimal128>>(27, 9)},
                     {"Decimal32", std::make_shared<DataTypeDecimal<Decimal32>>(
diff --git a/be/src/vec/data_types/data_type_time_v2.cpp b/be/src/vec/data_types/data_type_time_v2.cpp
index f9f48eed5b..af0f183031 100644
--- a/be/src/vec/data_types/data_type_time_v2.cpp
+++ b/be/src/vec/data_types/data_type_time_v2.cpp
@@ -30,7 +30,7 @@ std::string DataTypeDateV2::to_string(const IColumn& column, size_t row_num) con
     UInt32 int_val =
             assert_cast<const ColumnUInt32&>(*column.convert_to_full_column_if_const().get())
                     .get_data()[row_num];
-    DateV2Value val = binary_cast<UInt32, DateV2Value>(int_val);
+    DateV2Value<DateV2ValueType> val = binary_cast<UInt32, DateV2Value<DateV2ValueType>>(int_val);
     std::stringstream ss;
     ss << val;
     return ss.str();
@@ -40,7 +40,7 @@ void DataTypeDateV2::to_string(const IColumn& column, size_t row_num, BufferWrit
     UInt32 int_val =
             assert_cast<const ColumnUInt32&>(*column.convert_to_full_column_if_const().get())
                     .get_data()[row_num];
-    DateV2Value value = binary_cast<UInt32, DateV2Value>(int_val);
+    DateV2Value<DateV2ValueType> value = binary_cast<UInt32, DateV2Value<DateV2ValueType>>(int_val);
 
     char buf[64];
     char* pos = value.to_string(buf);
@@ -54,16 +54,70 @@ MutableColumnPtr DataTypeDateV2::create_column() const {
     return col;
 }
 
-void DataTypeDateV2::cast_to_date_time(const UInt32& from, Int64& to) {
+void DataTypeDateV2::cast_to_date_time(const UInt32 from, Int64& to) {
     auto& to_value = (doris::vectorized::VecDateTimeValue&)to;
-    auto& from_value = (doris::vectorized::DateV2Value&)from;
+    auto& from_value = (doris::vectorized::DateV2Value<DateV2ValueType>&)from;
     to_value.create_from_date_v2(from_value, TimeType::TIME_DATETIME);
 }
 
-void DataTypeDateV2::cast_to_date(const UInt32& from, Int64& to) {
+void DataTypeDateV2::cast_to_date(const UInt32 from, Int64& to) {
     auto& to_value = (doris::vectorized::VecDateTimeValue&)(to);
-    auto& from_value = (doris::vectorized::DateV2Value&)from;
+    auto& from_value = (doris::vectorized::DateV2Value<DateV2ValueType>&)from;
     to_value.create_from_date_v2(from_value, TimeType::TIME_DATE);
 }
 
+void DataTypeDateV2::cast_to_date_time_v2(const UInt32 from, UInt64& to) {
+    to = (UInt64)from << TIME_PART_LENGTH;
+}
+
+bool DataTypeDateTimeV2::equals(const IDataType& rhs) const {
+    return typeid(rhs) == typeid(*this);
+}
+
+std::string DataTypeDateTimeV2::to_string(const IColumn& column, size_t row_num) const {
+    UInt64 int_val =
+            assert_cast<const ColumnUInt64&>(*column.convert_to_full_column_if_const().get())
+                    .get_data()[row_num];
+    DateV2Value<DateTimeV2ValueType> val =
+            binary_cast<UInt64, DateV2Value<DateTimeV2ValueType>>(int_val);
+    std::stringstream ss;
+    ss << val;
+    return ss.str();
+}
+
+void DataTypeDateTimeV2::to_string(const IColumn& column, size_t row_num,
+                                   BufferWritable& ostr) const {
+    UInt64 int_val =
+            assert_cast<const ColumnUInt64&>(*column.convert_to_full_column_if_const().get())
+                    .get_data()[row_num];
+    DateV2Value<DateTimeV2ValueType> value =
+            binary_cast<UInt64, DateV2Value<DateTimeV2ValueType>>(int_val);
+
+    char buf[64];
+    char* pos = value.to_string(buf);
+    // DateTime to_string the end is /0
+    ostr.write(buf, pos - buf - 1);
+}
+
+MutableColumnPtr DataTypeDateTimeV2::create_column() const {
+    auto col = DataTypeNumberBase<UInt64>::create_column();
+    col->set_datetime_v2_type();
+    return col;
+}
+
+void DataTypeDateTimeV2::cast_to_date_time(const UInt64 from, Int64& to) {
+    auto& to_value = (doris::vectorized::VecDateTimeValue&)to;
+    auto& from_value = (doris::vectorized::DateV2Value<DateTimeV2ValueType>&)from;
+    to_value.create_from_date_v2(from_value, TimeType::TIME_DATETIME);
+}
+
+void DataTypeDateTimeV2::cast_to_date(const UInt64 from, Int64& to) {
+    auto& to_value = (doris::vectorized::VecDateTimeValue&)(to);
+    auto& from_value = (doris::vectorized::DateV2Value<DateTimeV2ValueType>&)from;
+    to_value.create_from_date_v2(from_value, TimeType::TIME_DATE);
+}
+
+void DataTypeDateTimeV2::cast_to_date_v2(const UInt64 from, UInt32& to) {
+    to = from >> TIME_PART_LENGTH;
+}
 } // namespace doris::vectorized
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 c126b0966b..13d54d1490 100644
--- a/be/src/vec/data_types/data_type_time_v2.h
+++ b/be/src/vec/data_types/data_type_time_v2.h
@@ -40,8 +40,29 @@ public:
 
     MutableColumnPtr create_column() const override;
 
-    static void cast_to_date(const UInt32& from, Int64& to);
-    static void cast_to_date_time(const UInt32& from, Int64& to);
+    static void cast_to_date(const UInt32 from, Int64& to);
+    static void cast_to_date_time(const UInt32 from, Int64& to);
+    static void cast_to_date_time_v2(const UInt32 from, UInt64& to);
+};
+
+class DataTypeDateTimeV2 final : public DataTypeNumberBase<UInt64> {
+public:
+    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"; }
+
+    bool can_be_used_as_version() const override { return true; }
+    bool can_be_inside_nullable() const override { return true; }
+
+    bool equals(const IDataType& rhs) const override;
+    std::string to_string(const IColumn& column, size_t row_num) const override;
+    void to_string(const IColumn& column, size_t row_num, BufferWritable& ostr) const override;
+
+    MutableColumnPtr create_column() const override;
+
+    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);
 };
 
 } // namespace doris::vectorized
diff --git a/be/src/vec/data_types/get_least_supertype.cpp b/be/src/vec/data_types/get_least_supertype.cpp
index 1aa9da5c55..4d22ab4870 100644
--- a/be/src/vec/data_types/get_least_supertype.cpp
+++ b/be/src/vec/data_types/get_least_supertype.cpp
@@ -149,13 +149,17 @@ DataTypePtr get_least_supertype(const DataTypes& types) {
     {
         UInt32 have_date_v2 = type_ids.count(TypeIndex::DateV2);
 
-        if (have_date_v2) {
-            if (type_ids.size() != have_date_v2) {
-                LOG(FATAL) << get_exception_message_prefix(types)
-                           << " because some of them are DateV2 and some of them are not";
+        UInt32 have_datetime_v2 = type_ids.count(TypeIndex::DateTimeV2);
+
+        if (have_date_v2 || have_datetime_v2) {
+            bool all_datev2_or_datetimev2 = type_ids.size() == (have_date_v2 + have_datetime_v2);
+            if (!all_datev2_or_datetimev2) {
+                LOG(FATAL)
+                        << get_exception_message_prefix(types)
+                        << " because some of them are DateV2/DateTimeV2 and some of them are not";
             }
 
-            return std::make_shared<DataTypeDateV2>();
+            return std::make_shared<DataTypeDateTimeV2>();
         }
     }
 
diff --git a/be/src/vec/exec/join/vhash_join_node.cpp b/be/src/vec/exec/join/vhash_join_node.cpp
index 26eb565fae..784cb1af0f 100644
--- a/be/src/vec/exec/join/vhash_join_node.cpp
+++ b/be/src/vec/exec/join/vhash_join_node.cpp
@@ -1349,6 +1349,7 @@ void HashJoinNode::_hash_table_init() {
         case TYPE_DOUBLE:
         case TYPE_DATETIME:
         case TYPE_DATE:
+        case TYPE_DATETIMEV2:
             _hash_table_variants.emplace<I64HashTableContext>();
             break;
         case TYPE_LARGEINT:
diff --git a/be/src/vec/exec/vaggregation_node.cpp b/be/src/vec/exec/vaggregation_node.cpp
index c5f3d7ddf9..4fc2b1eda5 100644
--- a/be/src/vec/exec/vaggregation_node.cpp
+++ b/be/src/vec/exec/vaggregation_node.cpp
@@ -148,6 +148,7 @@ void AggregationNode::_init_hash_method(std::vector<VExprContext*>& probe_exprs)
         case TYPE_DOUBLE:
         case TYPE_DATE:
         case TYPE_DATETIME:
+        case TYPE_DATETIMEV2:
             _agg_data.init(AggregatedDataVariants::Type::int64_key, is_nullable);
             return;
         case TYPE_LARGEINT:
diff --git a/be/src/vec/exec/volap_scan_node.cpp b/be/src/vec/exec/volap_scan_node.cpp
index 544aacdd3d..2e86210a50 100644
--- a/be/src/vec/exec/volap_scan_node.cpp
+++ b/be/src/vec/exec/volap_scan_node.cpp
@@ -707,6 +707,14 @@ Status VOlapScanNode::normalize_conjuncts() {
             break;
         }
 
+        case TYPE_DATETIMEV2: {
+            ColumnValueRange<TYPE_DATETIMEV2> range(slots[slot_idx]->col_name(),
+                                                    slots[slot_idx]->type().precision,
+                                                    slots[slot_idx]->type().scale);
+            normalize_predicate(range, slots[slot_idx]);
+            break;
+        }
+
         case TYPE_DECIMALV2: {
             ColumnValueRange<TYPE_DECIMALV2> range(slots[slot_idx]->col_name());
             normalize_predicate(range, slots[slot_idx]);
@@ -946,8 +954,10 @@ Status VOlapScanNode::normalize_predicate(ColumnValueRange<T>& range, SlotDescri
 }
 
 static bool ignore_cast(SlotDescriptor* slot, Expr* expr) {
-    if ((slot->type().is_date_type() || slot->type().is_date_v2_type()) &&
-        (expr->type().is_date_type() || expr->type().is_date_v2_type())) {
+    if ((slot->type().is_date_type() || slot->type().is_date_v2_type() ||
+         slot->type().is_datetime_v2_type()) &&
+        (expr->type().is_date_type() || expr->type().is_date_v2_type() ||
+         expr->type().is_datetime_v2_type())) {
         return true;
     }
     if (slot->type().is_string_type() && expr->type().is_string_type()) {
@@ -1060,6 +1070,7 @@ Status VOlapScanNode::change_fixed_value_range(ColumnValueRange<primitive_type>&
     case TYPE_VARCHAR:
     case TYPE_HLL:
     case TYPE_DATETIME:
+    case TYPE_DATETIMEV2:
     case TYPE_TINYINT:
     case TYPE_SMALLINT:
     case TYPE_INT:
@@ -1080,10 +1091,13 @@ Status VOlapScanNode::change_fixed_value_range(ColumnValueRange<primitive_type>&
         break;
     }
     case TYPE_DATEV2: {
-        DateTimeValue date_value = *reinterpret_cast<DateTimeValue*>(value);
-        if (!date_value.check_loss_accuracy_cast_to_date()) {
-            doris::vectorized::DateV2Value date_v2;
-            date_v2.convert_dt_to_date_v2(&date_value);
+        DateV2Value<DateTimeV2ValueType> datetimev2_value =
+                *reinterpret_cast<DateV2Value<DateTimeV2ValueType>*>(value);
+        if (datetimev2_value.can_cast_to_date_without_loss_accuracy()) {
+            DateV2Value<DateV2ValueType> date_v2;
+            date_v2.set_date_uint32(
+                    binary_cast<DateV2Value<DateTimeV2ValueType>, uint64_t>(datetimev2_value) >>
+                    TIME_PART_LENGTH);
             if constexpr (primitive_type == PrimitiveType::TYPE_DATEV2) {
                 func(temp_range, &date_v2);
             } else {
@@ -1439,14 +1453,17 @@ Status VOlapScanNode::normalize_noneq_binary_predicate(SlotDescriptor* slot,
                     break;
                 }
                 case TYPE_DATEV2: {
-                    DateTimeValue date_value = *reinterpret_cast<DateTimeValue*>(value);
-                    if (date_value.check_loss_accuracy_cast_to_date()) {
+                    DateV2Value<DateTimeV2ValueType> datetimev2_value =
+                            *reinterpret_cast<DateV2Value<DateTimeV2ValueType>*>(value);
+                    doris::vectorized::DateV2Value<DateV2ValueType> date_v2;
+                    date_v2.set_date_uint32(binary_cast<DateV2Value<DateTimeV2ValueType>, uint64_t>(
+                                                    datetimev2_value) >>
+                                            TIME_PART_LENGTH);
+                    if (!datetimev2_value.can_cast_to_date_without_loss_accuracy()) {
                         if (pred->op() == TExprOpcode::LT || pred->op() == TExprOpcode::GE) {
-                            ++date_value;
+                            ++date_v2;
                         }
                     }
-                    doris::vectorized::DateV2Value date_v2;
-                    date_v2.convert_dt_to_date_v2(&date_value);
                     if constexpr (T == PrimitiveType::TYPE_DATEV2) {
                         range->add_range(to_olap_filter_type(pred->op(), child_idx), date_v2);
                         break;
@@ -1463,6 +1480,7 @@ Status VOlapScanNode::normalize_noneq_binary_predicate(SlotDescriptor* slot,
                 case TYPE_VARCHAR:
                 case TYPE_HLL:
                 case TYPE_DATETIME:
+                case TYPE_DATETIMEV2:
                 case TYPE_SMALLINT:
                 case TYPE_INT:
                 case TYPE_BIGINT:
diff --git a/be/src/vec/exec/vschema_scan_node.cpp b/be/src/vec/exec/vschema_scan_node.cpp
index 9e6a01a095..baa2b4bd94 100644
--- a/be/src/vec/exec/vschema_scan_node.cpp
+++ b/be/src/vec/exec/vschema_scan_node.cpp
@@ -417,6 +417,12 @@ Status VSchemaScanNode::write_slot_to_vectorized_column(void* slot, SlotDescript
         break;
     }
 
+    case TYPE_DATETIMEV2: {
+        uint32_t num = *reinterpret_cast<uint64_t*>(slot);
+        reinterpret_cast<vectorized::ColumnVector<vectorized::UInt64>*>(col_ptr)->insert_value(num);
+        break;
+    }
+
     case TYPE_DECIMALV2:
     case TYPE_DECIMAL128: {
         __int128 num = (reinterpret_cast<PackedInt128*>(slot))->value;
diff --git a/be/src/vec/exec/vset_operation_node.cpp b/be/src/vec/exec/vset_operation_node.cpp
index e8c0ab96ad..11a85ebd47 100644
--- a/be/src/vec/exec/vset_operation_node.cpp
+++ b/be/src/vec/exec/vset_operation_node.cpp
@@ -167,6 +167,7 @@ void VSetOperationNode::hash_table_init() {
         case TYPE_DATETIME:
         case TYPE_DATE:
         case TYPE_DECIMAL64:
+        case TYPE_DATETIMEV2:
             _hash_table_variants.emplace<I64HashTableContext>();
             break;
         case TYPE_LARGEINT:
diff --git a/be/src/vec/exprs/vliteral.cpp b/be/src/vec/exprs/vliteral.cpp
index 514f6df070..e20597391d 100644
--- a/be/src/vec/exprs/vliteral.cpp
+++ b/be/src/vec/exprs/vliteral.cpp
@@ -97,13 +97,18 @@ void VLiteral::init(const TExprNode& node) {
             break;
         }
         case TYPE_DATEV2: {
-            DateV2Value value;
+            DateV2Value<DateV2ValueType> value;
             value.from_date_str(node.date_literal.value.c_str(), node.date_literal.value.size());
-            field = value.to_date_uint32();
+            field = value.to_date_int_val();
             break;
         }
-        case TYPE_DATETIME:
         case TYPE_DATETIMEV2: {
+            DateV2Value<DateTimeV2ValueType> value;
+            value.from_date_str(node.date_literal.value.c_str(), node.date_literal.value.size());
+            field = value.to_date_int_val();
+            break;
+        }
+        case TYPE_DATETIME: {
             VecDateTimeValue value;
             value.from_date_str(node.date_literal.value.c_str(), node.date_literal.value.size());
             value.to_datetime();
diff --git a/be/src/vec/functions/array/function_array_element.h b/be/src/vec/functions/array/function_array_element.h
index 579d45f53f..53c41b19eb 100644
--- a/be/src/vec/functions/array/function_array_element.h
+++ b/be/src/vec/functions/array/function_array_element.h
@@ -207,6 +207,10 @@ private:
             res = _execute_number<ColumnDateTime>(offsets, *nested_column, src_null_map,
                                                   *arguments[1].column, nested_null_map,
                                                   dst_null_map);
+        } else if (nested_column->is_datetime_v2_type()) {
+            res = _execute_number<ColumnDateTimeV2>(offsets, *nested_column, src_null_map,
+                                                    *arguments[1].column, nested_null_map,
+                                                    dst_null_map);
         } else if (check_column<ColumnUInt8>(*nested_column)) {
             res = _execute_number<ColumnUInt8>(offsets, *nested_column, src_null_map,
                                                *arguments[1].column, nested_null_map, dst_null_map);
diff --git a/be/src/vec/functions/array/function_array_index.h b/be/src/vec/functions/array/function_array_index.h
index 34eb56d416..3bb1df5003 100644
--- a/be/src/vec/functions/array/function_array_index.h
+++ b/be/src/vec/functions/array/function_array_index.h
@@ -177,6 +177,9 @@ private:
         } else if (right_column.is_date_v2_type()) {
             return _execute_number<NestedColumnType, ColumnDateV2>(offsets, nested_null_map,
                                                                    nested_column, right_column);
+        } else if (right_column.is_datetime_v2_type()) {
+            return _execute_number<NestedColumnType, ColumnDateTimeV2>(offsets, nested_null_map,
+                                                                       nested_column, right_column);
         } else if (right_column.is_datetime_type()) {
             return _execute_number<NestedColumnType, ColumnDateTime>(offsets, nested_null_map,
                                                                      nested_column, right_column);
@@ -260,6 +263,9 @@ private:
             } else if (nested_column->is_datetime_type()) {
                 return_column = _execute_number_expanded<ColumnDateTime>(
                         offsets, nested_null_map, *nested_column, *right_column);
+            } else if (nested_column->is_datetime_v2_type()) {
+                return_column = _execute_number_expanded<ColumnDateTimeV2>(
+                        offsets, nested_null_map, *nested_column, *right_column);
             }
         }
 
diff --git a/be/src/vec/functions/date_time_transforms.h b/be/src/vec/functions/date_time_transforms.h
index 1e8d7962bb..5d96f73eea 100644
--- a/be/src/vec/functions/date_time_transforms.h
+++ b/be/src/vec/functions/date_time_transforms.h
@@ -33,25 +33,27 @@
 
 namespace doris::vectorized {
 
-#define TIME_FUNCTION_IMPL(CLASS, UNIT, FUNCTION)                            \
-    template <typename DateValueType, typename ArgType>                      \
-    struct CLASS {                                                           \
-        using ARG_TYPE = ArgType;                                            \
-        static constexpr auto name = #UNIT;                                  \
-                                                                             \
-        static inline auto execute(const ARG_TYPE& t, bool& is_null) {       \
-            const auto& date_time_value = (DateValueType&)(t);               \
-            is_null = !date_time_value.is_valid_date();                      \
-            return date_time_value.FUNCTION;                                 \
-        }                                                                    \
-                                                                             \
-        static DataTypes get_variadic_argument_types() {                     \
-            if constexpr (std::is_same_v<DateValueType, VecDateTimeValue>) { \
-                return {std::make_shared<DataTypeDateTime>()};               \
-            } else {                                                         \
-                return {std::make_shared<DataTypeDateV2>()};                 \
-            }                                                                \
-        }                                                                    \
+#define TIME_FUNCTION_IMPL(CLASS, UNIT, FUNCTION)                                               \
+    template <typename DateValueType, typename ArgType>                                         \
+    struct CLASS {                                                                              \
+        using ARG_TYPE = ArgType;                                                               \
+        static constexpr auto name = #UNIT;                                                     \
+                                                                                                \
+        static inline auto execute(const ARG_TYPE& t, bool& is_null) {                          \
+            const auto& date_time_value = (DateValueType&)(t);                                  \
+            is_null = !date_time_value.is_valid_date();                                         \
+            return date_time_value.FUNCTION;                                                    \
+        }                                                                                       \
+                                                                                                \
+        static DataTypes get_variadic_argument_types() {                                        \
+            if constexpr (std::is_same_v<DateValueType, VecDateTimeValue>) {                    \
+                return {std::make_shared<DataTypeDateTime>()};                                  \
+            } else if constexpr (std::is_same_v<DateValueType, DateV2Value<DateV2ValueType>>) { \
+                return {std::make_shared<DataTypeDateV2>()};                                    \
+            } else {                                                                            \
+                return {std::make_shared<DataTypeDateTimeV2>()};                                \
+            }                                                                                   \
+        }                                                                                       \
     }
 
 #define TO_TIME_FUNCTION(CLASS, UNIT) TIME_FUNCTION_IMPL(CLASS, UNIT, UNIT())
@@ -72,25 +74,27 @@ TIME_FUNCTION_IMPL(WeekDayImpl, weekday, weekday());
 // TODO: the method should be always not nullable
 TIME_FUNCTION_IMPL(ToDaysImpl, to_days, daynr());
 
-#define TIME_FUNCTION_ONE_ARG_IMPL(CLASS, UNIT, FUNCTION)                    \
-    template <typename DateValueType, typename ArgType>                      \
-    struct CLASS {                                                           \
-        using ARG_TYPE = ArgType;                                            \
-        static constexpr auto name = #UNIT;                                  \
-                                                                             \
-        static inline auto execute(const ARG_TYPE& t, bool& is_null) {       \
-            const auto& date_time_value = (DateValueType&)(t);               \
-            is_null = !date_time_value.is_valid_date();                      \
-            return date_time_value.FUNCTION;                                 \
-        }                                                                    \
-                                                                             \
-        static DataTypes get_variadic_argument_types() {                     \
-            if constexpr (std::is_same_v<DateValueType, VecDateTimeValue>) { \
-                return {std::make_shared<DataTypeDateTime>()};               \
-            } else {                                                         \
-                return {std::make_shared<DataTypeDateV2>()};                 \
-            }                                                                \
-        }                                                                    \
+#define TIME_FUNCTION_ONE_ARG_IMPL(CLASS, UNIT, FUNCTION)                                       \
+    template <typename DateValueType, typename ArgType>                                         \
+    struct CLASS {                                                                              \
+        using ARG_TYPE = ArgType;                                                               \
+        static constexpr auto name = #UNIT;                                                     \
+                                                                                                \
+        static inline auto execute(const ARG_TYPE& t, bool& is_null) {                          \
+            const auto& date_time_value = (DateValueType&)(t);                                  \
+            is_null = !date_time_value.is_valid_date();                                         \
+            return date_time_value.FUNCTION;                                                    \
+        }                                                                                       \
+                                                                                                \
+        static DataTypes get_variadic_argument_types() {                                        \
+            if constexpr (std::is_same_v<DateValueType, VecDateTimeValue>) {                    \
+                return {std::make_shared<DataTypeDateTime>()};                                  \
+            } else if constexpr (std::is_same_v<DateValueType, DateV2Value<DateV2ValueType>>) { \
+                return {std::make_shared<DataTypeDateV2>()};                                    \
+            } else {                                                                            \
+                return {std::make_shared<DataTypeDateTimeV2>()};                                \
+            }                                                                                   \
+        }                                                                                       \
     }
 
 TIME_FUNCTION_ONE_ARG_IMPL(ToWeekOneArgImpl, week, week(mysql_week_mode(0)));
@@ -104,17 +108,23 @@ struct ToDateImpl {
     static inline auto execute(const ArgType& t, bool& is_null) {
         auto dt = binary_cast<ArgType, DateValueType>(t);
         is_null = !dt.is_valid_date();
-        if constexpr (!std::is_same_v<DateValueType, doris::vectorized::DateV2Value>) {
+        if constexpr (std::is_same_v<DateValueType, DateV2Value<DateV2ValueType>>) {
+            return binary_cast<DateValueType, ArgType>(dt);
+        } else if constexpr (std::is_same_v<DateValueType, VecDateTimeValue>) {
             dt.cast_to_date();
+            return binary_cast<DateValueType, ArgType>(dt);
+        } else {
+            return (UInt32)(binary_cast<DateValueType, ArgType>(dt) >> TIME_PART_LENGTH);
         }
-        return binary_cast<DateValueType, ArgType>(dt);
     }
 
     static DataTypes get_variadic_argument_types() {
         if constexpr (std::is_same_v<DateValueType, VecDateTimeValue>) {
             return {std::make_shared<DataTypeDateTime>()};
-        } else {
+        } else if constexpr (std::is_same_v<DateValueType, DateV2Value<DateV2ValueType>>) {
             return {std::make_shared<DataTypeDateV2>()};
+        } else {
+            return {std::make_shared<DataTypeDateTimeV2>()};
         }
     }
 };
@@ -159,8 +169,10 @@ struct DayNameImpl {
     static DataTypes get_variadic_argument_types() {
         if constexpr (std::is_same_v<DateValueType, VecDateTimeValue>) {
             return {std::make_shared<DataTypeDateTime>()};
-        } else {
+        } else if constexpr (std::is_same_v<DateValueType, DateV2Value<DateV2ValueType>>) {
             return {std::make_shared<DataTypeDateV2>()};
+        } else {
+            return {std::make_shared<DataTypeDateTimeV2>()};
         }
     }
 };
@@ -190,8 +202,10 @@ struct MonthNameImpl {
     static DataTypes get_variadic_argument_types() {
         if constexpr (std::is_same_v<DateValueType, VecDateTimeValue>) {
             return {std::make_shared<DataTypeDateTime>()};
-        } else {
+        } else if constexpr (std::is_same_v<DateValueType, DateV2Value<DateV2ValueType>>) {
             return {std::make_shared<DataTypeDateV2>()};
+        } else {
+            return {std::make_shared<DataTypeDateTimeV2>()};
         }
     }
 };
@@ -230,12 +244,18 @@ struct DateFormatImpl {
                             std::make_shared<vectorized::DataTypeDateTime>()),
                     std::dynamic_pointer_cast<const IDataType>(
                             std::make_shared<vectorized::DataTypeString>())};
-        } else {
+        } else if constexpr (std::is_same_v<DateType, DateV2Value<DateV2ValueType>>) {
             return std::vector<DataTypePtr> {
                     std::dynamic_pointer_cast<const IDataType>(
                             std::make_shared<vectorized::DataTypeDateV2>()),
                     std::dynamic_pointer_cast<const IDataType>(
                             std::make_shared<vectorized::DataTypeString>())};
+        } else {
+            return std::vector<DataTypePtr> {
+                    std::dynamic_pointer_cast<const IDataType>(
+                            std::make_shared<vectorized::DataTypeDateTimeV2>()),
+                    std::dynamic_pointer_cast<const IDataType>(
+                            std::make_shared<vectorized::DataTypeString>())};
         }
     }
 };
@@ -301,7 +321,24 @@ struct TransformerToStringOneArgument {
         size_t offset = 0;
         for (int i = 0; i < len; ++i) {
             const auto& t = ts[i];
-            const auto& date_time_value = reinterpret_cast<const DateV2Value&>(t);
+            const auto& date_time_value = reinterpret_cast<const DateV2Value<DateV2ValueType>&>(t);
+            res_offsets[i] = Transform::execute(date_time_value, res_data, offset,
+                                                reinterpret_cast<bool&>(null_map[i]));
+        }
+    }
+
+    static void vector(const PaddedPODArray<UInt64>& ts, ColumnString::Chars& res_data,
+                       ColumnString::Offsets& res_offsets, NullMap& null_map) {
+        const auto len = ts.size();
+        res_data.resize(len * Transform::max_size);
+        res_offsets.resize(len);
+        null_map.resize_fill(len, false);
+
+        size_t offset = 0;
+        for (int i = 0; i < len; ++i) {
+            const auto& t = ts[i];
+            const auto& date_time_value =
+                    reinterpret_cast<const DateV2Value<DateTimeV2ValueType>&>(t);
             res_offsets[i] = Transform::execute(date_time_value, res_data, offset,
                                                 reinterpret_cast<bool&>(null_map[i]));
         }
diff --git a/be/src/vec/functions/function.h b/be/src/vec/functions/function.h
index 0e90f8d681..24cb18e984 100644
--- a/be/src/vec/functions/function.h
+++ b/be/src/vec/functions/function.h
@@ -293,13 +293,15 @@ public:
                                             ? ((DataTypeNullable*)get_return_type(arguments).get())
                                                       ->get_nested_type()
                                             : get_return_type(arguments))) ||
-               (is_date_v2(return_type->is_nullable()
-                                   ? ((DataTypeNullable*)return_type.get())->get_nested_type()
-                                   : return_type) &&
-                is_date_v2(get_return_type(arguments)->is_nullable()
-                                   ? ((DataTypeNullable*)get_return_type(arguments).get())
-                                             ->get_nested_type()
-                                   : get_return_type(arguments))) ||
+               (is_date_v2_or_datetime_v2(
+                        return_type->is_nullable()
+                                ? ((DataTypeNullable*)return_type.get())->get_nested_type()
+                                : return_type) &&
+                is_date_v2_or_datetime_v2(
+                        get_return_type(arguments)->is_nullable()
+                                ? ((DataTypeNullable*)get_return_type(arguments).get())
+                                          ->get_nested_type()
+                                : get_return_type(arguments))) ||
                (is_decimal(return_type->is_nullable()
                                    ? ((DataTypeNullable*)return_type.get())->get_nested_type()
                                    : return_type) &&
diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h
index fed821f09f..be83d31b44 100644
--- a/be/src/vec/functions/function_cast.h
+++ b/be/src/vec/functions/function_cast.h
@@ -84,7 +84,7 @@ struct ConvertImpl {
 
         if constexpr (IsDataTypeDecimal<FromDataType> || IsDataTypeDecimal<ToDataType>) {
             if constexpr (!(IsDataTypeDecimalOrNumber<FromDataType> || IsTimeType<FromDataType> ||
-                            IsDateV2Type<FromDataType>) ||
+                            IsTimeV2Type<FromDataType>) ||
                           !IsDataTypeDecimalOrNumber<ToDataType>)
                 return Status::RuntimeError("Illegal column {} of first argument of function {}",
                                             named_from.column->get_name(), Name::name);
@@ -124,7 +124,16 @@ struct ConvertImpl {
                     } else if constexpr (IsDateV2Type<FromDataType> &&
                                          IsDataTypeDecimal<ToDataType>) {
                         vec_to[i] = convert_to_decimal<DataTypeUInt32, ToDataType>(
-                                reinterpret_cast<const DateV2Value&>(vec_from[i]).to_date_uint32(),
+                                reinterpret_cast<const DateV2Value<DateV2ValueType>&>(vec_from[i])
+                                        .to_date_int_val(),
+                                vec_to.get_scale());
+                    } else if constexpr (IsDateTimeV2Type<FromDataType> &&
+                                         IsDataTypeDecimal<ToDataType>) {
+                        // TODO: should we consider the scale of datetimev2?
+                        vec_to[i] = convert_to_decimal<DataTypeUInt64, ToDataType>(
+                                reinterpret_cast<const DateV2Value<DateTimeV2ValueType>&>(
+                                        vec_from[i])
+                                        .to_date_int_val(),
                                 vec_to.get_scale());
                     }
                 } else if constexpr (IsTimeType<FromDataType>) {
@@ -136,22 +145,50 @@ struct ConvertImpl {
                             DataTypeDate::cast_to_date(vec_to[i]);
                         }
                     } else if constexpr (IsDateV2Type<ToDataType>) {
-                        auto date_v2 = binary_cast<UInt32, DateV2Value>(vec_to[i]);
+                        auto date_v2 = binary_cast<UInt32, DateV2Value<DateV2ValueType>>(vec_to[i]);
+                        date_v2.from_date_int64(vec_from[i]);
+                    } else if constexpr (IsDateTimeV2Type<ToDataType>) {
+                        auto date_v2 =
+                                binary_cast<UInt64, DateV2Value<DateTimeV2ValueType>>(vec_to[i]);
                         date_v2.from_date_int64(vec_from[i]);
                     } else {
                         vec_to[i] =
                                 reinterpret_cast<const VecDateTimeValue&>(vec_from[i]).to_int64();
                     }
-                } else if constexpr (IsDateV2Type<FromDataType>) {
-                    if constexpr (IsTimeType<ToDataType> || IsDateTimeV2Type<ToDataType>) {
-                        if constexpr (IsDateTimeType<ToDataType> || IsDateTimeV2Type<ToDataType>) {
-                            DataTypeDateV2::cast_to_date_time(vec_from[i], vec_to[i]);
+                } else if constexpr (IsTimeV2Type<FromDataType>) {
+                    if constexpr (IsTimeV2Type<ToDataType>) {
+                        if constexpr (IsDateTimeV2Type<ToDataType> && IsDateV2Type<FromDataType>) {
+                            DataTypeDateV2::cast_to_date_time_v2(vec_from[i], vec_to[i]);
+                        } else if constexpr (IsDateTimeV2Type<FromDataType> &&
+                                             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];
+                        }
+                    } else if constexpr (IsTimeType<ToDataType>) {
+                        if constexpr (IsDateTimeType<ToDataType> && IsDateV2Type<FromDataType>) {
+                            DataTypeDateV2::cast_to_date_time(vec_from[i], vec_to[i]);
+                        } else if constexpr (IsDateTimeV2Type<ToDataType> &&
+                                             IsDateV2Type<FromDataType>) {
                             DataTypeDateV2::cast_to_date(vec_from[i], vec_to[i]);
+                        } else if constexpr (IsDateTimeType<ToDataType> &&
+                                             IsDateTimeV2Type<FromDataType>) {
+                            DataTypeDateTimeV2::cast_to_date_time(vec_from[i], vec_to[i]);
+                        } else if constexpr (IsDateTimeV2Type<ToDataType> &&
+                                             IsDateTimeV2Type<FromDataType>) {
+                            DataTypeDateTimeV2::cast_to_date(vec_from[i], vec_to[i]);
                         }
                     } else {
-                        vec_to[i] =
-                                reinterpret_cast<const DateV2Value&>(vec_from[i]).to_date_uint32();
+                        if constexpr (IsDateTimeV2Type<ToDataType>) {
+                            vec_to[i] = reinterpret_cast<const DateV2Value<DateTimeV2ValueType>&>(
+                                                vec_from[i])
+                                                .to_date_int_val();
+                        } else {
+                            vec_to[i] = reinterpret_cast<const DateV2Value<DateV2ValueType>&>(
+                                                vec_from[i])
+                                                .to_date_int_val();
+                        }
                     }
                 } else {
                     vec_to[i] = static_cast<ToFieldType>(vec_from[i]);
@@ -200,8 +237,11 @@ struct ConvertImplToTimeType {
                 std::conditional_t<IsDecimalNumber<FromFieldType>, ColumnDecimal<FromFieldType>,
                                    ColumnVector<FromFieldType>>;
 
-        using DateValueType =
-                std::conditional_t<IsDateV2Type<ToDataType>, DateV2Value, VecDateTimeValue>;
+        using DateValueType = std::conditional_t<
+                IsTimeV2Type<ToDataType>,
+                std::conditional_t<IsDateV2Type<ToDataType>, DateV2Value<DateV2ValueType>,
+                                   DateV2Value<DateTimeV2ValueType>>,
+                VecDateTimeValue>;
         using ColVecTo = ColumnVector<ToFieldType>;
 
         if (const ColVecFrom* col_from =
@@ -221,11 +261,10 @@ struct ConvertImplToTimeType {
             for (size_t i = 0; i < size; ++i) {
                 auto& date_value = reinterpret_cast<DateValueType&>(vec_to[i]);
                 if constexpr (IsDecimalNumber<FromFieldType>) {
+                    // TODO: should we consider the scale of datetimev2?
                     vec_null_map_to[i] = !date_value.from_date_int64(
                             convert_from_decimal<FromDataType, DataTypeInt64>(
                                     vec_from[i], vec_from.get_scale()));
-                } else if constexpr (IsDateV2Type<FromFieldType>) {
-                    DataTypeDateV2::cast_to_date_time(vec_from[i], vec_to[i]);
                 } else {
                     vec_null_map_to[i] = !date_value.from_date_int64(vec_from[i]);
                 }
@@ -380,6 +419,10 @@ bool try_parse_impl(typename DataType::FieldType& x, ReadBuffer& rb, const DateL
         return try_read_date_v2_text(x, rb);
     }
 
+    if constexpr (IsDateTimeV2Type<DataType>) {
+        return try_read_datetime_v2_text(x, rb);
+    }
+
     if constexpr (std::is_floating_point_v<typename DataType::FieldType>) {
         return try_read_float_text(x, rb);
     }
@@ -668,8 +711,10 @@ using FunctionToDecimal64 =
 using FunctionToDecimal128 =
         FunctionConvert<DataTypeDecimal<Decimal128>, NameToDecimal128, UnknownMonotonicity>;
 using FunctionToDate = FunctionConvert<DataTypeDate, NameToDate, UnknownMonotonicity>;
-using FunctionToDateV2 = FunctionConvert<DataTypeDateV2, NameToDate, UnknownMonotonicity>;
 using FunctionToDateTime = FunctionConvert<DataTypeDateTime, NameToDateTime, UnknownMonotonicity>;
+using FunctionToDateV2 = FunctionConvert<DataTypeDateV2, NameToDate, UnknownMonotonicity>;
+using FunctionToDateTimeV2 =
+        FunctionConvert<DataTypeDateTimeV2, NameToDateTime, UnknownMonotonicity>;
 
 template <typename DataType>
 struct FunctionTo;
@@ -734,12 +779,16 @@ struct FunctionTo<DataTypeDate> {
     using Type = FunctionToDate;
 };
 template <>
+struct FunctionTo<DataTypeDateTime> {
+    using Type = FunctionToDateTime;
+};
+template <>
 struct FunctionTo<DataTypeDateV2> {
     using Type = FunctionToDateV2;
 };
 template <>
-struct FunctionTo<DataTypeDateTime> {
-    using Type = FunctionToDateTime;
+struct FunctionTo<DataTypeDateTimeV2> {
+    using Type = FunctionToDateTimeV2;
 };
 
 class PreparedFunctionCast : public PreparedFunctionImpl {
@@ -1013,10 +1062,11 @@ private:
             /// that will not throw an exception but return NULL in case of malformed input.
             function = FunctionConvertFromString<DataType, NameCast>::create();
         } else if (requested_result_is_nullable &&
-                   (IsTimeType<DataType> || IsDateV2Type<DataType>)&&!(
+                   (IsTimeType<DataType> || IsTimeV2Type<DataType>)&&!(
                            check_and_get_data_type<DataTypeDateTime>(from_type.get()) ||
                            check_and_get_data_type<DataTypeDate>(from_type.get()) ||
-                           check_and_get_data_type<DataTypeDateV2>(from_type.get()))) {
+                           check_and_get_data_type<DataTypeDateV2>(from_type.get()) ||
+                           check_and_get_data_type<DataTypeDateTimeV2>(from_type.get()))) {
             function = FunctionConvertToTimeType<DataType, NameCast>::create();
         } else {
             function = FunctionTo<DataType>::Type::create();
@@ -1054,8 +1104,8 @@ private:
 
         WhichDataType which(type_index);
         bool ok = which.is_int() || which.is_native_uint() || which.is_decimal() ||
-                  which.is_float() || which.is_date_or_datetime() || which.is_date_v2() ||
-                  which.is_string_or_fixed_string();
+                  which.is_float() || which.is_date_or_datetime() ||
+                  which.is_date_v2_or_datetime_v2() || which.is_string_or_fixed_string();
         if (!ok) {
             LOG(FATAL) << fmt::format(
                     "Conversion from {} to {} to_type->get_name() is not supported",
@@ -1301,7 +1351,8 @@ private:
                           std::is_same_v<ToDataType, DataTypeFloat64> ||
                           std::is_same_v<ToDataType, DataTypeDate> ||
                           std::is_same_v<ToDataType, DataTypeDateTime> ||
-                          std::is_same_v<ToDataType, DataTypeDateV2>) {
+                          std::is_same_v<ToDataType, DataTypeDateV2> ||
+                          std::is_same_v<ToDataType, DataTypeDateTimeV2>) {
                 ret = create_wrapper(from_type, check_and_get_data_type<ToDataType>(to_type.get()),
                                      requested_result_is_nullable);
                 return true;
diff --git a/be/src/vec/functions/function_date_or_datetime_computation.cpp b/be/src/vec/functions/function_date_or_datetime_computation.cpp
index 6a923d7d5e..11f16b49dc 100644
--- a/be/src/vec/functions/function_date_or_datetime_computation.cpp
+++ b/be/src/vec/functions/function_date_or_datetime_computation.cpp
@@ -105,33 +105,89 @@ using FunctionMinutesDiff = FunctionDateOrDateTimeComputation<MintueSDiffImpl<
         VecDateTimeValue, VecDateTimeValue, DataTypeDateTime, DataTypeDateTime, Int64, Int64>>;
 using FunctionSecondsDiff = FunctionDateOrDateTimeComputation<SecondsDiffImpl<
         VecDateTimeValue, VecDateTimeValue, DataTypeDateTime, DataTypeDateTime, Int64, Int64>>;
-using FunctionDateDiffV2 = FunctionDateOrDateTimeComputation<
-        DateDiffImpl<DateV2Value, DateV2Value, DataTypeDateV2, DataTypeDateV2, UInt32, UInt32>>;
-using FunctionTimeDiffV2 = FunctionDateOrDateTimeComputation<
-        TimeDiffImpl<DateV2Value, DateV2Value, DataTypeDateV2, DataTypeDateV2, UInt32, UInt32>>;
-using FunctionYearsDiffV2 = FunctionDateOrDateTimeComputation<
-        YearsDiffImpl<DateV2Value, DateV2Value, DataTypeDateV2, DataTypeDateV2, UInt32, UInt32>>;
-using FunctionMonthsDiffV2 = FunctionDateOrDateTimeComputation<
-        MonthsDiffImpl<DateV2Value, DateV2Value, DataTypeDateV2, DataTypeDateV2, UInt32, UInt32>>;
-using FunctionDaysDiffV2 = FunctionDateOrDateTimeComputation<
-        DaysDiffImpl<DateV2Value, DateV2Value, DataTypeDateV2, DataTypeDateV2, UInt32, UInt32>>;
-using FunctionWeeksDiffV2 = FunctionDateOrDateTimeComputation<
-        WeeksDiffImpl<DateV2Value, DateV2Value, DataTypeDateV2, DataTypeDateV2, UInt32, UInt32>>;
-using FunctionHoursDiffV2 = FunctionDateOrDateTimeComputation<
-        HoursDiffImpl<DateV2Value, DateV2Value, DataTypeDateV2, DataTypeDateV2, UInt32, UInt32>>;
-using FunctionMinutesDiffV2 = FunctionDateOrDateTimeComputation<
-        MintueSDiffImpl<DateV2Value, DateV2Value, DataTypeDateV2, DataTypeDateV2, UInt32, UInt32>>;
-using FunctionSecondsDiffV2 = FunctionDateOrDateTimeComputation<
-        SecondsDiffImpl<DateV2Value, DateV2Value, DataTypeDateV2, DataTypeDateV2, UInt32, UInt32>>;
 
 using FunctionToYearWeekTwoArgs = FunctionDateOrDateTimeComputation<
         ToYearWeekTwoArgsImpl<VecDateTimeValue, DataTypeDateTime, Int64>>;
 using FunctionToYearWeekTwoArgsV2 = FunctionDateOrDateTimeComputation<
-        ToYearWeekTwoArgsImpl<DateV2Value, DataTypeDateV2, UInt32>>;
+        ToYearWeekTwoArgsImpl<DateV2Value<DateV2ValueType>, DataTypeDateV2, UInt32>>;
 using FunctionToWeekTwoArgs = FunctionDateOrDateTimeComputation<
         ToWeekTwoArgsImpl<VecDateTimeValue, DataTypeDateTime, Int64>>;
-using FunctionToWeekTwoArgsV2 =
-        FunctionDateOrDateTimeComputation<ToWeekTwoArgsImpl<DateV2Value, DataTypeDateV2, UInt32>>;
+using FunctionToWeekTwoArgsV2 = FunctionDateOrDateTimeComputation<
+        ToWeekTwoArgsImpl<DateV2Value<DateV2ValueType>, DataTypeDateV2, UInt32>>;
+
+using FunctionDatetimeV2AddSeconds = FunctionDateOrDateTimeComputation<
+        AddSecondsImpl<DataTypeDateTimeV2, UInt64, DataTypeDateTimeV2>>;
+using FunctionDatetimeV2AddMinutes = FunctionDateOrDateTimeComputation<
+        AddMinutesImpl<DataTypeDateTimeV2, UInt64, DataTypeDateTimeV2>>;
+using FunctionDatetimeV2AddHours = FunctionDateOrDateTimeComputation<
+        AddHoursImpl<DataTypeDateTimeV2, UInt64, DataTypeDateTimeV2>>;
+using FunctionDatetimeV2AddDays = FunctionDateOrDateTimeComputation<
+        AddDaysImpl<DataTypeDateTimeV2, UInt64, DataTypeDateTimeV2>>;
+using FunctionDatetimeV2AddWeeks = FunctionDateOrDateTimeComputation<
+        AddWeeksImpl<DataTypeDateTimeV2, UInt64, DataTypeDateTimeV2>>;
+using FunctionDatetimeV2AddMonths = FunctionDateOrDateTimeComputation<
+        AddMonthsImpl<DataTypeDateTimeV2, UInt64, DataTypeDateTimeV2>>;
+
+using FunctionDatetimeV2AddQuarters = FunctionDateOrDateTimeComputation<
+        AddQuartersImpl<DataTypeDateTimeV2, UInt64, DataTypeDateTimeV2>>;
+using FunctionDatetimeV2AddYears = FunctionDateOrDateTimeComputation<
+        AddYearsImpl<DataTypeDateTimeV2, UInt64, DataTypeDateTimeV2>>;
+
+using FunctionDatetimeV2SubSeconds = FunctionDateOrDateTimeComputation<
+        SubtractSecondsImpl<DataTypeDateTimeV2, UInt64, DataTypeDateTimeV2>>;
+using FunctionDatetimeV2SubMinutes = FunctionDateOrDateTimeComputation<
+        SubtractMinutesImpl<DataTypeDateTimeV2, UInt64, DataTypeDateTimeV2>>;
+using FunctionDatetimeV2SubHours = FunctionDateOrDateTimeComputation<
+        SubtractHoursImpl<DataTypeDateTimeV2, UInt64, DataTypeDateTimeV2>>;
+using FunctionDatetimeV2SubDays = FunctionDateOrDateTimeComputation<
+        SubtractDaysImpl<DataTypeDateTimeV2, UInt64, DataTypeDateTimeV2>>;
+using FunctionDatetimeV2SubWeeks = FunctionDateOrDateTimeComputation<
+        SubtractWeeksImpl<DataTypeDateTimeV2, UInt64, DataTypeDateTimeV2>>;
+using FunctionDatetimeV2SubMonths = FunctionDateOrDateTimeComputation<
+        SubtractMonthsImpl<DataTypeDateTimeV2, UInt64, DataTypeDateTimeV2>>;
+using FunctionDatetimeV2SubQuarters = FunctionDateOrDateTimeComputation<
+        SubtractQuartersImpl<DataTypeDateTimeV2, UInt64, DataTypeDateTimeV2>>;
+using FunctionDatetimeV2SubYears = FunctionDateOrDateTimeComputation<
+        SubtractYearsImpl<DataTypeDateTimeV2, UInt64, DataTypeDateTimeV2>>;
+
+#define FUNCTION_DATEV2_WITH_TWO_ARGS(NAME, IMPL, TYPE1, TYPE2, ARG1, ARG2, DATE_VALUE1, \
+                                      DATE_VALUE2)                                       \
+    using NAME##_##TYPE1##_##TYPE2 = FunctionDateOrDateTimeComputation<                  \
+            IMPL<DATE_VALUE1, DATE_VALUE2, TYPE1, TYPE2, ARG1, ARG2>>;
+
+#define ALL_FUNCTION_DATEV2_WITH_TWO_ARGS(NAME, IMPL)                                              \
+    FUNCTION_DATEV2_WITH_TWO_ARGS(NAME, IMPL, DataTypeDateTimeV2, DataTypeDateTimeV2, UInt64,      \
+                                  UInt64, DateV2Value<DateTimeV2ValueType>,                        \
+                                  DateV2Value<DateTimeV2ValueType>)                                \
+    FUNCTION_DATEV2_WITH_TWO_ARGS(NAME, IMPL, DataTypeDateTimeV2, DataTypeDateV2, UInt64, UInt32,  \
+                                  DateV2Value<DateTimeV2ValueType>, DateV2Value<DateV2ValueType>)  \
+    FUNCTION_DATEV2_WITH_TWO_ARGS(NAME, IMPL, DataTypeDateV2, DataTypeDateTimeV2, UInt32, UInt64,  \
+                                  DateV2Value<DateV2ValueType>, DateV2Value<DateTimeV2ValueType>)  \
+    FUNCTION_DATEV2_WITH_TWO_ARGS(NAME, IMPL, DataTypeDateTimeV2, DataTypeDateTime, UInt64, Int64, \
+                                  DateV2Value<DateTimeV2ValueType>, VecDateTimeValue)              \
+    FUNCTION_DATEV2_WITH_TWO_ARGS(NAME, IMPL, DataTypeDateTime, DataTypeDateTimeV2, Int64, UInt64, \
+                                  VecDateTimeValue, DateV2Value<DateTimeV2ValueType>)              \
+    FUNCTION_DATEV2_WITH_TWO_ARGS(NAME, IMPL, DataTypeDateTime, DataTypeDateV2, Int64, UInt32,     \
+                                  VecDateTimeValue, DateV2Value<DateV2ValueType>)                  \
+    FUNCTION_DATEV2_WITH_TWO_ARGS(NAME, IMPL, DataTypeDateV2, DataTypeDateTime, UInt32, Int64,     \
+                                  DateV2Value<DateV2ValueType>, VecDateTimeValue)                  \
+    FUNCTION_DATEV2_WITH_TWO_ARGS(NAME, IMPL, DataTypeDateV2, DataTypeDateV2, UInt32, UInt32,      \
+                                  DateV2Value<DateV2ValueType>, DateV2Value<DateV2ValueType>)
+
+ALL_FUNCTION_DATEV2_WITH_TWO_ARGS(FunctionDatetimeV2DateDiff, DateDiffImpl)
+ALL_FUNCTION_DATEV2_WITH_TWO_ARGS(FunctionDatetimeV2TimeDiff, TimeDiffImpl)
+ALL_FUNCTION_DATEV2_WITH_TWO_ARGS(FunctionDatetimeV2YearsDiff, YearsDiffImpl)
+ALL_FUNCTION_DATEV2_WITH_TWO_ARGS(FunctionDatetimeV2MonthsDiff, MonthsDiffImpl)
+ALL_FUNCTION_DATEV2_WITH_TWO_ARGS(FunctionDatetimeV2WeeksDiff, WeeksDiffImpl)
+ALL_FUNCTION_DATEV2_WITH_TWO_ARGS(FunctionDatetimeV2HoursDiff, HoursDiffImpl)
+ALL_FUNCTION_DATEV2_WITH_TWO_ARGS(FunctionDatetimeV2MinutesDiff, MintueSDiffImpl)
+ALL_FUNCTION_DATEV2_WITH_TWO_ARGS(FunctionDatetimeV2SecondsDiff, SecondsDiffImpl)
+ALL_FUNCTION_DATEV2_WITH_TWO_ARGS(FunctionDatetimeV2DaysDiff, DaysDiffImpl)
+
+using FunctionDatetimeV2ToYearWeekTwoArgs = FunctionDateOrDateTimeComputation<
+        ToYearWeekTwoArgsImpl<DateV2Value<DateTimeV2ValueType>, DataTypeDateTimeV2, UInt64>>;
+using FunctionDatetimeV2ToWeekTwoArgs = FunctionDateOrDateTimeComputation<
+        ToWeekTwoArgsImpl<DateV2Value<DateTimeV2ValueType>, DataTypeDateTimeV2, UInt64>>;
 
 struct NowFunctionName {
     static constexpr auto name = "now";
@@ -202,6 +258,15 @@ void register_function_date_time_computation(SimpleFunctionFactory& factory) {
     factory.register_function<FunctionAddYearsV2>();
     factory.register_function<FunctionAddQuartersV2>();
 
+    factory.register_function<FunctionDatetimeV2AddSeconds>();
+    factory.register_function<FunctionDatetimeV2AddMinutes>();
+    factory.register_function<FunctionDatetimeV2AddHours>();
+    factory.register_function<FunctionDatetimeV2AddDays>();
+    factory.register_function<FunctionDatetimeV2AddWeeks>();
+    factory.register_function<FunctionDatetimeV2AddMonths>();
+    factory.register_function<FunctionDatetimeV2AddYears>();
+    factory.register_function<FunctionDatetimeV2AddQuarters>();
+
     factory.register_function<FunctionSubSeconds>();
     factory.register_function<FunctionSubMinutes>();
     factory.register_function<FunctionSubHours>();
@@ -221,6 +286,15 @@ void register_function_date_time_computation(SimpleFunctionFactory& factory) {
     factory.register_function<FunctionSubQuartersV2>();
     factory.register_function<FunctionSubWeeksV2>();
 
+    factory.register_function<FunctionDatetimeV2SubSeconds>();
+    factory.register_function<FunctionDatetimeV2SubMinutes>();
+    factory.register_function<FunctionDatetimeV2SubHours>();
+    factory.register_function<FunctionDatetimeV2SubDays>();
+    factory.register_function<FunctionDatetimeV2SubMonths>();
+    factory.register_function<FunctionDatetimeV2SubYears>();
+    factory.register_function<FunctionDatetimeV2SubQuarters>();
+    factory.register_function<FunctionDatetimeV2SubWeeks>();
+
     factory.register_function<FunctionDateDiff>();
     factory.register_function<FunctionTimeDiff>();
     factory.register_function<FunctionYearsDiff>();
@@ -230,59 +304,36 @@ void register_function_date_time_computation(SimpleFunctionFactory& factory) {
     factory.register_function<FunctionHoursDiff>();
     factory.register_function<FunctionMinutesDiff>();
     factory.register_function<FunctionSecondsDiff>();
-    factory.register_function<FunctionDateDiffV2>();
-    factory.register_function<FunctionTimeDiffV2>();
-    factory.register_function<FunctionYearsDiffV2>();
-    factory.register_function<FunctionMonthsDiffV2>();
-    factory.register_function<FunctionWeeksDiffV2>();
-    factory.register_function<FunctionDaysDiffV2>();
-    factory.register_function<FunctionHoursDiffV2>();
-    factory.register_function<FunctionMinutesDiffV2>();
-    factory.register_function<FunctionSecondsDiffV2>();
-
-    // Functions below make dateV2 be compatible with V1
-    factory.register_function<FunctionDateOrDateTimeComputation<DateDiffImpl<
-            VecDateTimeValue, DateV2Value, DataTypeDateTime, DataTypeDateV2, Int64, UInt32>>>();
-    factory.register_function<FunctionDateOrDateTimeComputation<TimeDiffImpl<
-            VecDateTimeValue, DateV2Value, DataTypeDateTime, DataTypeDateV2, Int64, UInt32>>>();
-    factory.register_function<FunctionDateOrDateTimeComputation<YearsDiffImpl<
-            VecDateTimeValue, DateV2Value, DataTypeDateTime, DataTypeDateV2, Int64, UInt32>>>();
-    factory.register_function<FunctionDateOrDateTimeComputation<MonthsDiffImpl<
-            VecDateTimeValue, DateV2Value, DataTypeDateTime, DataTypeDateV2, Int64, UInt32>>>();
-    factory.register_function<FunctionDateOrDateTimeComputation<WeeksDiffImpl<
-            VecDateTimeValue, DateV2Value, DataTypeDateTime, DataTypeDateV2, Int64, UInt32>>>();
-    factory.register_function<FunctionDateOrDateTimeComputation<DaysDiffImpl<
-            VecDateTimeValue, DateV2Value, DataTypeDateTime, DataTypeDateV2, Int64, UInt32>>>();
-    factory.register_function<FunctionDateOrDateTimeComputation<HoursDiffImpl<
-            VecDateTimeValue, DateV2Value, DataTypeDateTime, DataTypeDateV2, Int64, UInt32>>>();
-    factory.register_function<FunctionDateOrDateTimeComputation<MintueSDiffImpl<
-            VecDateTimeValue, DateV2Value, DataTypeDateTime, DataTypeDateV2, Int64, UInt32>>>();
-    factory.register_function<FunctionDateOrDateTimeComputation<SecondsDiffImpl<
-            VecDateTimeValue, DateV2Value, DataTypeDateTime, DataTypeDateV2, Int64, UInt32>>>();
-
-    factory.register_function<FunctionDateOrDateTimeComputation<DateDiffImpl<
-            DateV2Value, VecDateTimeValue, DataTypeDateV2, DataTypeDateTime, UInt32, Int64>>>();
-    factory.register_function<FunctionDateOrDateTimeComputation<TimeDiffImpl<
-            DateV2Value, VecDateTimeValue, DataTypeDateV2, DataTypeDateTime, UInt32, Int64>>>();
-    factory.register_function<FunctionDateOrDateTimeComputation<YearsDiffImpl<
-            DateV2Value, VecDateTimeValue, DataTypeDateV2, DataTypeDateTime, UInt32, Int64>>>();
-    factory.register_function<FunctionDateOrDateTimeComputation<MonthsDiffImpl<
-            DateV2Value, VecDateTimeValue, DataTypeDateV2, DataTypeDateTime, UInt32, Int64>>>();
-    factory.register_function<FunctionDateOrDateTimeComputation<WeeksDiffImpl<
-            DateV2Value, VecDateTimeValue, DataTypeDateV2, DataTypeDateTime, UInt32, Int64>>>();
-    factory.register_function<FunctionDateOrDateTimeComputation<DaysDiffImpl<
-            DateV2Value, VecDateTimeValue, DataTypeDateV2, DataTypeDateTime, UInt32, Int64>>>();
-    factory.register_function<FunctionDateOrDateTimeComputation<HoursDiffImpl<
-            DateV2Value, VecDateTimeValue, DataTypeDateV2, DataTypeDateTime, UInt32, Int64>>>();
-    factory.register_function<FunctionDateOrDateTimeComputation<MintueSDiffImpl<
-            DateV2Value, VecDateTimeValue, DataTypeDateV2, DataTypeDateTime, UInt32, Int64>>>();
-    factory.register_function<FunctionDateOrDateTimeComputation<SecondsDiffImpl<
-            DateV2Value, VecDateTimeValue, DataTypeDateV2, DataTypeDateTime, UInt32, Int64>>>();
+
+#define REGISTER_DATEV2_FUNCTIONS_WITH_TWO_ARGS(NAME, TYPE1, TYPE2) \
+    factory.register_function<NAME##_##TYPE1##_##TYPE2>();
+
+#define REGISTER_ALL_DATEV2_FUNCTIONS_WITH_TWO_ARGS(NAME)                                 \
+    REGISTER_DATEV2_FUNCTIONS_WITH_TWO_ARGS(NAME, DataTypeDateTimeV2, DataTypeDateTimeV2) \
+    REGISTER_DATEV2_FUNCTIONS_WITH_TWO_ARGS(NAME, DataTypeDateTimeV2, DataTypeDateV2)     \
+    REGISTER_DATEV2_FUNCTIONS_WITH_TWO_ARGS(NAME, DataTypeDateV2, DataTypeDateTimeV2)     \
+    REGISTER_DATEV2_FUNCTIONS_WITH_TWO_ARGS(NAME, DataTypeDateTimeV2, DataTypeDateTime)   \
+    REGISTER_DATEV2_FUNCTIONS_WITH_TWO_ARGS(NAME, DataTypeDateV2, DataTypeDateTime)       \
+    REGISTER_DATEV2_FUNCTIONS_WITH_TWO_ARGS(NAME, DataTypeDateV2, DataTypeDateV2)         \
+    REGISTER_DATEV2_FUNCTIONS_WITH_TWO_ARGS(NAME, DataTypeDateTime, DataTypeDateV2)       \
+    REGISTER_DATEV2_FUNCTIONS_WITH_TWO_ARGS(NAME, DataTypeDateTime, DataTypeDateTimeV2)
+
+    REGISTER_ALL_DATEV2_FUNCTIONS_WITH_TWO_ARGS(FunctionDatetimeV2DateDiff)
+    REGISTER_ALL_DATEV2_FUNCTIONS_WITH_TWO_ARGS(FunctionDatetimeV2TimeDiff)
+    REGISTER_ALL_DATEV2_FUNCTIONS_WITH_TWO_ARGS(FunctionDatetimeV2YearsDiff)
+    REGISTER_ALL_DATEV2_FUNCTIONS_WITH_TWO_ARGS(FunctionDatetimeV2MonthsDiff)
+    REGISTER_ALL_DATEV2_FUNCTIONS_WITH_TWO_ARGS(FunctionDatetimeV2WeeksDiff)
+    REGISTER_ALL_DATEV2_FUNCTIONS_WITH_TWO_ARGS(FunctionDatetimeV2HoursDiff)
+    REGISTER_ALL_DATEV2_FUNCTIONS_WITH_TWO_ARGS(FunctionDatetimeV2MinutesDiff)
+    REGISTER_ALL_DATEV2_FUNCTIONS_WITH_TWO_ARGS(FunctionDatetimeV2SecondsDiff)
+    REGISTER_ALL_DATEV2_FUNCTIONS_WITH_TWO_ARGS(FunctionDatetimeV2DaysDiff)
 
     factory.register_function<FunctionToYearWeekTwoArgs>();
     factory.register_function<FunctionToWeekTwoArgs>();
     factory.register_function<FunctionToYearWeekTwoArgsV2>();
     factory.register_function<FunctionToWeekTwoArgsV2>();
+    factory.register_function<FunctionDatetimeV2ToYearWeekTwoArgs>();
+    factory.register_function<FunctionDatetimeV2ToWeekTwoArgs>();
 
     factory.register_function<FunctionNow>();
     factory.register_function<FunctionCurrentTimestamp>();
diff --git a/be/src/vec/functions/function_date_or_datetime_computation.h b/be/src/vec/functions/function_date_or_datetime_computation.h
index 69be1e6bae..107d8db493 100644
--- a/be/src/vec/functions/function_date_or_datetime_computation.h
+++ b/be/src/vec/functions/function_date_or_datetime_computation.h
@@ -32,37 +32,72 @@
 #include "vec/runtime/vdatetime_value.h"
 namespace doris::vectorized {
 
-template <TimeUnit unit>
-inline Int64 date_time_add(const Int64& t, Int64 delta, bool& is_null) {
-    auto ts_value = binary_cast<Int64, doris::vectorized::VecDateTimeValue>(t);
+template <TimeUnit unit, typename Arg, typename DateValueType, typename ResultDateValueType,
+          typename ResultType>
+inline ResultType date_time_add(const Arg& t, Int64 delta, bool& is_null) {
+    auto ts_value = binary_cast<Arg, DateValueType>(t);
     TimeInterval interval(unit, delta, false);
-    is_null = !ts_value.date_add_interval(interval, unit);
+    if constexpr (std::is_same_v<VecDateTimeValue, DateValueType> ||
+                  std::is_same_v<DateValueType, ResultDateValueType>) {
+        is_null = !ts_value.date_add_interval(interval, unit);
 
-    return binary_cast<doris::vectorized::VecDateTimeValue, Int64>(ts_value);
-}
-
-template <TimeUnit unit>
-inline Int64 date_time_add(const UInt32& t, Int64 delta, bool& is_null) {
-    auto ts_value = binary_cast<UInt32, doris::vectorized::DateV2Value>(t);
-    TimeInterval interval(unit, delta, false);
-    is_null = !ts_value.date_add_interval(interval, unit);
+        return binary_cast<ResultDateValueType, ResultType>(ts_value);
+    } else {
+        ResultDateValueType res;
+        is_null = !ts_value.date_add_interval(interval, unit, res);
 
-    return binary_cast<doris::vectorized::DateV2Value, UInt32>(ts_value);
+        return binary_cast<ResultDateValueType, ResultType>(res);
+    }
 }
 
-#define ADD_TIME_FUNCTION_IMPL(CLASS, NAME, UNIT)                                     \
-    template <typename DateType, typename ArgType, typename ResultType>               \
-    struct CLASS {                                                                    \
-        using ReturnType = ResultType;                                                \
-        static constexpr auto name = #NAME;                                           \
-        static constexpr auto is_nullable = true;                                     \
-        static inline ArgType execute(const ArgType& t, Int64 delta, bool& is_null) { \
-            return date_time_add<TimeUnit::UNIT>(t, delta, is_null);                  \
-        }                                                                             \
-                                                                                      \
-        static DataTypes get_variadic_argument_types() {                              \
-            return {std::make_shared<DateType>(), std::make_shared<DataTypeInt32>()}; \
-        }                                                                             \
+#define ADD_TIME_FUNCTION_IMPL(CLASS, NAME, UNIT)                                                  \
+    template <typename DateType, typename ArgType, typename ResultType>                            \
+    struct CLASS {                                                                                 \
+        using ReturnType = ResultType;                                                             \
+        using ReturnNativeType = std::conditional_t<                                               \
+                std::is_same_v<DateType, DataTypeDate> ||                                          \
+                        std::is_same_v<DateType, DataTypeDateTime>,                                \
+                Int64,                                                                             \
+                std::conditional_t<                                                                \
+                        std::is_same_v<DateType, DataTypeDateV2>,                                  \
+                        std::conditional_t<TimeUnit::UNIT == TimeUnit::HOUR ||                     \
+                                                   TimeUnit::UNIT == TimeUnit::MINUTE ||           \
+                                                   TimeUnit::UNIT == TimeUnit::SECOND ||           \
+                                                   TimeUnit::UNIT == TimeUnit::SECOND_MICROSECOND, \
+                                           UInt64, UInt32>,                                        \
+                        UInt64>>;                                                                  \
+        static constexpr auto name = #NAME;                                                        \
+        static constexpr auto is_nullable = true;                                                  \
+        static inline ReturnNativeType execute(const ArgType& t, Int64 delta, bool& is_null) {     \
+            if constexpr (std::is_same_v<DateType, DataTypeDate> ||                                \
+                          std::is_same_v<DateType, DataTypeDateTime>) {                            \
+                return date_time_add<TimeUnit::UNIT, ArgType, doris::vectorized::VecDateTimeValue, \
+                                     doris::vectorized::VecDateTimeValue, ReturnNativeType>(       \
+                        t, delta, is_null);                                                        \
+            } else if constexpr (std::is_same_v<DateType, DataTypeDateV2>) {                       \
+                if constexpr (TimeUnit::UNIT == TimeUnit::HOUR ||                                  \
+                              TimeUnit::UNIT == TimeUnit::MINUTE ||                                \
+                              TimeUnit::UNIT == TimeUnit::SECOND ||                                \
+                              TimeUnit::UNIT == TimeUnit::SECOND_MICROSECOND) {                    \
+                    return date_time_add<TimeUnit::UNIT, ArgType, DateV2Value<DateV2ValueType>,    \
+                                         DateV2Value<DateTimeV2ValueType>, ReturnNativeType>(      \
+                            t, delta, is_null);                                                    \
+                } else {                                                                           \
+                    return date_time_add<TimeUnit::UNIT, ArgType, DateV2Value<DateV2ValueType>,    \
+                                         DateV2Value<DateV2ValueType>, ReturnNativeType>(t, delta, \
+                                                                                         is_null); \
+                }                                                                                  \
+                                                                                                   \
+            } else {                                                                               \
+                return date_time_add<TimeUnit::UNIT, ArgType, DateV2Value<DateTimeV2ValueType>,    \
+                                     DateV2Value<DateTimeV2ValueType>, ReturnNativeType>(t, delta, \
+                                                                                         is_null); \
+            }                                                                                      \
+        }                                                                                          \
+                                                                                                   \
+        static DataTypes get_variadic_argument_types() {                                           \
+            return {std::make_shared<DateType>(), std::make_shared<DataTypeInt32>()};              \
+        }                                                                                          \
     }
 
 ADD_TIME_FUNCTION_IMPL(AddSecondsImpl, seconds_add, SECOND);
@@ -76,10 +111,25 @@ ADD_TIME_FUNCTION_IMPL(AddYearsImpl, years_add, YEAR);
 template <typename DateType, typename ArgType, typename ResultType>
 struct AddQuartersImpl {
     using ReturnType = ResultType;
+    using ReturnNativeType = std::conditional_t<
+            std::is_same_v<DateType, DataTypeDate> || std::is_same_v<DateType, DataTypeDateTime>,
+            Int64, std::conditional_t<std::is_same_v<DateType, DataTypeDateV2>, UInt32, UInt64>>;
     static constexpr auto name = "quarters_add";
     static constexpr auto is_nullable = true;
-    static inline Int64 execute(const ArgType& t, Int64 delta, bool& is_null) {
-        return date_time_add<TimeUnit::MONTH>(t, delta * 3, is_null);
+    static inline ReturnNativeType execute(const ArgType& t, Int64 delta, bool& is_null) {
+        if constexpr (std::is_same_v<DateType, DataTypeDate> ||
+                      std::is_same_v<DateType, DataTypeDateTime>) {
+            return date_time_add<TimeUnit::MONTH, ArgType, doris::vectorized::VecDateTimeValue,
+                                 doris::vectorized::VecDateTimeValue, ReturnNativeType>(t, delta,
+                                                                                        is_null);
+        } else if constexpr (std::is_same_v<DateType, DataTypeDateV2>) {
+            return date_time_add<TimeUnit::MONTH, ArgType, DateV2Value<DateV2ValueType>,
+                                 DateV2Value<DateV2ValueType>, ReturnNativeType>(t, delta, is_null);
+        } else {
+            return date_time_add<TimeUnit::MONTH, ArgType, DateV2Value<DateTimeV2ValueType>,
+                                 DateV2Value<DateTimeV2ValueType>, ReturnNativeType>(t, delta,
+                                                                                     is_null);
+        }
     }
 
     static DataTypes get_variadic_argument_types() { return {std::make_shared<DateType>()}; }
@@ -450,6 +500,10 @@ public:
             return DateTimeAddIntervalImpl<DataTypeDateV2::FieldType, Transform,
                                            DataTypeDate::FieldType>::execute(block, arguments,
                                                                              result);
+        } else if (which1.is_date_time_v2() && which2.is_date()) {
+            return DateTimeAddIntervalImpl<DataTypeDateTimeV2::FieldType, Transform,
+                                           DataTypeDate::FieldType>::execute(block, arguments,
+                                                                             result);
         } else if (which1.is_date() && which2.is_date_time()) {
             return DateTimeAddIntervalImpl<DataTypeDate::FieldType, Transform,
                                            DataTypeDateTime::FieldType>::execute(block, arguments,
@@ -458,6 +512,10 @@ public:
             return DateTimeAddIntervalImpl<DataTypeDate::FieldType, Transform,
                                            DataTypeDateV2::FieldType>::execute(block, arguments,
                                                                                result);
+        } else if (which1.is_date() && which2.is_date_time_v2()) {
+            return DateTimeAddIntervalImpl<DataTypeDate::FieldType, Transform,
+                                           DataTypeDateTimeV2::FieldType>::execute(block, arguments,
+                                                                                   result);
         } else if (which1.is_date_v2() && which2.is_date_time()) {
             return DateTimeAddIntervalImpl<DataTypeDateV2::FieldType, Transform,
                                            DataTypeDateTime::FieldType>::execute(block, arguments,
@@ -466,6 +524,14 @@ public:
             return DateTimeAddIntervalImpl<DataTypeDateV2::FieldType, Transform,
                                            DataTypeDateV2::FieldType>::execute(block, arguments,
                                                                                result);
+        } else if (which1.is_date_time_v2() && which2.is_date_time()) {
+            return DateTimeAddIntervalImpl<DataTypeDateTimeV2::FieldType, Transform,
+                                           DataTypeDateTime::FieldType>::execute(block, arguments,
+                                                                                 result);
+        } else if (which1.is_date_time_v2() && which2.is_date_time_v2()) {
+            return DateTimeAddIntervalImpl<DataTypeDateTimeV2::FieldType, Transform,
+                                           DataTypeDateTimeV2::FieldType>::execute(block, arguments,
+                                                                                   result);
         } else if (which1.is_date_time() && which2.is_date_time()) {
             return DateTimeAddIntervalImpl<DataTypeDateTime::FieldType, Transform,
                                            DataTypeDateTime::FieldType>::execute(block, arguments,
@@ -474,6 +540,18 @@ public:
             return DateTimeAddIntervalImpl<DataTypeDateTime::FieldType, Transform,
                                            DataTypeDateV2::FieldType>::execute(block, arguments,
                                                                                result);
+        } else if (which1.is_date_time() && which2.is_date_time_v2()) {
+            return DateTimeAddIntervalImpl<DataTypeDateTime::FieldType, Transform,
+                                           DataTypeDateTimeV2::FieldType>::execute(block, arguments,
+                                                                                   result);
+        } else if (which1.is_date_v2() && which2.is_date_time_v2()) {
+            return DateTimeAddIntervalImpl<DataTypeDateV2::FieldType, Transform,
+                                           DataTypeDateTimeV2::FieldType>::execute(block, arguments,
+                                                                                   result);
+        } else if (which1.is_date_time_v2() && which2.is_date_v2()) {
+            return DateTimeAddIntervalImpl<DataTypeDateTimeV2::FieldType, Transform,
+                                           DataTypeDateV2::FieldType>::execute(block, arguments,
+                                                                               result);
         } else if (which1.is_date()) {
             return DateTimeAddIntervalImpl<DataTypeDate::FieldType, Transform>::execute(
                     block, arguments, result);
@@ -483,6 +561,9 @@ public:
         } else if (which1.is_date_v2()) {
             return DateTimeAddIntervalImpl<DataTypeDateV2::FieldType, Transform>::execute(
                     block, arguments, result);
+        } else if (which1.is_date_time_v2()) {
+            return DateTimeAddIntervalImpl<DataTypeDateTimeV2::FieldType, Transform>::execute(
+                    block, arguments, result);
         } else {
             return Status::RuntimeError("Illegal type {} of argument of function {}",
                                         block.get_by_position(arguments[0]).type->get_name(),
@@ -549,11 +630,11 @@ struct CurrentDateImpl {
                           size_t input_rows_count) {
         auto col_to = ColumnVector<NativeType>::create();
         if constexpr (std::is_same_v<DateType, DataTypeDateV2>) {
-            DateV2Value dtv;
+            DateV2Value<DateV2ValueType> dtv;
             if (dtv.from_unixtime(context->impl()->state()->timestamp_ms() / 1000,
                                   context->impl()->state()->timezone_obj())) {
-                auto date_packed_int =
-                        binary_cast<DateV2Value, uint32_t>(*reinterpret_cast<DateV2Value*>(&dtv));
+                auto date_packed_int = binary_cast<DateV2Value<DateV2ValueType>, uint32_t>(
+                        *reinterpret_cast<DateV2Value<DateV2ValueType>*>(&dtv));
                 for (int i = 0; i < input_rows_count; i++) {
                     col_to->insert_data(
                             const_cast<const char*>(reinterpret_cast<char*>(&date_packed_int)), 0);
@@ -668,6 +749,10 @@ protected:
             auto function = FunctionCurrentDateOrDateTime<
                     CurrentDateImpl<FunctionName, DataTypeDateV2, UInt32>>::create();
             return std::make_shared<DefaultFunction>(function, data_types, return_type);
+        } else if (is_date_time_v2(return_type)) {
+            auto function = FunctionCurrentDateOrDateTime<
+                    CurrentDateImpl<FunctionName, DataTypeDateTimeV2, UInt64>>::create();
+            return std::make_shared<DefaultFunction>(function, data_types, return_type);
         } else {
             auto function = FunctionCurrentDateOrDateTime<
                     CurrentDateImpl<FunctionName, DataTypeDate, Int64>>::create();
diff --git a/be/src/vec/functions/function_date_or_datetime_to_string.cpp b/be/src/vec/functions/function_date_or_datetime_to_string.cpp
index 6f9d815fdd..374f38a90e 100644
--- a/be/src/vec/functions/function_date_or_datetime_to_string.cpp
+++ b/be/src/vec/functions/function_date_or_datetime_to_string.cpp
@@ -25,15 +25,24 @@
 namespace doris::vectorized {
 
 using FunctionDayName = FunctionDateOrDateTimeToString<DayNameImpl<VecDateTimeValue, Int64>>;
-using FunctionDayNameV2 = FunctionDateOrDateTimeToString<DayNameImpl<DateV2Value, UInt32>>;
+using FunctionDayNameV2 =
+        FunctionDateOrDateTimeToString<DayNameImpl<DateV2Value<DateV2ValueType>, UInt32>>;
 using FunctionMonthName = FunctionDateOrDateTimeToString<MonthNameImpl<VecDateTimeValue, Int64>>;
-using FunctionMonthNameV2 = FunctionDateOrDateTimeToString<MonthNameImpl<DateV2Value, UInt32>>;
+using FunctionMonthNameV2 =
+        FunctionDateOrDateTimeToString<MonthNameImpl<DateV2Value<DateV2ValueType>, UInt32>>;
+
+using FunctionDateTimeV2DayName =
+        FunctionDateOrDateTimeToString<DayNameImpl<DateV2Value<DateTimeV2ValueType>, UInt64>>;
+using FunctionDateTimeV2MonthName =
+        FunctionDateOrDateTimeToString<MonthNameImpl<DateV2Value<DateTimeV2ValueType>, UInt64>>;
 
 void register_function_date_time_to_string(SimpleFunctionFactory& factory) {
     factory.register_function<FunctionDayName>();
     factory.register_function<FunctionMonthName>();
     factory.register_function<FunctionDayNameV2>();
     factory.register_function<FunctionMonthNameV2>();
+    factory.register_function<FunctionDateTimeV2DayName>();
+    factory.register_function<FunctionDateTimeV2MonthName>();
 }
 
 } // namespace doris::vectorized
\ No newline at end of file
diff --git a/be/src/vec/functions/function_datetime_string_to_string.cpp b/be/src/vec/functions/function_datetime_string_to_string.cpp
index e4685a96a7..a871ac224b 100644
--- a/be/src/vec/functions/function_datetime_string_to_string.cpp
+++ b/be/src/vec/functions/function_datetime_string_to_string.cpp
@@ -22,7 +22,10 @@
 namespace doris::vectorized {
 
 using FunctionDateFormat = FunctionDateTimeStringToString<DateFormatImpl<VecDateTimeValue, Int64>>;
-using FunctionDateFormatV2 = FunctionDateTimeStringToString<DateFormatImpl<DateV2Value, UInt32>>;
+using FunctionDateFormatV2 =
+        FunctionDateTimeStringToString<DateFormatImpl<DateV2Value<DateV2ValueType>, UInt32>>;
+using FunctionDateTimeV2DateFormat =
+        FunctionDateTimeStringToString<DateFormatImpl<DateV2Value<DateTimeV2ValueType>, UInt64>>;
 using FunctionFromUnixTime = FunctionDateTimeStringToString<FromUnixTimeImpl<VecDateTimeValue>>;
 
 FunctionBuilderPtr createFromUnixTimeFunction() {
@@ -33,6 +36,7 @@ void register_function_date_time_string_to_string(SimpleFunctionFactory& factory
     factory.register_function<FunctionDateFormat>();
     factory.register_function<FunctionDateFormatV2>();
     factory.register_function<FunctionFromUnixTime>();
+    factory.register_function<FunctionDateTimeV2DateFormat>();
     factory.register_function("from_unixtime", &createFromUnixTimeFunction);
 }
 
diff --git a/be/src/vec/functions/function_timestamp.cpp b/be/src/vec/functions/function_timestamp.cpp
index 8a50953e85..4c3786dfba 100644
--- a/be/src/vec/functions/function_timestamp.cpp
+++ b/be/src/vec/functions/function_timestamp.cpp
@@ -62,7 +62,7 @@ struct StrToDate {
                     ts_val.cast_to_date();
                 }
             } else {
-                auto& ts_val = *reinterpret_cast<DateV2Value*>(&res[i]);
+                auto& ts_val = *reinterpret_cast<DateV2Value<DateV2ValueType>*>(&res[i]);
                 if (!ts_val.from_date_format_str(r_raw_str, r_str_size, l_raw_str, l_str_size)) {
                     null_map[i] = 1;
                 }
@@ -119,10 +119,10 @@ struct MakeDateImpl {
                 }
                 res_val.cast_to_date();
             } else {
-                DateV2Value* value = new (&res[i]) DateV2Value();
+                DateV2Value<DateV2ValueType>* value = new (&res[i]) DateV2Value<DateV2ValueType>();
                 value->set_time(l, 1, 1);
                 TimeInterval interval(DAY, r - 1, false);
-                if (!value->date_add_interval(interval, DAY)) {
+                if (!value->date_add_interval(interval, DAY, *value)) {
                     null_map[i] = 1;
                 }
             }
@@ -171,7 +171,7 @@ public:
                 ts_value = VecDateTimeValue::from_datetime_val(ts_val);
             } else {
                 const auto& cur_data = data_col->get_data()[i];
-                auto& ts_value = *reinterpret_cast<DateV2Value*>(&res_data[i]);
+                auto& ts_value = *reinterpret_cast<DateV2Value<DateV2ValueType>*>(&res_data[i]);
                 if (!ts_value.get_date_from_daynr(cur_data)) {
                     null_map->get_data()[i] = 1;
                 }
@@ -259,7 +259,8 @@ struct UnixTimeStampDateImpl {
                     col_result_data[i] = UnixTimeStampImpl::trim_timestamp(timestamp);
                 }
             } else {
-                const DateV2Value& ts_value = reinterpret_cast<const DateV2Value&>(*source.data);
+                const DateV2Value<DateV2ValueType>& ts_value =
+                        reinterpret_cast<const DateV2Value<DateV2ValueType>&>(*source.data);
                 int64_t timestamp;
                 if (!ts_value.unix_timestamp(&timestamp,
                                              context->impl()->state()->timezone_obj())) {
diff --git a/be/src/vec/functions/time_of_function.cpp b/be/src/vec/functions/time_of_function.cpp
index 86aedbae48..42b0b8c198 100644
--- a/be/src/vec/functions/time_of_function.cpp
+++ b/be/src/vec/functions/time_of_function.cpp
@@ -25,28 +25,51 @@ namespace doris::vectorized {
 using FunctionWeekOfYear =
         FunctionDateOrDateTimeToSomething<DataTypeInt32, WeekOfYearImpl<VecDateTimeValue, Int64>>;
 using FunctionWeekOfYearV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeInt32, WeekOfYearImpl<DateV2Value, UInt32>>;
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          WeekOfYearImpl<DateV2Value<DateV2ValueType>, UInt32>>;
 using FunctionDayOfYear =
         FunctionDateOrDateTimeToSomething<DataTypeInt32, DayOfYearImpl<VecDateTimeValue, Int64>>;
 using FunctionDayOfYearV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeInt32, DayOfYearImpl<DateV2Value, UInt32>>;
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          DayOfYearImpl<DateV2Value<DateV2ValueType>, UInt32>>;
 using FunctionDayOfWeek =
         FunctionDateOrDateTimeToSomething<DataTypeInt32, DayOfWeekImpl<VecDateTimeValue, Int64>>;
 using FunctionDayOfWeekV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeInt32, DayOfWeekImpl<DateV2Value, UInt32>>;
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          DayOfWeekImpl<DateV2Value<DateV2ValueType>, UInt32>>;
 using FunctionDayOfMonth =
         FunctionDateOrDateTimeToSomething<DataTypeInt32, DayOfMonthImpl<VecDateTimeValue, Int64>>;
 using FunctionDayOfMonthV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeInt32, DayOfMonthImpl<DateV2Value, UInt32>>;
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          DayOfMonthImpl<DateV2Value<DateV2ValueType>, UInt32>>;
 using FunctionYearWeek =
         FunctionDateOrDateTimeToSomething<DataTypeInt32,
                                           ToYearWeekOneArgImpl<VecDateTimeValue, Int64>>;
-using FunctionYearWeekV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeInt32, ToYearWeekOneArgImpl<DateV2Value, UInt32>>;
+using FunctionYearWeekV2 = FunctionDateOrDateTimeToSomething<
+        DataTypeInt32, ToYearWeekOneArgImpl<DateV2Value<DateV2ValueType>, UInt32>>;
 using FunctionWeekDay =
         FunctionDateOrDateTimeToSomething<DataTypeInt32, WeekDayImpl<VecDateTimeValue, Int64>>;
 using FunctionWeekDayV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeInt32, WeekDayImpl<DateV2Value, UInt32>>;
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          WeekDayImpl<DateV2Value<DateV2ValueType>, UInt32>>;
+
+using FunctionDateTimeV2WeekOfYear =
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          WeekOfYearImpl<DateV2Value<DateTimeV2ValueType>, UInt64>>;
+using FunctionDateTimeV2DayOfYear =
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          DayOfYearImpl<DateV2Value<DateTimeV2ValueType>, UInt64>>;
+using FunctionDateTimeV2DayOfWeek =
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          DayOfWeekImpl<DateV2Value<DateTimeV2ValueType>, UInt64>>;
+using FunctionDateTimeV2DayOfMonth =
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          DayOfMonthImpl<DateV2Value<DateTimeV2ValueType>, UInt64>>;
+using FunctionDateTimeV2YearWeek = FunctionDateOrDateTimeToSomething<
+        DataTypeInt32, ToYearWeekOneArgImpl<DateV2Value<DateTimeV2ValueType>, UInt64>>;
+using FunctionDateTimeV2WeekDay =
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          WeekDayImpl<DateV2Value<DateTimeV2ValueType>, UInt64>>;
 
 void register_function_time_of_function(SimpleFunctionFactory& factory) {
     factory.register_function<FunctionDayOfWeek>();
@@ -61,5 +84,11 @@ void register_function_time_of_function(SimpleFunctionFactory& factory) {
     factory.register_function<FunctionWeekOfYearV2>();
     factory.register_function<FunctionYearWeekV2>();
     factory.register_function<FunctionWeekDayV2>();
+    factory.register_function<FunctionDateTimeV2WeekOfYear>();
+    factory.register_function<FunctionDateTimeV2DayOfYear>();
+    factory.register_function<FunctionDateTimeV2DayOfWeek>();
+    factory.register_function<FunctionDateTimeV2DayOfMonth>();
+    factory.register_function<FunctionDateTimeV2YearWeek>();
+    factory.register_function<FunctionDateTimeV2WeekDay>();
 }
 } // namespace doris::vectorized
\ No newline at end of file
diff --git a/be/src/vec/functions/to_time_function.cpp b/be/src/vec/functions/to_time_function.cpp
index 812e16fbad..b370dc19a7 100644
--- a/be/src/vec/functions/to_time_function.cpp
+++ b/be/src/vec/functions/to_time_function.cpp
@@ -26,47 +26,93 @@ namespace doris::vectorized {
 using FunctionYear =
         FunctionDateOrDateTimeToSomething<DataTypeInt32, ToYearImpl<VecDateTimeValue, Int64>>;
 using FunctionYearV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeInt32, ToYearImpl<DateV2Value, UInt32>>;
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToYearImpl<DateV2Value<DateV2ValueType>, UInt32>>;
 using FunctionQuarter =
         FunctionDateOrDateTimeToSomething<DataTypeInt32, ToQuarterImpl<VecDateTimeValue, Int64>>;
 using FunctionQuarterV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeInt32, ToQuarterImpl<DateV2Value, UInt32>>;
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToQuarterImpl<DateV2Value<DateV2ValueType>, UInt32>>;
 using FunctionMonth =
         FunctionDateOrDateTimeToSomething<DataTypeInt32, ToMonthImpl<VecDateTimeValue, Int64>>;
 using FunctionMonthV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeInt32, ToMonthImpl<DateV2Value, UInt32>>;
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToMonthImpl<DateV2Value<DateV2ValueType>, UInt32>>;
 using FunctionDay =
         FunctionDateOrDateTimeToSomething<DataTypeInt32, ToDayImpl<VecDateTimeValue, Int64>>;
 using FunctionDayV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeInt32, ToDayImpl<DateV2Value, UInt32>>;
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToDayImpl<DateV2Value<DateV2ValueType>, UInt32>>;
 using FunctionWeek =
         FunctionDateOrDateTimeToSomething<DataTypeInt32, ToWeekOneArgImpl<VecDateTimeValue, Int64>>;
 using FunctionWeekV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeInt32, ToWeekOneArgImpl<DateV2Value, UInt32>>;
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToWeekOneArgImpl<DateV2Value<DateV2ValueType>, UInt32>>;
 using FunctionHour =
         FunctionDateOrDateTimeToSomething<DataTypeInt32, ToHourImpl<VecDateTimeValue, Int64>>;
 using FunctionHourV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeInt32, ToHourImpl<DateV2Value, UInt32>>;
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToHourImpl<DateV2Value<DateV2ValueType>, UInt32>>;
 using FunctionMinute =
         FunctionDateOrDateTimeToSomething<DataTypeInt32, ToMinuteImpl<VecDateTimeValue, Int64>>;
 using FunctionMinuteV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeInt32, ToMinuteImpl<DateV2Value, UInt32>>;
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToMinuteImpl<DateV2Value<DateV2ValueType>, UInt32>>;
 using FunctionSecond =
         FunctionDateOrDateTimeToSomething<DataTypeInt32, ToSecondImpl<VecDateTimeValue, Int64>>;
 using FunctionSecondV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeInt32, ToSecondImpl<DateV2Value, UInt32>>;
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToSecondImpl<DateV2Value<DateV2ValueType>, UInt32>>;
 using FunctionToDays =
         FunctionDateOrDateTimeToSomething<DataTypeInt32, ToDaysImpl<VecDateTimeValue, Int64>>;
 using FunctionToDaysV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeInt32, ToDaysImpl<DateV2Value, UInt32>>;
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToDaysImpl<DateV2Value<DateV2ValueType>, UInt32>>;
 using FunctionToDate =
         FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToDateImpl<VecDateTimeValue, Int64>>;
 using FunctionToDateV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeDateV2, ToDateImpl<DateV2Value, UInt32>>;
+        FunctionDateOrDateTimeToSomething<DataTypeDateV2,
+                                          ToDateImpl<DateV2Value<DateV2ValueType>, UInt32>>;
 using FunctionDate =
         FunctionDateOrDateTimeToSomething<DataTypeDateTime, DateImpl<VecDateTimeValue, Int64>>;
 using FunctionDateV2 =
-        FunctionDateOrDateTimeToSomething<DataTypeDateV2, DateImpl<DateV2Value, UInt32>>;
+        FunctionDateOrDateTimeToSomething<DataTypeDateV2,
+                                          DateImpl<DateV2Value<DateV2ValueType>, UInt32>>;
+
+using FunctionDateTimeV2Year =
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToYearImpl<DateV2Value<DateV2ValueType>, UInt32>>;
+using FunctionDateTimeV2Quarter =
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToQuarterImpl<DateV2Value<DateV2ValueType>, UInt32>>;
+using FunctionDateTimeV2Month =
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToMonthImpl<DateV2Value<DateV2ValueType>, UInt32>>;
+using FunctionDateTimeV2Day =
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToDayImpl<DateV2Value<DateV2ValueType>, UInt32>>;
+using FunctionDateTimeV2Week =
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToWeekOneArgImpl<DateV2Value<DateV2ValueType>, UInt32>>;
+using FunctionDateTimeV2Hour =
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToHourImpl<DateV2Value<DateV2ValueType>, UInt32>>;
+using FunctionDateTimeV2Minute =
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToMinuteImpl<DateV2Value<DateV2ValueType>, UInt32>>;
+using FunctionDateTimeV2Second =
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToSecondImpl<DateV2Value<DateV2ValueType>, UInt32>>;
+using FunctionDateTimeV2ToDays =
+        FunctionDateOrDateTimeToSomething<DataTypeInt32,
+                                          ToDaysImpl<DateV2Value<DateV2ValueType>, UInt32>>;
+using FunctionDateTimeV2ToDate =
+        FunctionDateOrDateTimeToSomething<DataTypeDateV2,
+                                          ToDateImpl<DateV2Value<DateV2ValueType>, UInt32>>;
+using FunctionDateTimeV2Date =
+        FunctionDateOrDateTimeToSomething<DataTypeDateV2,
+                                          DateImpl<DateV2Value<DateV2ValueType>, UInt32>>;
+
 using FunctionTimeStamp = FunctionDateOrDateTimeToSomething<DataTypeDateTime, TimeStampImpl<Int64>>;
 
 void register_function_to_time_function(SimpleFunctionFactory& factory) {
@@ -93,6 +139,17 @@ void register_function_to_time_function(SimpleFunctionFactory& factory) {
     factory.register_function<FunctionToDaysV2>();
     factory.register_function<FunctionToDateV2>();
     factory.register_function<FunctionDateV2>();
+    factory.register_function<FunctionDateTimeV2Second>();
+    factory.register_function<FunctionDateTimeV2Minute>();
+    factory.register_function<FunctionDateTimeV2Hour>();
+    factory.register_function<FunctionDateTimeV2Day>();
+    factory.register_function<FunctionDateTimeV2Week>();
+    factory.register_function<FunctionDateTimeV2Month>();
+    factory.register_function<FunctionDateTimeV2Year>();
+    factory.register_function<FunctionDateTimeV2Quarter>();
+    factory.register_function<FunctionDateTimeV2ToDays>();
+    factory.register_function<FunctionDateTimeV2ToDate>();
+    factory.register_function<FunctionDateTimeV2Date>();
 }
 
 } // namespace doris::vectorized
diff --git a/be/src/vec/io/io_helper.h b/be/src/vec/io/io_helper.h
index 51122b26b5..00cddc22e9 100644
--- a/be/src/vec/io/io_helper.h
+++ b/be/src/vec/io/io_helper.h
@@ -283,12 +283,24 @@ bool read_date_text_impl(T& x, ReadBuffer& buf) {
 template <typename T>
 bool read_date_v2_text_impl(T& x, ReadBuffer& buf) {
     static_assert(std::is_same_v<UInt32, T>);
-    auto dv = binary_cast<UInt32, DateV2Value>(x);
+    auto dv = binary_cast<UInt32, DateV2Value<DateV2ValueType>>(x);
     auto ans = dv.from_date_str(buf.position(), buf.count());
 
     // only to match the is_all_read() check to prevent return null
     buf.position() = buf.end();
-    x = binary_cast<DateV2Value, UInt32>(dv);
+    x = binary_cast<DateV2Value<DateV2ValueType>, UInt32>(dv);
+    return ans;
+}
+
+template <typename T>
+bool read_datetime_v2_text_impl(T& x, ReadBuffer& buf) {
+    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());
+
+    // only to match the is_all_read() check to prevent return null
+    buf.position() = buf.end();
+    x = binary_cast<DateV2Value<DateTimeV2ValueType>, UInt64>(dv);
     return ans;
 }
 
@@ -360,4 +372,9 @@ template <typename T>
 bool try_read_date_v2_text(T& x, ReadBuffer& in) {
     return read_date_v2_text_impl<T>(x, in);
 }
+
+template <typename T>
+bool try_read_datetime_v2_text(T& x, ReadBuffer& in) {
+    return read_datetime_v2_text_impl<T>(x, in);
+}
 } // namespace doris::vectorized
diff --git a/be/src/vec/olap/olap_data_convertor.cpp b/be/src/vec/olap/olap_data_convertor.cpp
index 63ee54ed28..c8d07e4a8e 100644
--- a/be/src/vec/olap/olap_data_convertor.cpp
+++ b/be/src/vec/olap/olap_data_convertor.cpp
@@ -56,12 +56,16 @@ OlapBlockDataConvertor::create_olap_column_data_convertor(const TabletColumn& co
     case FieldType::OLAP_FIELD_TYPE_DATE: {
         return std::make_unique<OlapColumnDataConvertorDate>();
     }
+    case FieldType::OLAP_FIELD_TYPE_DATETIME: {
+        return std::make_unique<OlapColumnDataConvertorDateTime>();
+    }
     case FieldType::OLAP_FIELD_TYPE_DATEV2: {
         return std::make_unique<OlapColumnDataConvertorDateV2>();
         break;
     }
-    case FieldType::OLAP_FIELD_TYPE_DATETIME: {
-        return std::make_unique<OlapColumnDataConvertorDateTime>();
+    case FieldType::OLAP_FIELD_TYPE_DATETIMEV2: {
+        return std::make_unique<OlapColumnDataConvertorDateTimeV2>();
+        break;
     }
     case FieldType::OLAP_FIELD_TYPE_DECIMAL: {
         return std::make_unique<OlapColumnDataConvertorDecimal>();
@@ -507,9 +511,10 @@ Status OlapBlockDataConvertor::OlapColumnDataConvertorDate::convert_to_olap() {
 
         assert(column_datetime);
 
-        const DateV2Value* datetime_cur =
-                (const DateV2Value*)(column_datetime->get_data().data()) + _row_pos;
-        const DateV2Value* datetime_end = datetime_cur + _num_rows;
+        const DateV2Value<DateV2ValueType>* datetime_cur =
+                (const DateV2Value<DateV2ValueType>*)(column_datetime->get_data().data()) +
+                _row_pos;
+        const DateV2Value<DateV2ValueType>* datetime_end = datetime_cur + _num_rows;
         uint24_t* value = _values.data();
         if (_nullmap) {
             const UInt8* nullmap_cur = _nullmap + _row_pos;
@@ -578,47 +583,103 @@ Status OlapBlockDataConvertor::OlapColumnDataConvertorDate::convert_to_olap() {
     }
 }
 
-Status OlapBlockDataConvertor::OlapColumnDataConvertorDateTime::convert_to_olap() {
-    assert(_typed_column.column);
-    const vectorized::ColumnVector<vectorized::Int64>* column_datetime = nullptr;
-    if (_nullmap) {
-        auto nullable_column =
-                assert_cast<const vectorized::ColumnNullable*>(_typed_column.column.get());
-        column_datetime = assert_cast<const vectorized::ColumnVector<vectorized::Int64>*>(
-                nullable_column->get_nested_column_ptr().get());
+void OlapBlockDataConvertor::OlapColumnDataConvertorDateTime::set_source_column(
+        const ColumnWithTypeAndName& typed_column, size_t row_pos, size_t num_rows) {
+    OlapBlockDataConvertor::OlapColumnDataConvertorPaddedPODArray<uint64_t>::set_source_column(
+            typed_column, row_pos, num_rows);
+    if (is_date_v2_or_datetime_v2(typed_column.type)) {
+        from_datetime_v2_ = true;
     } else {
-        column_datetime = assert_cast<const vectorized::ColumnVector<vectorized::Int64>*>(
-                _typed_column.column.get());
+        from_datetime_v2_ = false;
     }
+}
 
-    assert(column_datetime);
+Status OlapBlockDataConvertor::OlapColumnDataConvertorDateTime::convert_to_olap() {
+    assert(_typed_column.column);
+    if (from_datetime_v2_) {
+        const vectorized::ColumnVector<vectorized::UInt64>* column_datetimev2 = nullptr;
+        if (_nullmap) {
+            auto nullable_column =
+                    assert_cast<const vectorized::ColumnNullable*>(_typed_column.column.get());
+            column_datetimev2 = assert_cast<const vectorized::ColumnVector<vectorized::UInt64>*>(
+                    nullable_column->get_nested_column_ptr().get());
+        } else {
+            column_datetimev2 = assert_cast<const vectorized::ColumnVector<vectorized::UInt64>*>(
+                    _typed_column.column.get());
+        }
 
-    const VecDateTimeValue* datetime_cur =
-            (const VecDateTimeValue*)(column_datetime->get_data().data()) + _row_pos;
-    const VecDateTimeValue* datetime_end = datetime_cur + _num_rows;
-    uint64_t* value = _values.data();
-    if (_nullmap) {
-        const UInt8* nullmap_cur = _nullmap + _row_pos;
-        while (datetime_cur != datetime_end) {
-            if (!*nullmap_cur) {
+        assert(column_datetimev2);
+
+        const DateV2Value<DateTimeV2ValueType>* datetime_cur =
+                (const DateV2Value<DateTimeV2ValueType>*)(column_datetimev2->get_data().data()) +
+                _row_pos;
+        const DateV2Value<DateTimeV2ValueType>* datetime_end = datetime_cur + _num_rows;
+        uint64_t* value = _values.data();
+        if (_nullmap) {
+            const UInt8* nullmap_cur = _nullmap + _row_pos;
+            while (datetime_cur != datetime_end) {
+                if (!*nullmap_cur) {
+                    *value = datetime_cur->to_olap_datetime();
+                } else {
+                    // do nothing
+                }
+                ++value;
+                ++datetime_cur;
+                ++nullmap_cur;
+            }
+            assert(nullmap_cur == _nullmap + _row_pos + _num_rows &&
+                   value == _values.get_end_ptr());
+        } else {
+            while (datetime_cur != datetime_end) {
                 *value = datetime_cur->to_olap_datetime();
-            } else {
-                // do nothing
+                ++value;
+                ++datetime_cur;
             }
-            ++value;
-            ++datetime_cur;
-            ++nullmap_cur;
+            assert(value == _values.get_end_ptr());
         }
-        assert(nullmap_cur == _nullmap + _row_pos + _num_rows && value == _values.get_end_ptr());
+        return Status::OK();
     } else {
-        while (datetime_cur != datetime_end) {
-            *value = datetime_cur->to_olap_datetime();
-            ++value;
-            ++datetime_cur;
+        const vectorized::ColumnVector<vectorized::Int64>* column_datetime = nullptr;
+        if (_nullmap) {
+            auto nullable_column =
+                    assert_cast<const vectorized::ColumnNullable*>(_typed_column.column.get());
+            column_datetime = assert_cast<const vectorized::ColumnVector<vectorized::Int64>*>(
+                    nullable_column->get_nested_column_ptr().get());
+        } else {
+            column_datetime = assert_cast<const vectorized::ColumnVector<vectorized::Int64>*>(
+                    _typed_column.column.get());
         }
-        assert(value == _values.get_end_ptr());
+
+        assert(column_datetime);
+
+        const VecDateTimeValue* datetime_cur =
+                (const VecDateTimeValue*)(column_datetime->get_data().data()) + _row_pos;
+        const VecDateTimeValue* datetime_end = datetime_cur + _num_rows;
+        uint64_t* value = _values.data();
+        if (_nullmap) {
+            const UInt8* nullmap_cur = _nullmap + _row_pos;
+            while (datetime_cur != datetime_end) {
+                if (!*nullmap_cur) {
+                    *value = datetime_cur->to_olap_datetime();
+                } else {
+                    // do nothing
+                }
+                ++value;
+                ++datetime_cur;
+                ++nullmap_cur;
+            }
+            assert(nullmap_cur == _nullmap + _row_pos + _num_rows &&
+                   value == _values.get_end_ptr());
+        } else {
+            while (datetime_cur != datetime_end) {
+                *value = datetime_cur->to_olap_datetime();
+                ++value;
+                ++datetime_cur;
+            }
+            assert(value == _values.get_end_ptr());
+        }
+        return Status::OK();
     }
-    return Status::OK();
 }
 
 Status OlapBlockDataConvertor::OlapColumnDataConvertorDecimal::convert_to_olap() {
diff --git a/be/src/vec/olap/olap_data_convertor.h b/be/src/vec/olap/olap_data_convertor.h
index 7b844d04f6..719c7d78ca 100644
--- a/be/src/vec/olap/olap_data_convertor.h
+++ b/be/src/vec/olap/olap_data_convertor.h
@@ -201,7 +201,12 @@ private:
 
     class OlapColumnDataConvertorDateTime : public OlapColumnDataConvertorPaddedPODArray<uint64_t> {
     public:
+        void set_source_column(const ColumnWithTypeAndName& typed_column, size_t row_pos,
+                               size_t num_rows) override;
         Status convert_to_olap() override;
+
+    private:
+        bool from_datetime_v2_;
     };
 
     class OlapColumnDataConvertorDecimal
@@ -310,7 +315,7 @@ private:
                     }
                 } else {
                     while (datetime_cur != datetime_end) {
-                        *value = datetime_cur->to_olap_date();
+                        *value = datetime_cur->to_date_v2();
                         ++value;
                         ++datetime_cur;
                     }
@@ -339,6 +344,96 @@ private:
         bool from_date_to_date_v2_;
     };
 
+    class OlapColumnDataConvertorDateTimeV2 : public OlapColumnDataConvertorBase {
+    public:
+        OlapColumnDataConvertorDateTimeV2() = default;
+        ~OlapColumnDataConvertorDateTimeV2() override = default;
+
+        void set_source_column(const ColumnWithTypeAndName& typed_column, size_t row_pos,
+                               size_t num_rows) override {
+            OlapColumnDataConvertorBase::set_source_column(typed_column, row_pos, num_rows);
+            if (is_date_or_datetime(typed_column.type)) {
+                from_datetime_to_datetime_v2_ = true;
+            } else {
+                from_datetime_to_datetime_v2_ = false;
+            }
+        }
+
+        const void* get_data() const override { return values_; }
+
+        const void* get_data_at(size_t offset) const override {
+            assert(offset < _num_rows);
+            UInt8 null_flag = 0;
+            if (_nullmap) {
+                null_flag = _nullmap[offset];
+            }
+            return null_flag ? nullptr : values_ + offset;
+        }
+
+        Status convert_to_olap() override {
+            if (UNLIKELY(from_datetime_to_datetime_v2_)) {
+                const vectorized::ColumnVector<vectorized::Int64>* column_datetime = nullptr;
+                if (_nullmap) {
+                    auto nullable_column = assert_cast<const vectorized::ColumnNullable*>(
+                            _typed_column.column.get());
+                    column_datetime =
+                            assert_cast<const vectorized::ColumnVector<vectorized::Int64>*>(
+                                    nullable_column->get_nested_column_ptr().get());
+                } else {
+                    column_datetime =
+                            assert_cast<const vectorized::ColumnVector<vectorized::Int64>*>(
+                                    _typed_column.column.get());
+                }
+
+                assert(column_datetime);
+
+                const VecDateTimeValue* datetime_cur =
+                        (const VecDateTimeValue*)(column_datetime->get_data().data()) + _row_pos;
+                const VecDateTimeValue* datetime_end = datetime_cur + _num_rows;
+                uint64_t* value = const_cast<uint64_t*>(values_);
+                if (_nullmap) {
+                    const UInt8* nullmap_cur = _nullmap + _row_pos;
+                    while (datetime_cur != datetime_end) {
+                        if (!*nullmap_cur) {
+                            *value = datetime_cur->to_datetime_v2();
+                        } else {
+                            // do nothing
+                        }
+                        ++value;
+                        ++datetime_cur;
+                        ++nullmap_cur;
+                    }
+                } else {
+                    while (datetime_cur != datetime_end) {
+                        *value = datetime_cur->to_datetime_v2();
+                        ++value;
+                        ++datetime_cur;
+                    }
+                }
+                return Status::OK();
+            } else {
+                const vectorized::ColumnVector<uint64_t>* column_data = nullptr;
+                if (_nullmap) {
+                    auto nullable_column = assert_cast<const vectorized::ColumnNullable*>(
+                            _typed_column.column.get());
+                    column_data = assert_cast<const vectorized::ColumnVector<uint64_t>*>(
+                            nullable_column->get_nested_column_ptr().get());
+                } else {
+                    column_data = assert_cast<const vectorized::ColumnVector<uint64_t>*>(
+                            _typed_column.column.get());
+                }
+
+                assert(column_data);
+                values_ = (const uint64_t*)(column_data->get_data().data()) + _row_pos;
+                return Status::OK();
+            }
+        }
+
+    private:
+        const uint64_t* values_ = nullptr;
+        bool from_datetime_to_datetime_v2_;
+    };
+
     // decimalv3 don't need to do any convert
     template <typename T>
     class OlapColumnDataConvertorDecimalV3
diff --git a/be/src/vec/runtime/vdatetime_value.cpp b/be/src/vec/runtime/vdatetime_value.cpp
index dd0d009ee5..1b3bcfdb37 100644
--- a/be/src/vec/runtime/vdatetime_value.cpp
+++ b/be/src/vec/runtime/vdatetime_value.cpp
@@ -23,6 +23,7 @@
 
 #include <limits>
 #include <sstream>
+#include <valarray>
 
 #include "runtime/datetime_value.h"
 #include "util/timezone_utils.h"
@@ -915,7 +916,8 @@ uint32_t VecDateTimeValue::year_week(uint8_t mode) const {
     return year * 100 + week;
 }
 
-bool VecDateTimeValue::operator>=(const DateV2Value& other) const {
+template <typename T>
+bool VecDateTimeValue::operator>=(const DateV2Value<T>& other) const {
     int64_t ts1;
     int64_t ts2;
     this->unix_timestamp(&ts1, TimezoneUtils::default_time_zone);
@@ -923,7 +925,8 @@ bool VecDateTimeValue::operator>=(const DateV2Value& other) const {
     return ts1 >= ts2;
 }
 
-bool VecDateTimeValue::operator<=(const DateV2Value& other) const {
+template <typename T>
+bool VecDateTimeValue::operator<=(const DateV2Value<T>& other) const {
     int64_t ts1;
     int64_t ts2;
     this->unix_timestamp(&ts1, TimezoneUtils::default_time_zone);
@@ -931,7 +934,8 @@ bool VecDateTimeValue::operator<=(const DateV2Value& other) const {
     return ts1 <= ts2;
 }
 
-bool VecDateTimeValue::operator>(const DateV2Value& other) const {
+template <typename T>
+bool VecDateTimeValue::operator>(const DateV2Value<T>& other) const {
     int64_t ts1;
     int64_t ts2;
     this->unix_timestamp(&ts1, TimezoneUtils::default_time_zone);
@@ -939,7 +943,8 @@ bool VecDateTimeValue::operator>(const DateV2Value& other) const {
     return ts1 > ts2;
 }
 
-bool VecDateTimeValue::operator<(const DateV2Value& other) const {
+template <typename T>
+bool VecDateTimeValue::operator<(const DateV2Value<T>& other) const {
     int64_t ts1;
     int64_t ts2;
     this->unix_timestamp(&ts1, TimezoneUtils::default_time_zone);
@@ -947,7 +952,8 @@ bool VecDateTimeValue::operator<(const DateV2Value& other) const {
     return ts1 < ts2;
 }
 
-bool VecDateTimeValue::operator==(const DateV2Value& other) const {
+template <typename T>
+bool VecDateTimeValue::operator==(const DateV2Value<T>& other) const {
     int64_t ts1;
     int64_t ts2;
     this->unix_timestamp(&ts1, TimezoneUtils::default_time_zone);
@@ -1451,9 +1457,11 @@ bool VecDateTimeValue::from_date_format_str(const char* format, int format_len,
     return check_range_and_set_time(year, month, day, hour, minute, second, _type);
 }
 
-int64_t VecDateTimeValue::second_diff(const DateV2Value& rhs) const {
+template <typename T>
+int64_t VecDateTimeValue::second_diff(const DateV2Value<T>& rhs) const {
     int day_diff = daynr() - rhs.daynr();
-    int time_diff = hour() * 3600 + minute() * 60 + second();
+    int time_diff = (hour() * 3600 + minute() * 60 + second()) -
+                    (rhs.hour() * 3600 + rhs.minute() * 60 + rhs.second());
     return day_diff * 3600 * 24 + time_diff;
 }
 
@@ -1612,8 +1620,14 @@ void VecDateTimeValue::set_time(uint32_t year, uint32_t month, uint32_t day, uin
     _second = second;
 }
 
-void VecDateTimeValue::create_from_date_v2(DateV2Value& value, TimeType type) {
-    this->set_time(value.year(), value.month(), value.day(), 0, 0, 0);
+template <typename T>
+void VecDateTimeValue::create_from_date_v2(DateV2Value<T>& value, TimeType type) {
+    if constexpr (std::is_same_v<T, DateV2ValueType>) {
+        this->set_time(value.year(), value.month(), value.day(), 0, 0, 0);
+    } else {
+        this->set_time(value.year(), value.month(), value.day(), value.hour(), value.minute(),
+                       value.second());
+    }
     this->set_type(type);
     this->_neg = 0;
 }
@@ -1655,11 +1669,13 @@ std::size_t operator-(const VecDateTimeValue& v1, const VecDateTimeValue& v2) {
     return v1.daynr() - v2.daynr();
 }
 
-std::size_t operator-(const DateV2Value& v1, const VecDateTimeValue& v2) {
+template <typename T>
+std::size_t operator-(const DateV2Value<T>& v1, const VecDateTimeValue& v2) {
     return v1.daynr() - v2.daynr();
 }
 
-std::size_t operator-(const VecDateTimeValue& v1, const DateV2Value& v2) {
+template <typename T>
+std::size_t operator-(const VecDateTimeValue& v1, const DateV2Value<T>& v2) {
     return v1.daynr() - v2.daynr();
 }
 
@@ -1667,7 +1683,9 @@ std::size_t hash_value(VecDateTimeValue const& value) {
     return HashUtil::hash(&value, sizeof(VecDateTimeValue), 0);
 }
 
-bool DateV2Value::is_invalid(uint32_t year, uint32_t month, uint32_t day) {
+template <typename T>
+bool DateV2Value<T>::is_invalid(uint32_t year, uint32_t month, uint32_t day, uint8_t hour,
+                                uint8_t minute, uint8_t second, uint32_t microsecond) {
     if (month == 2 && day == 29 && doris::is_leap(year)) return false;
     if (year < MIN_YEAR || year > MAX_YEAR || month == 0 || month > 12 ||
         day > s_days_in_month[month] || day == 0) {
@@ -1679,14 +1697,15 @@ bool DateV2Value::is_invalid(uint32_t year, uint32_t month, uint32_t day) {
 // The interval format is that with no delimiters
 // YYYY-MM-DD HH-MM-DD.FFFFFF AM in default format
 // 0    1  2  3  4  5  6      7
-bool DateV2Value::from_date_str(const char* date_str, int len) {
+template <typename T>
+bool DateV2Value<T>::from_date_str(const char* date_str, int len) {
     const char* ptr = date_str;
     const char* end = date_str + len;
     // ONLY 2, 6 can follow by a sapce
     const static int allow_space_mask = 4 | 64;
-    const static int MAX_DATE_PARTS = 3;
-    uint32_t date_val[MAX_DATE_PARTS];
-    int32_t date_len[MAX_DATE_PARTS] = {0, 0, 0};
+    const static int MAX_DATE_PARTS = 7;
+    uint32_t date_val[MAX_DATE_PARTS] = {0};
+    int32_t date_len[MAX_DATE_PARTS] = {0};
 
     // Skip space character
     while (ptr < end && isspace(*ptr)) {
@@ -1717,13 +1736,17 @@ bool DateV2Value::from_date_str(const char* date_str, int len) {
 
     int field_idx = 0;
     int field_len = year_len;
-    while (ptr < end && isdigit(*ptr) && field_idx < MAX_DATE_PARTS) {
+    while (ptr < end && isdigit(*ptr) && field_idx <= MAX_DATE_PARTS) {
         const char* start = ptr;
         int temp_val = 0;
         bool scan_to_delim = (!is_interval_format) && (field_idx != 6);
         while (ptr < end && isdigit(*ptr) && (scan_to_delim || field_len--)) {
             temp_val = temp_val * 10 + (*ptr++ - '0');
         }
+        if (field_idx == 6) {
+            // Microsecond
+            temp_val *= std::pow(10, 6 - (end - start));
+        }
         // Imposible
         if (temp_val > 999999L) {
             return false;
@@ -1767,11 +1790,13 @@ bool DateV2Value::from_date_str(const char* date_str, int len) {
         field_idx++;
     }
     int num_field = field_idx;
+    if constexpr (!is_datetime) {
+        DCHECK(num_field == 3);
+    }
     if (!is_interval_format) {
         year_len = date_len[0];
     }
     for (; field_idx < MAX_DATE_PARTS; ++field_idx) {
-        date_len[field_idx] = 0;
         date_val[field_idx] = 0;
     }
 
@@ -1783,27 +1808,31 @@ bool DateV2Value::from_date_str(const char* date_str, int len) {
         }
     }
 
-    if (num_field > 3) {
-        return false;
-    }
-    return check_range_and_set_time(date_val[0], date_val[1], date_val[2]);
+    if (num_field < 3) return false;
+    return check_range_and_set_time(date_val[0], date_val[1], date_val[2], date_val[3], date_val[4],
+                                    date_val[5], date_val[6]);
 }
 
-void DateV2Value::set_zero(int type) {
-    date_v2_value_.month_ = 0;
-    date_v2_value_.year_ = 0;
-    date_v2_value_.day_ = 0;
+template <typename T>
+void DateV2Value<T>::set_zero() {
+    int_val_ = 0;
 }
 
 // this method is exactly same as fromDateFormatStr() in DateLiteral.java in FE
 // change this method should also change that.
-bool DateV2Value::from_date_format_str(const char* format, int format_len, const char* value,
-                                       int value_len, const char** sub_val_end) {
+template <typename T>
+bool DateV2Value<T>::from_date_format_str(const char* format, int format_len, const char* value,
+                                          int value_len, const char** sub_val_end) {
     const char* ptr = format;
     const char* end = format + format_len;
     const char* val = value;
     const char* val_end = value + value_len;
 
+    bool date_part_used = false;
+    bool time_part_used = false;
+    bool frac_part_used = false;
+    bool already_set_time_part = false;
+
     int day_part = 0;
     int weekday = -1;
     int yearday = -1;
@@ -1815,8 +1844,7 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
     int strict_week_number_year = -1;
     bool usa_time = false;
 
-    // TODO: should we process [hour minute second] for datev2 type here?
-    auto [year, month, day, hour, minute, second] = std::tuple {0, 0, 0, 0, 0, 0};
+    auto [year, month, day, hour, minute, second, microsecond] = std::tuple {0, 0, 0, 0, 0, 0, 0};
     while (ptr < end && val < val_end) {
         // Skip space character
         while (val < val_end && isspace(*val)) {
@@ -1841,6 +1869,7 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
                 int_value += int_value >= 70 ? 1900 : 2000;
                 year = int_value;
                 val = tmp;
+                date_part_used = true;
                 break;
             case 'Y':
                 // Year, numeric, four digits
@@ -1853,6 +1882,7 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
                 }
                 year = int_value;
                 val = tmp;
+                date_part_used = true;
                 break;
                 // Month
             case 'm':
@@ -1863,6 +1893,7 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
                 }
                 month = int_value;
                 val = tmp;
+                date_part_used = true;
                 break;
             case 'M':
                 int_value = check_word(const_cast<const char**>(s_month_name), val, val_end, &val);
@@ -1887,6 +1918,7 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
                 }
                 day = int_value;
                 val = tmp;
+                date_part_used = true;
                 break;
             case 'D':
                 tmp = val + min(2, val_end - val);
@@ -1895,6 +1927,7 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
                 }
                 day = int_value;
                 val = tmp + min(2, val_end - tmp);
+                date_part_used = true;
                 break;
                 // Hour
             case 'h':
@@ -1910,6 +1943,7 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
                 }
                 hour = int_value;
                 val = tmp;
+                time_part_used = true;
                 break;
                 // Minute
             case 'i':
@@ -1919,6 +1953,7 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
                 }
                 minute = int_value;
                 val = tmp;
+                time_part_used = true;
                 break;
                 // Second
             case 's':
@@ -1929,9 +1964,18 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
                 }
                 second = int_value;
                 val = tmp;
+                time_part_used = true;
                 break;
-                // Micro second
+            // Micro second
             case 'f':
+                tmp = val + min(6, val_end - val);
+                if (!str_to_int64(val, &tmp, &int_value)) {
+                    return false;
+                }
+                microsecond = int_value * std::pow(10, 6 - min(6, val_end - val));
+                val = tmp;
+                time_part_used = true;
+                frac_part_used = true;
                 break;
                 // AM/PM
             case 'p':
@@ -1942,6 +1986,7 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
                     // PM
                     day_part = 12;
                 }
+                time_part_used = true;
                 val += 2;
                 break;
                 // Weekday
@@ -1952,6 +1997,7 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
                 }
                 int_value++;
                 weekday = int_value;
+                date_part_used = true;
                 break;
             case 'a':
                 int_value = check_word(s_ab_day_name, val, val_end, &val);
@@ -1960,6 +2006,7 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
                 }
                 int_value++;
                 weekday = int_value;
+                date_part_used = true;
                 break;
             case 'w':
                 tmp = val + min(1, val_end - val);
@@ -1974,6 +2021,7 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
                 }
                 weekday = int_value;
                 val = tmp;
+                date_part_used = true;
                 break;
             case 'j':
                 tmp = val + min(3, val_end - val);
@@ -1982,6 +2030,7 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
                 }
                 yearday = int_value;
                 val = tmp;
+                date_part_used = true;
                 break;
             case 'u':
             case 'v':
@@ -1999,6 +2048,7 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
                     return false;
                 }
                 val = tmp;
+                date_part_used = true;
                 break;
                 // strict week number, must be used with %V or %v
             case 'x':
@@ -2010,17 +2060,22 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
                 }
                 strict_week_number_year = int_value;
                 val = tmp;
+                date_part_used = true;
                 break;
             case 'r':
                 if (!from_date_format_str("%I:%i:%S %p", 11, val, val_end - val, &tmp)) {
                     return false;
                 }
                 val = tmp;
+                time_part_used = true;
+                already_set_time_part = true;
                 break;
             case 'T':
                 if (!from_date_format_str("%H:%i:%S", 8, val, val_end - val, &tmp)) {
                     return false;
                 }
+                time_part_used = true;
+                already_set_time_part = true;
                 val = tmp;
                 break;
             case '.':
@@ -2072,9 +2127,11 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
             case 'l':
             case 'r':
             case 's':
+            case 'f':
             case 'S':
             case 'p':
             case 'T':
+                time_part_used = true;
                 break;
             default:
                 break;
@@ -2094,6 +2151,30 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
         *sub_val_end = val;
     }
 
+    // Compute timestamp type
+    if (frac_part_used) {
+        if constexpr (!is_datetime) {
+            LOG(WARNING) << "Microsecond is not allowed for date type!";
+            return false;
+        }
+        if (!date_part_used) {
+            LOG(WARNING) << "Time type is not supported yet!";
+            return false;
+        }
+    } else {
+        if (date_part_used) {
+            if (time_part_used) {
+                if constexpr (!is_datetime) {
+                    LOG(WARNING) << "Time part is not allowed for date type!";
+                    return false;
+                }
+            }
+        } else {
+            LOG(WARNING) << "Time type is not supported yet!";
+            return false;
+        }
+    }
+
     // Year day
     if (yearday > 0) {
         uint64_t days = doris::calc_daynr(year, 1, 1) + yearday - 1;
@@ -2123,10 +2204,29 @@ bool DateV2Value::from_date_format_str(const char* format, int format_len, const
             return false;
         }
     }
-    return check_range_and_set_time(year, month, day);
+    // 1. already_set_date_part means _year, _month, _day be set, so we only set time part
+    // 2. already_set_time_part means _hour, _minute, _second, _microsecond be set,
+    //    so we only neet to set date part
+    // 3. if both are true, means all part of date_time be set, no need check_range_and_set_time
+    bool already_set_date_part = yearday > 0 || (week_num >= 0 && weekday > 0);
+    if (already_set_date_part && already_set_time_part) return true;
+    if (already_set_date_part)
+        return check_range_and_set_time(date_v2_value_.year_, date_v2_value_.month_,
+                                        date_v2_value_.day_, hour, minute, second, microsecond);
+    if (already_set_time_part) {
+        if constexpr (is_datetime) {
+            return check_range_and_set_time(year, month, day, date_v2_value_.hour_,
+                                            date_v2_value_.minute_, date_v2_value_.second_,
+                                            microsecond);
+        } else {
+            return check_range_and_set_time(year, month, day, 0, 0, 0, 0);
+        }
+    }
+    return check_range_and_set_time(year, month, day, hour, minute, second, microsecond);
 }
 
-int32_t DateV2Value::to_buffer(char* buffer) const {
+template <typename T>
+int32_t DateV2Value<T>::to_buffer(char* buffer, int scale) const {
     char* start = buffer;
     uint32_t temp;
     // Year
@@ -2144,40 +2244,92 @@ int32_t DateV2Value::to_buffer(char* buffer) const {
     // Day
     *buffer++ = (char)('0' + (date_v2_value_.day_ / 10));
     *buffer++ = (char)('0' + (date_v2_value_.day_ % 10));
+    if constexpr (is_datetime) {
+        *buffer++ = ' ';
+        // Hour
+        temp = date_v2_value_.hour_;
+        if (temp >= 100) {
+            *buffer++ = (char)('0' + (temp / 100));
+            temp %= 100;
+        }
+        *buffer++ = (char)('0' + (temp / 10));
+        *buffer++ = (char)('0' + (temp % 10));
+        *buffer++ = ':';
+        // Minute
+        *buffer++ = (char)('0' + (date_v2_value_.minute_ / 10));
+        *buffer++ = (char)('0' + (date_v2_value_.minute_ % 10));
+        *buffer++ = ':';
+        /* Second */
+        *buffer++ = (char)('0' + (date_v2_value_.second_ / 10));
+        *buffer++ = (char)('0' + (date_v2_value_.second_ % 10));
+        if (scale != 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);
+        }
+    }
     return buffer - start;
 }
 
-char* DateV2Value::to_string(char* to) const {
-    int len = to_buffer(to);
+template <typename T>
+char* DateV2Value<T>::to_string(char* to, int scale) const {
+    int len = to_buffer(to, scale);
     *(to + len) = '\0';
     return to + len + 1;
 }
 
-uint32_t DateV2Value::to_date_uint32() const {
+template <typename T>
+typename DateV2Value<T>::underlying_value DateV2Value<T>::to_date_int_val() const {
     return int_val_;
 }
 
-uint32_t DateV2Value::set_date_uint32(uint32_t int_val) {
+template <typename T>
+uint32_t DateV2Value<T>::set_date_uint32(uint32_t int_val) {
     union DateV2UInt32Union {
-        doris::vectorized::DateV2Value dt;
+        doris::vectorized::DateV2Value<T> dt;
         uint32_t ui32;
         ~DateV2UInt32Union() {}
     };
     DateV2UInt32Union conv = {.ui32 = int_val};
-    if (is_invalid(conv.dt.year(), conv.dt.month(), conv.dt.day())) {
+    if (is_invalid(conv.dt.year(), conv.dt.month(), conv.dt.day(), 0, 0, 0, 0)) {
         return 0;
     }
-    this->set_time(conv.dt.year(), conv.dt.month(), conv.dt.day());
+    this->set_time(conv.dt.year(), conv.dt.month(), conv.dt.day(), 0, 0, 0, 0);
 
     return int_val;
 }
 
-uint8_t DateV2Value::week(uint8_t mode) const {
+template <typename T>
+uint64_t DateV2Value<T>::set_datetime_uint64(uint64_t int_val) {
+    union DateTimeV2UInt64Union {
+        doris::vectorized::DateV2Value<T> dt;
+        uint64_t ui64;
+        ~DateTimeV2UInt64Union() {}
+    };
+    DateTimeV2UInt64Union conv = {.ui64 = int_val};
+    if (is_invalid(conv.dt.year(), conv.dt.month(), conv.dt.day(), conv.dt.hour(), conv.dt.minute(),
+                   conv.dt.second(), conv.dt.microsecond())) {
+        return 0;
+    }
+    this->set_time(conv.dt.year(), conv.dt.month(), conv.dt.day(), conv.dt.hour(), conv.dt.minute(),
+                   conv.dt.second(), conv.dt.microsecond());
+
+    return int_val;
+}
+
+template <typename T>
+uint8_t DateV2Value<T>::week(uint8_t mode) const {
     uint16_t year = 0;
     return calc_week(this->daynr(), this->year(), this->month(), this->day(), mode, &year);
 }
 
-uint32_t DateV2Value::year_week(uint8_t mode) const {
+template <typename T>
+uint32_t DateV2Value<T>::year_week(uint8_t mode) const {
     uint16_t year = 0;
     // The range of the week in the year_week is 1-53, so the mode WEEK_YEAR is always true.
     uint8_t week = calc_week(this->daynr(), this->year(), this->month(), this->day(), mode, &year);
@@ -2196,7 +2348,8 @@ uint32_t DateV2Value::year_week(uint8_t mode) const {
     return year * 100 + week;
 }
 
-bool DateV2Value::get_date_from_daynr(uint64_t daynr) {
+template <typename T>
+bool DateV2Value<T>::get_date_from_daynr(uint64_t daynr) {
     if (daynr <= 0 || daynr > DATE_MAX_DAYNR) {
         return false;
     }
@@ -2224,14 +2377,17 @@ bool DateV2Value::get_date_from_daynr(uint64_t daynr) {
     }
     day = days_of_year + leap_day;
 
-    if (is_invalid(year, month, day)) {
+    if (is_invalid(year, month, day, 0, 0, 0, 0)) {
         return false;
     }
-    set_time(year, month, day);
+    set_time(year, month, day, 0, 0, 0, 0);
     return true;
 }
 
-bool DateV2Value::date_add_interval(const TimeInterval& interval, TimeUnit unit) {
+template <typename T>
+template <typename TO>
+bool DateV2Value<T>::date_add_interval(const TimeInterval& interval, TimeUnit unit,
+                                       DateV2Value<TO>& to_value) {
     if (!is_valid_date()) return false;
 
     switch (unit) {
@@ -2247,25 +2403,51 @@ bool DateV2Value::date_add_interval(const TimeInterval& interval, TimeUnit unit)
     case DAY_MICROSECOND:
     case DAY_SECOND:
     case DAY_MINUTE:
-    case DAY_HOUR:
+    case DAY_HOUR: {
+        if constexpr (DateV2Value<TO>::is_datetime) {
+            // This may change the day information
+            int64_t seconds = (this->day() - 1) * 86400L + this->hour() * 3600L +
+                              this->minute() * 60 + this->second() + interval.day * 86400 +
+                              interval.hour * 3600 + interval.minute * 60 + interval.second;
+            uint64_t microsecond = interval.microsecond + this->microsecond();
+            if (microsecond > 1000000) {
+                seconds += 1;
+                microsecond %= 1000000;
+            }
+            int64_t days = seconds / 86400;
+            seconds %= 86400L;
+            if (seconds < 0) {
+                seconds += 86400L;
+                days--;
+            }
+            to_value.set_time(seconds / 3600, (seconds / 60) % 60, seconds % 60, microsecond);
+            int64_t day_nr = doris::calc_daynr(this->year(), this->month(), 1) + days;
+            if (!to_value.get_date_from_daynr(day_nr)) {
+                return false;
+            }
+            break;
+        } else {
+            LOG(FATAL) << "Invalid interval to add " << int(unit) << " for date!";
+        }
+    }
     case DAY:
     case WEEK: {
         // This only change day information, not change second information
         uint32_t day_nr = daynr() + interval.day;
-        if (!get_date_from_daynr(day_nr)) {
+        if (!to_value.get_date_from_daynr(day_nr)) {
             return false;
         }
         break;
     }
     case YEAR: {
         // This only change year information
-        date_v2_value_.year_ += interval.year;
-        if (date_v2_value_.year_ > 9999) {
+        to_value.template set_time_unit<TimeUnit::YEAR>(date_v2_value_.year_ + interval.year);
+        if (to_value.year() > 9999) {
             return false;
         }
         if (date_v2_value_.month_ == 2 && date_v2_value_.day_ == 29 &&
-            !doris::is_leap(date_v2_value_.year_)) {
-            date_v2_value_.day_ = 28;
+            !doris::is_leap(to_value.year())) {
+            to_value.template set_time_unit<TimeUnit::DAY>(28);
         }
         break;
     }
@@ -2275,15 +2457,15 @@ bool DateV2Value::date_add_interval(const TimeInterval& interval, TimeUnit unit)
         // This will change month and year information, maybe date.
         int64_t months = date_v2_value_.year_ * 12 + date_v2_value_.month_ - 1 +
                          12 * interval.year + interval.month;
-        date_v2_value_.year_ = months / 12;
-        if (date_v2_value_.year_ > 9999) {
+        to_value.template set_time_unit<TimeUnit::YEAR>(months / 12);
+        if (to_value.year() > 9999) {
             return false;
         }
-        date_v2_value_.month_ = (months % 12) + 1;
-        if (date_v2_value_.day_ > s_days_in_month[date_v2_value_.month_]) {
-            date_v2_value_.day_ = s_days_in_month[date_v2_value_.month_];
-            if (date_v2_value_.month_ == 2 && doris::is_leap(date_v2_value_.year_)) {
-                date_v2_value_.day_++;
+        to_value.template set_time_unit<TimeUnit::MONTH>((months % 12) + 1);
+        if (date_v2_value_.day_ > s_days_in_month[to_value.month()]) {
+            date_v2_value_.day_ = s_days_in_month[to_value.month()];
+            if (to_value.month() == 2 && doris::is_leap(to_value.year())) {
+                to_value.template set_time_unit<TimeUnit::DAY>(date_v2_value_.day_ + 1);
             }
         }
         break;
@@ -2292,7 +2474,8 @@ bool DateV2Value::date_add_interval(const TimeInterval& interval, TimeUnit unit)
     return true;
 }
 
-bool DateV2Value::unix_timestamp(int64_t* timestamp, const std::string& timezone) const {
+template <typename T>
+bool DateV2Value<T>::unix_timestamp(int64_t* timestamp, const std::string& timezone) const {
     cctz::time_zone ctz;
     if (!TimezoneUtils::find_cctz_time_zone(timezone, ctz)) {
         return false;
@@ -2300,15 +2483,28 @@ bool DateV2Value::unix_timestamp(int64_t* timestamp, const std::string& timezone
     return unix_timestamp(timestamp, ctz);
 }
 
-bool DateV2Value::unix_timestamp(int64_t* timestamp, const cctz::time_zone& ctz) const {
-    const auto tp = cctz::convert(cctz::civil_second(date_v2_value_.year_, date_v2_value_.month_,
-                                                     date_v2_value_.day_, 0, 0, 0),
-                                  ctz);
-    *timestamp = tp.time_since_epoch().count();
-    return true;
+template <typename T>
+bool DateV2Value<T>::unix_timestamp(int64_t* timestamp, const cctz::time_zone& ctz) const {
+    if constexpr (is_datetime) {
+        const auto tp =
+                cctz::convert(cctz::civil_second(date_v2_value_.year_, date_v2_value_.month_,
+                                                 date_v2_value_.day_, date_v2_value_.hour_,
+                                                 date_v2_value_.minute_, date_v2_value_.second_),
+                              ctz);
+        *timestamp = tp.time_since_epoch().count();
+        return true;
+    } else {
+        const auto tp =
+                cctz::convert(cctz::civil_second(date_v2_value_.year_, date_v2_value_.month_,
+                                                 date_v2_value_.day_, 0, 0, 0),
+                              ctz);
+        *timestamp = tp.time_since_epoch().count();
+        return true;
+    }
 }
 
-bool DateV2Value::from_unixtime(int64_t timestamp, const std::string& timezone) {
+template <typename T>
+bool DateV2Value<T>::from_unixtime(int64_t timestamp, const std::string& timezone) {
     cctz::time_zone ctz;
     if (!TimezoneUtils::find_cctz_time_zone(timezone, ctz)) {
         return false;
@@ -2316,7 +2512,9 @@ bool DateV2Value::from_unixtime(int64_t timestamp, const std::string& timezone)
     return from_unixtime(timestamp, ctz);
 }
 
-bool DateV2Value::from_unixtime(int64_t timestamp, const cctz::time_zone& ctz) {
+template <typename T>
+bool DateV2Value<T>::from_unixtime(int64_t timestamp, const cctz::time_zone& ctz) {
+    DCHECK(is_datetime);
     static const cctz::time_point<cctz::sys_seconds> epoch =
             std::chrono::time_point_cast<cctz::sys_seconds>(
                     std::chrono::system_clock::from_time_t(0));
@@ -2324,18 +2522,20 @@ bool DateV2Value::from_unixtime(int64_t timestamp, const cctz::time_zone& ctz) {
 
     const auto tp = cctz::convert(t, ctz);
 
-    set_time(tp.year(), tp.month(), tp.day());
+    set_time(tp.year(), tp.month(), tp.day(), tp.hour(), tp.minute(), tp.second(), 0);
     return true;
 }
 
-const char* DateV2Value::month_name() const {
+template <typename T>
+const char* DateV2Value<T>::month_name() const {
     if (date_v2_value_.month_ < 1 || date_v2_value_.month_ > 12) {
         return nullptr;
     }
     return s_month_name[date_v2_value_.month_];
 }
 
-const char* DateV2Value::day_name() const {
+template <typename T>
+const char* DateV2Value<T>::day_name() const {
     int day = weekday();
     if (day < 0 || day >= 7) {
         return nullptr;
@@ -2343,32 +2543,66 @@ const char* DateV2Value::day_name() const {
     return s_day_name[day];
 }
 
-void DateV2Value::set_time(uint16_t year, uint8_t month, uint8_t day) {
+template <typename T>
+void DateV2Value<T>::set_time(uint16_t year, uint8_t month, uint8_t day, uint8_t hour,
+                              uint8_t minute, uint8_t second, uint32_t microsecond) {
     date_v2_value_.year_ = year;
     date_v2_value_.month_ = month;
     date_v2_value_.day_ = day;
+    if constexpr (is_datetime) {
+        date_v2_value_.hour_ = hour;
+        date_v2_value_.minute_ = minute;
+        date_v2_value_.second_ = second;
+        date_v2_value_.microsecond_ = microsecond;
+    }
 }
 
-void DateV2Value::convert_date_v2_to_dt(
-        doris::DateTimeValue* dt) { //use convert VecDateTimeValue to DateTimeValue
+template <typename T>
+void DateV2Value<T>::set_time(uint8_t hour, uint8_t minute, uint8_t second, uint32_t microsecond) {
+    if constexpr (is_datetime) {
+        date_v2_value_.hour_ = hour;
+        date_v2_value_.minute_ = minute;
+        date_v2_value_.second_ = second;
+        date_v2_value_.microsecond_ = microsecond;
+    } else {
+        LOG(FATAL) << "Invalid operation 'set_time' for date!";
+    }
+}
+
+template <typename T>
+void DateV2Value<T>::convert_date_v2_to_dt(doris::DateTimeValue* dt) {
+    if constexpr (is_datetime) {
+        dt->_type = TIME_DATETIME;
+        dt->_hour = this->hour();
+        dt->_minute = this->minute();
+        dt->_second = this->second();
+    } else {
+        dt->_type = TIME_DATE;
+        dt->_hour = 0;
+        dt->_minute = 0;
+        dt->_second = 0;
+    }
     dt->_neg = 0;
-    dt->_type = TIME_DATE;
-    dt->_hour = 0;
-    dt->_minute = 0;
-    dt->_second = 0;
     dt->_year = date_v2_value_.year_;
     dt->_month = date_v2_value_.month_;
     dt->_day = date_v2_value_.day_;
     dt->_microsecond = 0;
 }
 
-void DateV2Value::convert_dt_to_date_v2(doris::DateTimeValue* dt) {
+template <typename T>
+void DateV2Value<T>::convert_dt_to_date_v2(doris::DateTimeValue* dt) {
     date_v2_value_.year_ = dt->_year;
     date_v2_value_.month_ = dt->_month;
     date_v2_value_.day_ = dt->_day;
+    if constexpr (is_datetime) {
+        date_v2_value_.hour_ = dt->hour();
+        date_v2_value_.minute_ = dt->minute();
+        date_v2_value_.second_ = dt->second();
+    }
 }
 
-bool DateV2Value::to_format_string(const char* format, int len, char* to) const {
+template <typename T>
+bool DateV2Value<T>::to_format_string(const char* format, int len, char* to) const {
     char buf[64];
     char* pos = nullptr;
     const char* ptr = format;
@@ -2437,7 +2671,7 @@ bool DateV2Value::to_format_string(const char* format, int len, char* to) const
             break;
         case 'f':
             // Microseconds (000000..999999)
-            pos = int_to_str(0, buf);
+            pos = int_to_str(this->microsecond(), buf);
             to = append_with_prefix(buf, pos - buf, '0', 6, to);
             break;
         case 'h':
@@ -2448,12 +2682,12 @@ bool DateV2Value::to_format_string(const char* format, int len, char* to) const
             break;
         case 'H':
             // Hour (00..23)
-            pos = int_to_str(0, buf);
+            pos = int_to_str(this->hour(), buf);
             to = append_with_prefix(buf, pos - buf, '0', 2, to);
             break;
         case 'i':
             // Minutes, numeric (00..59)
-            pos = int_to_str(0, buf);
+            pos = int_to_str(this->minute(), buf);
             to = append_with_prefix(buf, pos - buf, '0', 2, to);
             break;
         case 'j':
@@ -2463,12 +2697,12 @@ bool DateV2Value::to_format_string(const char* format, int len, char* to) const
             break;
         case 'k':
             // Hour (0..23)
-            pos = int_to_str(0, buf);
+            pos = int_to_str(this->hour(), buf);
             to = append_with_prefix(buf, pos - buf, '0', 1, to);
             break;
         case 'l':
             // Hour (1..12)
-            pos = int_to_str(12, buf);
+            pos = int_to_str((this->hour() % 12) + 1, buf);
             to = append_with_prefix(buf, pos - buf, '0', 1, to);
             break;
         case 'm':
@@ -2497,7 +2731,7 @@ bool DateV2Value::to_format_string(const char* format, int len, char* to) const
         case 's':
         case 'S':
             // Seconds (00..59)
-            pos = int_to_str(0, buf);
+            pos = int_to_str(this->second(), buf);
             to = append_with_prefix(buf, pos - buf, '0', 2, to);
             break;
         case 'T': {
@@ -2582,7 +2816,9 @@ bool DateV2Value::to_format_string(const char* format, int len, char* to) const
     return true;
 }
 
-bool DateV2Value::from_date(uint32_t value) {
+template <typename T>
+bool DateV2Value<T>::from_date(uint32_t value) {
+    DCHECK(!is_datetime);
     if (value < MIN_DATE_V2 || value > MAX_DATE_V2) {
         return false;
     }
@@ -2590,7 +2826,18 @@ bool DateV2Value::from_date(uint32_t value) {
     return set_date_uint32(value);
 }
 
-int64_t DateV2Value::standardize_timevalue(int64_t value) {
+template <typename T>
+bool DateV2Value<T>::from_datetime(uint64_t value) {
+    DCHECK(is_datetime);
+    if (value < MIN_DATETIME_V2 || value > MAX_DATETIME_V2) {
+        return false;
+    }
+
+    return set_datetime_uint64(value);
+}
+
+template <typename T>
+int64_t DateV2Value<T>::standardize_timevalue(int64_t value) {
     if (value <= 0) {
         return 0;
     }
@@ -2620,7 +2867,6 @@ int64_t DateV2Value::standardize_timevalue(int64_t value) {
     if (value <= 991231L) {
         return (value + 19000000L) * 1000000L;
     }
-    // TODO(zhaochun): Don't allow year betwen 1000-01-01
     if (value < 10000101) {
         return 0;
     }
@@ -2648,24 +2894,36 @@ int64_t DateV2Value::standardize_timevalue(int64_t value) {
     return value;
 }
 
-bool DateV2Value::from_date_int64(int64_t value) {
+template <typename T>
+bool DateV2Value<T>::from_date_int64(int64_t value) {
     value = standardize_timevalue(value);
     if (value <= 0) {
         return false;
     }
     uint64_t date = value / 1000000;
 
-    auto [year, month, day] = std::tuple {0, 0, 0};
+    auto [year, month, day, hour, minute, second] = std::tuple {0, 0, 0, 0, 0, 0};
     year = date / 10000;
     date %= 10000;
     month = date / 100;
     day = date % 100;
 
-    return check_range_and_set_time(year, month, day);
+    if constexpr (is_datetime) {
+        uint64_t time = value % 1000000;
+        hour = time / 10000;
+        time %= 10000;
+        minute = time / 100;
+        second = time % 100;
+        return check_range_and_set_time(year, month, day, hour, minute, second, 0);
+    } else {
+        return check_range_and_set_time(year, month, day, 0, 0, 0, 0);
+    }
 }
 
-uint8_t DateV2Value::calc_week(const uint32_t& day_nr, const uint16_t& year, const uint8_t& month,
-                               const uint8_t& day, uint8_t mode, uint16_t* to_year) {
+template <typename T>
+uint8_t DateV2Value<T>::calc_week(const uint32_t& day_nr, const uint16_t& year,
+                                  const uint8_t& month, const uint8_t& day, uint8_t mode,
+                                  uint16_t* to_year) {
     bool monday_first = mode & WEEK_MONDAY_FIRST;
     bool week_year = mode & WEEK_YEAR;
     bool first_weekday = mode & WEEK_FIRST_WEEKDAY;
@@ -2709,19 +2967,76 @@ uint8_t DateV2Value::calc_week(const uint32_t& day_nr, const uint16_t& year, con
     return days / 7 + 1;
 }
 
-std::ostream& operator<<(std::ostream& os, const DateV2Value& value) {
-    char buf[11];
+template <typename T>
+std::ostream& operator<<(std::ostream& os, const DateV2Value<T>& value) {
+    char buf[30];
     value.to_string(buf);
     return os << buf;
 }
 
 // NOTE:
 //  only support DATE - DATE (no support DATETIME - DATETIME)
-std::size_t operator-(const DateV2Value& v1, const DateV2Value& v2) {
+template <typename T0, typename T1>
+std::size_t operator-(const DateV2Value<T0>& v1, const DateV2Value<T1>& v2) {
     return v1.daynr() - v2.daynr();
 }
 
-std::size_t hash_value(DateV2Value const& value) {
-    return HashUtil::hash(&value, sizeof(DateV2Value), 0);
-}
+template <typename T>
+std::size_t hash_value(DateV2Value<T> const& value) {
+    return HashUtil::hash(&value, sizeof(DateV2Value<T>), 0);
+}
+
+template class DateV2Value<DateV2ValueType>;
+template class DateV2Value<DateTimeV2ValueType>;
+
+template std::size_t hash_value<DateV2ValueType>(DateV2Value<DateV2ValueType> const& value);
+template std::size_t hash_value<DateTimeV2ValueType>(DateV2Value<DateTimeV2ValueType> const& value);
+
+template std::ostream& operator<<(std::ostream& os, const DateV2Value<DateV2ValueType>& value);
+template std::ostream& operator<<(std::ostream& os, const DateV2Value<DateTimeV2ValueType>& value);
+
+template std::size_t operator-(const VecDateTimeValue& v1, const DateV2Value<DateV2ValueType>& v2);
+template std::size_t operator-(const VecDateTimeValue& v1,
+                               const DateV2Value<DateTimeV2ValueType>& v2);
+
+template std::size_t operator-(const DateV2Value<DateV2ValueType>& v1, const VecDateTimeValue& v2);
+template std::size_t operator-(const DateV2Value<DateTimeV2ValueType>& v1,
+                               const VecDateTimeValue& v2);
+
+template std::size_t operator-(const DateV2Value<DateV2ValueType>& v1,
+                               const DateV2Value<DateV2ValueType>& v2);
+template std::size_t operator-(const DateV2Value<DateV2ValueType>& v1,
+                               const DateV2Value<DateTimeV2ValueType>& v2);
+template std::size_t operator-(const DateV2Value<DateTimeV2ValueType>& v1,
+                               const DateV2Value<DateV2ValueType>& v2);
+template std::size_t operator-(const DateV2Value<DateTimeV2ValueType>& v1,
+                               const DateV2Value<DateTimeV2ValueType>& v2);
+
+template void VecDateTimeValue::create_from_date_v2<DateV2ValueType>(
+        DateV2Value<DateV2ValueType>& value, TimeType type);
+template void VecDateTimeValue::create_from_date_v2<DateTimeV2ValueType>(
+        DateV2Value<DateTimeV2ValueType>& value, TimeType type);
+
+template int64_t VecDateTimeValue::second_diff<DateV2ValueType>(
+        const DateV2Value<DateV2ValueType>& rhs) const;
+template int64_t VecDateTimeValue::second_diff<DateTimeV2ValueType>(
+        const DateV2Value<DateTimeV2ValueType>& rhs) const;
+
+template bool doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>::date_add_interval<
+        doris::vectorized::DateTimeV2ValueType>(
+        doris::vectorized::TimeInterval const&, doris::vectorized::TimeUnit,
+        doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>&);
+template bool doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>::date_add_interval<
+        doris::vectorized::DateV2ValueType>(
+        doris::vectorized::TimeInterval const&, doris::vectorized::TimeUnit,
+        doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>&);
+
+template bool doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>::
+        date_add_interval<doris::vectorized::DateTimeV2ValueType>(
+                doris::vectorized::TimeInterval const&, doris::vectorized::TimeUnit,
+                doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>&);
+template bool doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>::
+        date_add_interval<doris::vectorized::DateV2ValueType>(
+                doris::vectorized::TimeInterval const&, doris::vectorized::TimeUnit,
+                doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>&);
 } // namespace doris::vectorized
diff --git a/be/src/vec/runtime/vdatetime_value.h b/be/src/vec/runtime/vdatetime_value.h
index a4ca92d4a2..83a37803b5 100644
--- a/be/src/vec/runtime/vdatetime_value.h
+++ b/be/src/vec/runtime/vdatetime_value.h
@@ -67,12 +67,28 @@ struct TimeInterval {
     int64_t hour;
     int64_t minute;
     int64_t second;
+    int64_t microsecond;
     bool is_neg;
 
-    TimeInterval() : year(0), month(0), day(0), hour(0), minute(0), second(0), is_neg(false) {}
+    TimeInterval()
+            : year(0),
+              month(0),
+              day(0),
+              hour(0),
+              minute(0),
+              second(0),
+              microsecond(0),
+              is_neg(false) {}
 
     TimeInterval(TimeUnit unit, int64_t count, bool is_neg_param)
-            : year(0), month(0), day(0), hour(0), minute(0), second(0), is_neg(is_neg_param) {
+            : year(0),
+              month(0),
+              day(0),
+              hour(0),
+              minute(0),
+              second(0),
+              microsecond(0),
+              is_neg(is_neg_param) {
         switch (unit) {
         case YEAR:
             year = count;
@@ -95,6 +111,9 @@ struct TimeInterval {
         case SECOND:
             second = count;
             break;
+        case SECOND_MICROSECOND:
+            microsecond = count;
+            break;
         default:
             break;
         }
@@ -144,16 +163,57 @@ static constexpr const char* s_day_name[] = {"Monday", "Tuesday",  "Wednesday",
 static constexpr size_t MAX_DAY_NAME_LEN = max_char_length(s_day_name, std::size(s_day_name));
 static constexpr size_t MAX_MONTH_NAME_LEN = max_char_length(s_month_name, std::size(s_month_name));
 
-const uint32_t MAX_DATE_V2 = 31 | (12 << 5) | (9999 << 9);
-const uint32_t MIN_DATE_V2 = 1 | (1 << 5) | (1000 << 9);
+static constexpr uint8_t TIME_PART_LENGTH = 37;
+
+static constexpr uint32_t MAX_DATE_V2 = 31 | (12 << 5) | (9999 << 9);
+static constexpr uint32_t MIN_DATE_V2 = 1 | (1 << 5) | (1000 << 9);
+
+static constexpr uint64_t MAX_DATETIME_V2 = ((uint64_t)MAX_DATE_V2 << TIME_PART_LENGTH) |
+                                            ((uint64_t)23 << 32) | ((uint64_t)59 << 26) |
+                                            ((uint64_t)59 << 20) | 999999;
+static constexpr uint64_t MIN_DATETIME_V2 = (uint64_t)MIN_DATE_V2 << TIME_PART_LENGTH;
+
+static constexpr uint32_t MAX_YEAR = 9999;
+static constexpr uint32_t MIN_YEAR = 1000;
 
-const uint32_t MAX_YEAR = 9999;
-const uint32_t MIN_YEAR = 1000;
+static constexpr uint32_t DATEV2_YEAR_WIDTH = 23;
+static constexpr uint32_t DATETIMEV2_YEAR_WIDTH = 18;
 
 static RE2 time_zone_offset_format_reg("^[+-]{1}\\d{2}\\:\\d{2}$");
 
 uint8_t mysql_week_mode(uint32_t mode);
 
+struct DateV2ValueType {
+    uint32_t day_ : 5;
+    uint32_t month_ : 4;
+    uint32_t year_ : 23;
+
+    DateV2ValueType(uint16_t year, uint8_t month, uint8_t day, uint8_t hour, uint8_t minute,
+                    uint8_t second, uint32_t microsecond)
+            : day_(day), month_(month), year_(year) {}
+};
+
+struct DateTimeV2ValueType {
+    uint64_t microsecond_ : 20;
+    uint64_t second_ : 6;
+    uint64_t minute_ : 6;
+    uint64_t hour_ : 5;
+    uint64_t day_ : 5;
+    uint64_t month_ : 4;
+    uint64_t year_ : 18;
+
+    DateTimeV2ValueType(uint16_t year, uint8_t month, uint8_t day, uint8_t hour, uint8_t minute,
+                        uint8_t second, uint32_t microsecond)
+            : microsecond_(microsecond),
+              second_(second),
+              minute_(minute),
+              hour_(hour),
+              day_(day),
+              month_(month),
+              year_(year) {}
+};
+
+template <typename T>
 class DateV2Value;
 
 class VecDateTimeValue { // Now this type is a temp solution with little changes, maybe large refactoring follow-up.
@@ -191,7 +251,8 @@ public:
         return datetime;
     }
 
-    void create_from_date_v2(DateV2Value& value, TimeType type);
+    template <typename T>
+    void create_from_date_v2(DateV2Value<T>& value, TimeType type);
 
     void set_time(uint32_t year, uint32_t month, uint32_t day, uint32_t hour, uint32_t minute,
                   uint32_t second);
@@ -437,17 +498,25 @@ public:
         return v1 > v2;
     }
 
-    bool operator==(const DateV2Value& other) const;
+    template <typename T>
+    bool operator==(const DateV2Value<T>& other) const;
 
-    bool operator!=(const DateV2Value& other) const { return !(*this == other); };
+    template <typename T>
+    bool operator!=(const DateV2Value<T>& other) const {
+        return !(*this == other);
+    };
 
-    bool operator<=(const DateV2Value& other) const;
+    template <typename T>
+    bool operator<=(const DateV2Value<T>& other) const;
 
-    bool operator>=(const DateV2Value& other) const;
+    template <typename T>
+    bool operator>=(const DateV2Value<T>& other) const;
 
-    bool operator<(const DateV2Value& other) const;
+    template <typename T>
+    bool operator<(const DateV2Value<T>& other) const;
 
-    bool operator>(const DateV2Value& other) const;
+    template <typename T>
+    bool operator>(const DateV2Value<T>& other) const;
 
     const char* month_name() const;
 
@@ -481,7 +550,14 @@ public:
 
     uint32_t to_date_v2() const {
         CHECK(_type == TIME_DATE);
-        return (year() << 16 | month() << 8 | day());
+        return (year() << 9 | month() << 5 | day());
+    };
+
+    uint64_t to_datetime_v2() const {
+        CHECK(_type == TIME_DATETIME);
+        return (uint64_t)(((uint64_t)year() << 46) | ((uint64_t)month() << 42) |
+                          ((uint64_t)day() << 37) | ((uint64_t)hour() << 32) |
+                          ((uint64_t)minute() << 26) | ((uint64_t)second() << 20));
     };
 
     static VecDateTimeValue from_datetime_val(const doris_udf::DateTimeVal& tv) {
@@ -523,7 +599,8 @@ public:
         return day_diff * 3600 * 24 + time_diff;
     }
 
-    int64_t second_diff(const DateV2Value& rhs) const;
+    template <typename T>
+    int64_t second_diff(const DateV2Value<T>& rhs) const;
 
     int64_t time_part_diff(const VecDateTimeValue& rhs) const {
         int time_diff = (hour() * 3600 + minute() * 60 + second()) -
@@ -630,33 +707,37 @@ private:
               _year(year) {}
 };
 
-struct DateV2ValueType {
-    uint32_t day_ : 5;
-    uint32_t month_ : 4;
-    uint32_t year_ : 23;
-
-    DateV2ValueType(uint16_t year, uint8_t month, uint8_t day)
-            : day_(day), month_(month), year_(year) {}
-};
-
-class DateV2Value { // Now this type is a temp solution with little changes, maybe large refactoring follow-up.
+template <typename T>
+class DateV2Value {
 public:
+    static constexpr bool is_datetime = std::is_same_v<T, DateTimeV2ValueType>;
+    using underlying_value =
+            std::conditional_t<std::is_same_v<T, DateTimeV2ValueType>, uint64_t, uint32_t>;
+
     // Constructor
-    DateV2Value() : date_v2_value_(0, 0, 0) {}
+    DateV2Value<T>() : date_v2_value_(0, 0, 0, 0, 0, 0, 0) {}
 
-    DateV2Value(DateV2Value& other) { int_val_ = other.to_date_uint32(); }
+    DateV2Value<T>(DateV2Value<T>& other) { int_val_ = other.to_date_int_val(); }
 
-    DateV2Value(const DateV2Value& other) { int_val_ = other.to_date_uint32(); }
+    DateV2Value<T>(const DateV2Value<T>& other) { int_val_ = other.to_date_int_val(); }
 
-    static DateV2Value create_from_olap_date(uint64_t value) {
-        DateV2Value date;
+    static DateV2Value<T> create_from_olap_date(uint64_t value) {
+        DateV2Value<T> date;
         date.from_olap_date(value);
         return date;
     }
 
-    void set_time(uint16_t year, uint8_t month, uint8_t day);
+    static DateV2Value<T> create_from_olap_datetime(uint64_t value) {
+        DateV2Value<T> datetime;
+        datetime.from_olap_datetime(value);
+        return datetime;
+    }
+
+    void set_time(uint16_t year, uint8_t month, uint8_t day, uint8_t hour, uint8_t minute,
+                  uint8_t second, uint32_t microsecond);
+
+    void set_time(uint8_t hour, uint8_t minute, uint8_t second, uint32_t microsecond);
 
-    // TODO(gabriel): unify execution and storage
     bool from_olap_date(uint64_t date) {
         auto [year, month, day] = std::tuple {0, 0, 0};
 
@@ -666,7 +747,24 @@ public:
         date >>= 4;
         year = date;
 
-        return check_range_and_set_time(year, month, day);
+        return check_range_and_set_time(year, month, day, 0, 0, 0, 0);
+    }
+
+    bool from_olap_datetime(uint64_t datetime) {
+        uint64_t date = datetime / 1000000;
+        uint64_t time = datetime % 1000000;
+
+        auto [year, month, day, hour, minute, second] = std::tuple {0, 0, 0, 0, 0, 0};
+        year = date / 10000;
+        date %= 10000;
+        month = date / 100;
+        day = date % 100;
+        hour = time / 10000;
+        time %= 10000;
+        minute = time / 100;
+        second = time % 100;
+
+        return check_range_and_set_time(year, month, day, hour, minute, second, 0);
     }
 
     uint64_t to_olap_date() const {
@@ -679,6 +777,17 @@ public:
         return val;
     }
 
+    uint64_t to_olap_datetime() const {
+        uint64_t date_val =
+                date_v2_value_.year_ * 10000 + date_v2_value_.month_ * 100 + date_v2_value_.day_;
+        uint64_t time_val = 0;
+        if constexpr (is_datetime) {
+            time_val = date_v2_value_.hour_ * 10000 + date_v2_value_.minute_ * 100 +
+                       date_v2_value_.second_;
+        }
+        return date_val * 1000000 + time_val;
+    }
+
     bool to_format_string(const char* format, int len, char* to) const;
 
     bool from_date_format_str(const char* format, int format_len, const char* value,
@@ -698,18 +807,20 @@ public:
     // TIME:  format 'hh:mm:ss.xxxxxx'
     // DATE:  format 'YYYY-MM-DD'
     // DATETIME:  format 'YYYY-MM-DD hh:mm:ss.xxxxxx'
-    int32_t to_buffer(char* buffer) const;
+    int32_t to_buffer(char* buffer, int scale = -1) const;
 
-    char* to_string(char* to) const;
+    char* to_string(char* to, int scale = -1) const;
 
     // Return true if range or date is invalid
-    static bool is_invalid(uint32_t year, uint32_t month, uint32_t day);
+    static bool is_invalid(uint32_t year, uint32_t month, uint32_t day, uint8_t hour,
+                           uint8_t minute, uint8_t second, uint32_t microsecond);
 
-    bool check_range_and_set_time(uint16_t year, uint8_t month, uint8_t day) {
-        if (is_invalid(year, month, day)) {
+    bool check_range_and_set_time(uint16_t year, uint8_t month, uint8_t day, uint8_t hour,
+                                  uint8_t minute, uint8_t second, uint32_t microsecond) {
+        if (is_invalid(year, month, day, hour, minute, second, microsecond)) {
             return false;
         }
-        set_time(year, month, day);
+        set_time(year, month, day, hour, minute, second, microsecond);
         return true;
     };
 
@@ -717,9 +828,38 @@ public:
         return calc_daynr(date_v2_value_.year_, date_v2_value_.month_, date_v2_value_.day_);
     }
 
-    int hour() const { return 0; }
-    int minute() const { return 0; }
-    int second() const { return 0; }
+    int hour() const {
+        if constexpr (is_datetime) {
+            return date_v2_value_.hour_;
+        } else {
+            return 0;
+        }
+    }
+
+    int minute() const {
+        if constexpr (is_datetime) {
+            return date_v2_value_.minute_;
+        } else {
+            return 0;
+        }
+    }
+
+    int second() const {
+        if constexpr (is_datetime) {
+            return date_v2_value_.second_;
+        } else {
+            return 0;
+        }
+    }
+
+    int microsecond() const {
+        if constexpr (is_datetime) {
+            return date_v2_value_.microsecond_;
+        } else {
+            return 0;
+        }
+    }
+
     uint16_t year() const { return date_v2_value_.year_; }
     uint8_t month() const { return date_v2_value_.month_; }
     int quarter() const { return (date_v2_value_.month_ - 1) / 3 + 1; }
@@ -767,7 +907,12 @@ public:
     uint32_t year_week(uint8_t mode) const;
 
     // Add interval
-    bool date_add_interval(const TimeInterval& interval, TimeUnit unit);
+    template <typename TO>
+    bool date_add_interval(const TimeInterval& interval, TimeUnit unit, DateV2Value<TO>& to_value);
+
+    bool date_add_interval(const TimeInterval& interval, TimeUnit unit) {
+        return this->date_add_interval(interval, unit, *this);
+    }
 
     //unix_timestamp is called with a timezone argument,
     //it returns seconds of the value of date literal since '1970-01-01 00:00:00' UTC
@@ -779,11 +924,11 @@ public:
     bool from_unixtime(int64_t, const std::string& timezone);
     bool from_unixtime(int64_t, const cctz::time_zone& ctz);
 
-    bool operator==(const DateV2Value& other) const {
+    bool operator==(const DateV2Value<T>& other) const {
         // NOTE: This is not same with MySQL.
         // MySQL convert both to int with left value type and then compare
         // We think all fields equals.
-        return this->to_date_uint32() == other.to_date_uint32();
+        return this->to_date_int_val() == other.to_date_int_val();
     }
 
     bool operator==(const VecDateTimeValue& other) const {
@@ -794,13 +939,13 @@ public:
         return ts1 == ts2;
     }
 
-    bool operator!=(const DateV2Value& other) const {
-        return this->to_date_uint32() != other.to_date_uint32();
+    bool operator!=(const DateV2Value<T>& other) const {
+        return this->to_date_int_val() != other.to_date_int_val();
     }
 
     bool operator!=(const VecDateTimeValue& other) const { return !(*this == other); }
 
-    bool operator<=(const DateV2Value& other) const { return !(*this > other); }
+    bool operator<=(const DateV2Value<T>& other) const { return !(*this > other); }
 
     bool operator<=(const VecDateTimeValue& other) const {
         int64_t ts1;
@@ -810,7 +955,7 @@ public:
         return ts1 <= ts2;
     }
 
-    bool operator>=(const DateV2Value& other) const { return !(*this < other); }
+    bool operator>=(const DateV2Value<T>& other) const { return !(*this < other); }
 
     bool operator>=(const VecDateTimeValue& other) const {
         int64_t ts1;
@@ -820,8 +965,8 @@ public:
         return ts1 >= ts2;
     }
 
-    bool operator<(const DateV2Value& other) const {
-        return to_date_uint32() < other.to_date_uint32();
+    bool operator<(const DateV2Value<T>& other) const {
+        return this->to_date_int_val() < other.to_date_int_val();
     }
 
     bool operator<(const VecDateTimeValue& other) const {
@@ -832,8 +977,8 @@ public:
         return ts1 < ts2;
     }
 
-    bool operator>(const DateV2Value& other) const {
-        return to_date_uint32() > other.to_date_uint32();
+    bool operator>(const DateV2Value<T>& other) const {
+        return this->to_date_int_val() > other.to_date_int_val();
     }
 
     bool operator>(const VecDateTimeValue& other) const {
@@ -844,13 +989,13 @@ public:
         return ts1 > ts2;
     }
 
-    DateV2Value& operator=(const DateV2Value& other) {
-        int_val_ = other.to_date_uint32();
+    DateV2Value<T>& operator=(const DateV2Value<T>& other) {
+        int_val_ = other.to_date_int_val();
         return *this;
     }
 
-    DateV2Value& operator=(DateV2Value& other) {
-        int_val_ = other.to_date_uint32();
+    DateV2Value<T>& operator=(DateV2Value<T>& other) {
+        int_val_ = other.to_date_int_val();
         return *this;
     }
 
@@ -858,9 +1003,14 @@ public:
 
     const char* day_name() const;
 
-    DateV2Value& operator++() {
-        TimeInterval interval(DAY, 1, false);
-        date_add_interval(interval, DAY);
+    DateV2Value<T>& operator++() {
+        if constexpr (is_datetime) {
+            TimeInterval interval(SECOND, 1, false);
+            date_add_interval(interval, SECOND, *this);
+        } else {
+            TimeInterval interval(DAY, 1, false);
+            date_add_interval(interval, DAY, *this);
+        }
         return *this;
     }
 
@@ -874,42 +1024,105 @@ public:
         return std::string(buf, end - buf);
     }
 
-    bool is_valid_date() const { return !is_invalid(this->year(), this->month(), this->day()); }
+    bool is_valid_date() const {
+        if constexpr (is_datetime) {
+            return !is_invalid(this->year(), this->month(), this->day(), this->hour(),
+                               this->minute(), this->second(), this->microsecond());
+        } else {
+            return !is_invalid(this->year(), this->month(), this->day(), 0, 0, 0, 0);
+        }
+    }
 
-    int64_t second_diff(const DateV2Value& rhs) const {
+    template <typename RHS>
+    int64_t second_diff(const DateV2Value<RHS>& rhs) const {
         int day_diff = daynr() - rhs.daynr();
-        return day_diff * 3600 * 24;
+        return day_diff * 3600 * 24 + (hour() * 3600 + minute() * 60 + second()) -
+               (rhs.hour() * 3600 + rhs.minute() * 60 + rhs.second());
     }
 
     int64_t second_diff(const VecDateTimeValue& rhs) const {
         int day_diff = daynr() - rhs.daynr();
-        int time_diff = rhs.hour() * 3600 + rhs.minute() * 60 + rhs.second();
-        return day_diff * 3600 * 24 - time_diff;
+        return day_diff * 3600 * 24 + (hour() * 3600 + minute() * 60 + second()) -
+               (rhs.hour() * 3600 + rhs.minute() * 60 + rhs.second());
     };
 
+    int64_t time_part_diff(const VecDateTimeValue& rhs) const {
+        int time_diff = (hour() * 3600 + minute() * 60 + second()) -
+                        (rhs.hour() * 3600 + rhs.minute() * 60 + rhs.second());
+        return time_diff;
+    }
+
     void convert_date_v2_to_dt(doris::DateTimeValue* dt);
     void convert_dt_to_date_v2(doris::DateTimeValue* dt);
 
-    uint32_t to_date_uint32() const;
+    bool can_cast_to_date_without_loss_accuracy() {
+        return this->hour() == 0 && this->minute() == 0 && this->second() == 0 &&
+               this->microsecond() == 0;
+    }
+
+    underlying_value to_date_int_val() const;
 
     bool from_date(uint32_t value);
-    bool from_date(int64_t value);
+    bool from_datetime(uint64_t value);
 
     bool from_date_int64(int64_t value);
     uint32_t set_date_uint32(uint32_t int_val);
+    uint64_t set_datetime_uint64(uint64_t int_val);
 
     bool get_date_from_daynr(uint64_t);
 
     void to_datev2_val(doris_udf::DateV2Val* tv) const {
-        tv->datev2_value = this->to_date_uint32();
+        DCHECK(!is_datetime);
+        tv->datev2_value = this->to_date_int_val();
     }
 
-    static DateV2Value from_datev2_val(const doris_udf::DateV2Val& tv) {
-        DateV2Value value;
+    static DateV2Value<DateV2ValueType> from_datev2_val(const doris_udf::DateV2Val& tv) {
+        DCHECK(!is_datetime);
+        DateV2Value<DateV2ValueType> value;
         value.from_date(tv.datev2_value);
         return value;
     }
 
+    void to_datetimev2_val(doris_udf::DateTimeV2Val* tv) const {
+        DCHECK(is_datetime);
+        tv->datetimev2_value = this->to_date_int_val();
+    }
+
+    static DateV2Value<DateTimeV2ValueType> from_datetimev2_val(
+            const doris_udf::DateTimeV2Val& tv) {
+        DCHECK(is_datetime);
+        DateV2Value<DateTimeV2ValueType> value;
+        value.from_datetime(tv.datetimev2_value);
+        return value;
+    }
+
+    template <TimeUnit unit>
+    void set_time_unit(uint32_t val) {
+        if constexpr (unit == TimeUnit::YEAR) {
+            date_v2_value_.year_ = val;
+        } else if constexpr (unit == TimeUnit::MONTH) {
+            date_v2_value_.month_ = val;
+        } else if constexpr (unit == TimeUnit::DAY) {
+            date_v2_value_.day_ = val;
+        } else if constexpr (unit == TimeUnit::HOUR) {
+            if constexpr (is_datetime) {
+                date_v2_value_.hour_ = val;
+            }
+        } else if constexpr (unit == TimeUnit::MINUTE) {
+            if constexpr (is_datetime) {
+                date_v2_value_.minute_ = val;
+            }
+        } else if constexpr (unit == TimeUnit::SECOND) {
+            if constexpr (is_datetime) {
+                date_v2_value_.second_ = val;
+            }
+        } else if constexpr (unit == TimeUnit::SECOND_MICROSECOND) {
+            if constexpr (is_datetime) {
+                date_v2_value_.microsecond_ = val;
+            }
+        }
+    }
+
 private:
     static uint8_t calc_week(const uint32_t& day_nr, const uint16_t& year, const uint8_t& month,
                              const uint8_t& day, uint8_t mode, uint16_t* to_year);
@@ -918,35 +1131,42 @@ private:
     int64_t standardize_timevalue(int64_t value);
 
     // Helper to set max, min, zero
-    void set_zero(int type);
+    void set_zero();
 
     bool from_date_format_str(const char* format, int format_len, const char* value, int value_len,
                               const char** sub_val_end);
 
     union {
-        DateV2ValueType date_v2_value_;
-        uint32_t int_val_;
+        T date_v2_value_;
+        underlying_value int_val_;
     };
 
-    DateV2Value(uint16_t year, uint8_t month, uint8_t day) : date_v2_value_(year, month, day) {}
+    DateV2Value<T>(uint16_t year, uint8_t month, uint8_t day, uint8_t hour, uint8_t minute,
+                   uint8_t second, uint32_t microsecond)
+            : date_v2_value_(year, month, day, hour, minute, second, microsecond) {}
 };
 
 // only support DATE - DATE (no support DATETIME - DATETIME)
 std::size_t operator-(const VecDateTimeValue& v1, const VecDateTimeValue& v2);
 
-std::size_t operator-(const VecDateTimeValue& v1, const DateV2Value& v2);
+template <typename T>
+std::size_t operator-(const VecDateTimeValue& v1, const DateV2Value<T>& v2);
 
-std::size_t operator-(const DateV2Value& v1, const VecDateTimeValue& v2);
+template <typename T>
+std::size_t operator-(const DateV2Value<T>& v1, const VecDateTimeValue& v2);
 
 std::ostream& operator<<(std::ostream& os, const VecDateTimeValue& value);
 
 std::size_t hash_value(VecDateTimeValue const& value);
 
-std::size_t operator-(const DateV2Value& v1, const DateV2Value& v2);
+template <typename T0, typename T1>
+std::size_t operator-(const DateV2Value<T0>& v1, const DateV2Value<T1>& v2);
 
-std::ostream& operator<<(std::ostream& os, const DateV2Value& value);
+template <typename T>
+std::ostream& operator<<(std::ostream& os, const DateV2Value<T>& value);
 
-std::size_t hash_value(DateV2Value const& value);
+template <typename T>
+std::size_t hash_value(DateV2Value<T> const& value);
 
 template <TimeUnit unit>
 int64_t datetime_diff(const VecDateTimeValue& ts_value1, const VecDateTimeValue& ts_value2) {
@@ -1011,27 +1231,79 @@ int64_t datetime_diff(const VecDateTimeValue& ts_value1, const VecDateTimeValue&
     return 0;
 }
 
-template <TimeUnit unit>
-int64_t datetime_diff(const DateV2Value& ts_value1, const DateV2Value& ts_value2) {
+template <TimeUnit unit, typename T0, typename T1>
+int64_t datetime_diff(const DateV2Value<T0>& ts_value1, const DateV2Value<T1>& ts_value2) {
+    constexpr uint32_t minus_one = -1;
     switch (unit) {
     case YEAR: {
         int year = (ts_value2.year() - ts_value1.year());
-        if (year > 0) {
-            year -= ((ts_value2.to_date_uint32() & 0x0000FFFF) -
-                     (ts_value1.to_date_uint32() & 0x0000FFFF)) < 0;
-        } else if (year < 0) {
-            year += ((ts_value2.to_date_uint32() & 0x0000FFFF) -
-                     (ts_value1.to_date_uint32() & 0x0000FFFF)) > 0;
+        if constexpr (std::is_same_v<T0, T1>) {
+            int year_width =
+                    DateV2Value<T0>::is_datetime ? DATETIMEV2_YEAR_WIDTH : DATEV2_YEAR_WIDTH;
+            if (year > 0) {
+                year -= ((ts_value2.to_date_int_val() & (minus_one >> year_width)) -
+                         (ts_value1.to_date_int_val() & (minus_one >> year_width))) < 0;
+            } else if (year < 0) {
+                year += ((ts_value2.to_date_int_val() & (minus_one >> year_width)) -
+                         (ts_value1.to_date_int_val() & (minus_one >> year_width))) > 0;
+            }
+        } else if constexpr (std::is_same_v<T0, DateV2ValueType>) {
+            auto ts2_int_value = (uint64_t)ts_value2.to_date_int_val() << TIME_PART_LENGTH;
+            if (year > 0) {
+                year -= ((ts2_int_value & (minus_one >> DATETIMEV2_YEAR_WIDTH)) -
+                         (ts_value1.to_date_int_val() & (minus_one >> DATETIMEV2_YEAR_WIDTH))) < 0;
+            } else if (year < 0) {
+                year += ((ts2_int_value & (minus_one >> DATETIMEV2_YEAR_WIDTH)) -
+                         (ts_value1.to_date_int_val() & (minus_one >> DATETIMEV2_YEAR_WIDTH))) > 0;
+            }
+        } else {
+            auto ts1_int_value = (uint64_t)ts_value2.to_date_int_val() << TIME_PART_LENGTH;
+            if (year > 0) {
+                year -= ((ts_value2.to_date_int_val() & (minus_one >> DATETIMEV2_YEAR_WIDTH)) -
+                         (ts1_int_value & (minus_one >> DATETIMEV2_YEAR_WIDTH))) < 0;
+            } else if (year < 0) {
+                year += ((ts_value2.to_date_int_val() & (minus_one >> DATETIMEV2_YEAR_WIDTH)) -
+                         (ts1_int_value & (minus_one >> DATETIMEV2_YEAR_WIDTH))) > 0;
+            }
         }
+
         return year;
     }
     case MONTH: {
         int month = (ts_value2.year() - ts_value1.year()) * 12 +
                     (ts_value2.month() - ts_value1.month());
-        if (month > 0) {
-            month -= (ts_value2.day() - ts_value1.day()) < 0;
-        } else if (month < 0) {
-            month += (ts_value2.day() - ts_value1.day()) > 0;
+        if constexpr (std::is_same_v<T0, T1>) {
+            int shift_bits = DateV2Value<T0>::is_datetime ? DATETIMEV2_YEAR_WIDTH + 5
+                                                          : DATEV2_YEAR_WIDTH + 5;
+            if (month > 0) {
+                month -= ((ts_value2.to_date_int_val() & (minus_one >> shift_bits)) -
+                          (ts_value1.to_date_int_val() & (minus_one >> shift_bits))) < 0;
+            } else if (month < 0) {
+                month += ((ts_value2.to_date_int_val() & (minus_one >> shift_bits)) -
+                          (ts_value1.to_date_int_val() & (minus_one >> shift_bits))) > 0;
+            }
+        } else if constexpr (std::is_same_v<T0, DateV2ValueType>) {
+            auto ts2_int_value = (uint64_t)ts_value2.to_date_int_val() << TIME_PART_LENGTH;
+            if (month > 0) {
+                month -= ((ts2_int_value & (minus_one >> DATETIMEV2_YEAR_WIDTH)) -
+                          (ts_value1.to_date_int_val() &
+                           (minus_one >> (DATETIMEV2_YEAR_WIDTH + 5)))) < 0;
+            } else if (month < 0) {
+                month += ((ts2_int_value & (minus_one >> DATETIMEV2_YEAR_WIDTH)) -
+                          (ts_value1.to_date_int_val() &
+                           (minus_one >> (DATETIMEV2_YEAR_WIDTH + 5)))) > 0;
+            }
+        } else {
+            auto ts1_int_value = (uint64_t)ts_value2.to_date_int_val() << TIME_PART_LENGTH;
+            if (month > 0) {
+                month -= ((ts_value2.to_date_int_val() &
+                           (minus_one >> (DATETIMEV2_YEAR_WIDTH + 5))) -
+                          (ts1_int_value & (minus_one >> DATETIMEV2_YEAR_WIDTH))) < 0;
+            } else if (month < 0) {
+                month += ((ts_value2.to_date_int_val() &
+                           (minus_one >> (DATETIMEV2_YEAR_WIDTH + 5))) -
+                          (ts1_int_value & (minus_one >> DATETIMEV2_YEAR_WIDTH))) > 0;
+            }
         }
         return month;
     }
@@ -1062,17 +1334,16 @@ int64_t datetime_diff(const DateV2Value& ts_value1, const DateV2Value& ts_value2
     return 0;
 }
 
-template <TimeUnit unit>
-int64_t datetime_diff(const DateV2Value& ts_value1, const VecDateTimeValue& ts_value2) {
+template <TimeUnit unit, typename T>
+int64_t datetime_diff(const DateV2Value<T>& ts_value1, const VecDateTimeValue& ts_value2) {
+    // FIXME:
     switch (unit) {
     case YEAR: {
         int year = (ts_value2.year() - ts_value1.year());
         if (year > 0) {
-            year -= ((((ts_value2.to_datetime_int64() % 10000000000) / 1000000L) % 10000L) -
-                     (ts_value1.to_date_uint32() & 0x0000FFFF)) < 0;
+            year -= ts_value1.month() - ts_value2.month() < 0;
         } else if (year < 0) {
-            year += ((((ts_value2.to_datetime_int64() % 10000000000) / 1000000L) % 10000L) -
-                     (ts_value1.to_date_uint32() & 0x0000FFFF)) > 0;
+            year += ts_value1.month() - ts_value2.month() > 0;
         }
         return year;
     }
@@ -1113,17 +1384,15 @@ int64_t datetime_diff(const DateV2Value& ts_value1, const VecDateTimeValue& ts_v
     return 0;
 }
 
-template <TimeUnit unit>
-int64_t datetime_diff(const VecDateTimeValue& ts_value1, const DateV2Value& ts_value2) {
+template <TimeUnit unit, typename T>
+int64_t datetime_diff(const VecDateTimeValue& ts_value1, const DateV2Value<T>& ts_value2) {
     switch (unit) {
     case YEAR: {
         int year = (ts_value2.year() - ts_value1.year());
         if (year > 0) {
-            year -= ((ts_value2.to_date_uint32() & 0x0000FFFF) -
-                     (((ts_value1.to_datetime_int64() % 10000000000) / 1000000L) % 10000L)) < 0;
+            year -= ts_value1.month() - ts_value2.month() < 0;
         } else if (year < 0) {
-            year += ((ts_value2.to_date_uint32() & 0x0000FFFF) -
-                     (((ts_value1.to_datetime_int64() % 10000000000) / 1000000L) % 10000L)) > 0;
+            year -= ts_value1.month() - ts_value2.month() > 0;
         }
         return year;
     }
@@ -1176,8 +1445,17 @@ struct hash<doris::vectorized::VecDateTimeValue> {
 };
 
 template <>
-struct hash<doris::vectorized::DateV2Value> {
-    size_t operator()(const doris::vectorized::DateV2Value& v) const {
+struct hash<doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>> {
+    size_t operator()(
+            const doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>& v) const {
+        return doris::vectorized::hash_value(v);
+    }
+};
+
+template <>
+struct hash<doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>> {
+    size_t operator()(
+            const doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>& v) const {
         return doris::vectorized::hash_value(v);
     }
 };
diff --git a/be/src/vec/runtime/vfile_result_writer.cpp b/be/src/vec/runtime/vfile_result_writer.cpp
index 128f5c9cda..a132eb696d 100644
--- a/be/src/vec/runtime/vfile_result_writer.cpp
+++ b/be/src/vec/runtime/vfile_result_writer.cpp
@@ -258,12 +258,21 @@ Status VFileResultWriter::_write_csv_file(const Block& block) {
                 }
                 case TYPE_DATEV2: {
                     char buf[64];
-                    const DateV2Value* time_val =
-                            (const DateV2Value*)(col.column->get_data_at(i).data);
+                    const DateV2Value<DateV2ValueType>* time_val =
+                            (const DateV2Value<DateV2ValueType>*)(col.column->get_data_at(i).data);
                     time_val->to_string(buf);
                     _plain_text_outstream << buf;
                     break;
                 }
+                case TYPE_DATETIMEV2: {
+                    char buf[64];
+                    const DateV2Value<DateTimeV2ValueType>* time_val =
+                            (const DateV2Value<DateTimeV2ValueType>*)(col.column->get_data_at(i)
+                                                                              .data);
+                    time_val->to_string(buf, _output_vexpr_ctxs[col_id]->root()->type().scale);
+                    _plain_text_outstream << buf;
+                    break;
+                }
                 case TYPE_DATE:
                 case TYPE_DATETIME: {
                     char buf[64];
diff --git a/be/src/vec/sink/vmysql_result_writer.cpp b/be/src/vec/sink/vmysql_result_writer.cpp
index a520217c25..a0c7dbb02a 100644
--- a/be/src/vec/sink/vmysql_result_writer.cpp
+++ b/be/src/vec/sink/vmysql_result_writer.cpp
@@ -58,7 +58,7 @@ void VMysqlResultWriter::_init_profile() {
 template <PrimitiveType type, bool is_nullable>
 Status VMysqlResultWriter::_add_one_column(const ColumnPtr& column_ptr,
                                            std::unique_ptr<TFetchDataResult>& result,
-                                           const DataTypePtr& nested_type_ptr) {
+                                           const DataTypePtr& nested_type_ptr, int scale) {
     SCOPED_TIMER(_convert_tuple_timer);
 
     const auto row_size = column_ptr->size();
@@ -229,11 +229,20 @@ Status VMysqlResultWriter::_add_one_column(const ColumnPtr& column_ptr,
             if constexpr (type == TYPE_DATEV2) {
                 char buf[64];
                 auto time_num = data[i];
-                doris::vectorized::DateV2Value date_val;
+                doris::vectorized::DateV2Value<DateV2ValueType> date_val;
                 memcpy(static_cast<void*>(&date_val), &time_num, sizeof(UInt32));
                 char* pos = date_val.to_string(buf);
                 buf_ret = _buffer.push_string(buf, pos - buf - 1);
             }
+            if constexpr (type == TYPE_DATETIMEV2) {
+                // TODO: use correct scale here
+                char buf[64];
+                auto time_num = data[i];
+                doris::vectorized::DateV2Value<DateTimeV2ValueType> date_val;
+                memcpy(static_cast<void*>(&date_val), &time_num, sizeof(UInt64));
+                char* pos = date_val.to_string(buf, scale);
+                buf_ret = _buffer.push_string(buf, pos - buf - 1);
+            }
             if constexpr (type == TYPE_DECIMALV2) {
                 DecimalV2Value decimal_val(data[i]);
                 auto decimal_str = decimal_val.to_string();
@@ -319,7 +328,7 @@ int VMysqlResultWriter::_add_one_cell(const ColumnPtr& column_ptr, size_t row_id
     } else if (which.is_date_v2()) {
         auto& column_vector = assert_cast<const ColumnVector<UInt32>&>(*column);
         auto value = column_vector[row_idx].get<UInt32>();
-        DateV2Value datev2;
+        DateV2Value<DateV2ValueType> datev2;
         memcpy(static_cast<void*>(&datev2), static_cast<void*>(&value), sizeof(value));
         char buf[64];
         char* pos = datev2.to_string(buf);
@@ -568,6 +577,17 @@ 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);
+            } else {
+                status = _add_one_column<PrimitiveType::TYPE_DATETIMEV2, false>(column_ptr, result,
+                                                                                nullptr, scale);
+            }
+            break;
+        }
         case TYPE_HLL:
         case TYPE_OBJECT: {
             if (type_ptr->is_nullable()) {
diff --git a/be/src/vec/sink/vmysql_result_writer.h b/be/src/vec/sink/vmysql_result_writer.h
index 49f03ee356..7b8df80306 100644
--- a/be/src/vec/sink/vmysql_result_writer.h
+++ b/be/src/vec/sink/vmysql_result_writer.h
@@ -50,7 +50,7 @@ private:
 
     template <PrimitiveType type, bool is_nullable>
     Status _add_one_column(const ColumnPtr& column_ptr, std::unique_ptr<TFetchDataResult>& result,
-                           const DataTypePtr& nested_type_ptr = nullptr);
+                           const DataTypePtr& nested_type_ptr = nullptr, int scale = -1);
     int _add_one_cell(const ColumnPtr& column_ptr, size_t row_idx, const DataTypePtr& type,
                       MysqlRowBuffer& buffer);
 
diff --git a/be/src/vec/sink/vmysql_table_writer.cpp b/be/src/vec/sink/vmysql_table_writer.cpp
index f71a7a2329..251844e9ef 100644
--- a/be/src/vec/sink/vmysql_table_writer.cpp
+++ b/be/src/vec/sink/vmysql_table_writer.cpp
@@ -196,8 +196,8 @@ Status VMysqlTableWriter::insert_row(vectorized::Block& block, size_t row) {
         case TYPE_DATEV2: {
             uint32_t int_val =
                     assert_cast<const vectorized::ColumnUInt32&>(*column).get_data()[row];
-            vectorized::DateV2Value value =
-                    binary_cast<uint32_t, doris::vectorized::DateV2Value>(int_val);
+            vectorized::DateV2Value<DateV2ValueType> value =
+                    binary_cast<uint32_t, doris::vectorized::DateV2Value<DateV2ValueType>>(int_val);
 
             char buf[64];
             char* pos = value.to_string(buf);
@@ -205,6 +205,19 @@ Status VMysqlTableWriter::insert_row(vectorized::Block& block, size_t row) {
             fmt::format_to(_insert_stmt_buffer, "'{}'", str);
             break;
         }
+        case TYPE_DATETIMEV2: {
+            uint32_t int_val =
+                    assert_cast<const vectorized::ColumnUInt64&>(*column).get_data()[row];
+            vectorized::DateV2Value<DateTimeV2ValueType> value =
+                    binary_cast<uint64_t, doris::vectorized::DateV2Value<DateTimeV2ValueType>>(
+                            int_val);
+
+            char buf[64];
+            char* pos = value.to_string(buf, _vec_output_expr_ctxs[i]->root()->type().scale);
+            std::string str(buf, pos - buf - 1);
+            fmt::format_to(_insert_stmt_buffer, "'{}'", str);
+            break;
+        }
         default: {
             fmt::memory_buffer err_out;
             fmt::format_to(err_out, "can't convert this type to mysql type. type = {}",
diff --git a/be/src/vec/utils/arrow_column_to_doris_column.cpp b/be/src/vec/utils/arrow_column_to_doris_column.cpp
index 9f2f7ddb26..eec727ebbc 100644
--- a/be/src/vec/utils/arrow_column_to_doris_column.cpp
+++ b/be/src/vec/utils/arrow_column_to_doris_column.cpp
@@ -72,6 +72,7 @@ namespace doris::vectorized {
 
 const PrimitiveType arrow_type_to_primitive_type(::arrow::Type::type type) {
     switch (type) {
+        // TODO: convert arrow date type to datev2/datetimev2
 #define DISPATCH(ARROW_TYPE, CPP_TYPE) \
     case ARROW_TYPE:                   \
         return CPP_TYPE;
@@ -235,10 +236,40 @@ static Status convert_column_with_date_v2_data(const arrow::Array* array, size_t
     }
 
     for (size_t value_i = array_idx; value_i < array_idx + num_elements; ++value_i) {
-        DateV2Value v;
+        DateV2Value<DateV2ValueType> v;
         v.from_unixtime(static_cast<Int64>(concrete_array->Value(value_i)) / divisor * multiplier,
                         ctz);
-        column_data.emplace_back(binary_cast<DateV2Value, UInt32>(v));
+        column_data.emplace_back(binary_cast<DateV2Value<DateV2ValueType>, UInt32>(v));
+    }
+    return Status::OK();
+}
+
+template <typename ArrowType>
+static Status convert_column_with_datetime_v2_data(const arrow::Array* array, size_t array_idx,
+                                                   MutableColumnPtr& data_column,
+                                                   size_t num_elements,
+                                                   const cctz::time_zone& ctz) {
+    auto& column_data = static_cast<ColumnVector<UInt64>&>(*data_column).get_data();
+    auto concrete_array = down_cast<const ArrowType*>(array);
+    int64_t divisor = 1;
+    int64_t multiplier = 1;
+    if constexpr (std::is_same_v<ArrowType, arrow::TimestampArray>) {
+        const auto type = std::static_pointer_cast<arrow::TimestampType>(array->type());
+        divisor = time_unit_divisor(type->unit());
+        if (divisor == 0L) {
+            return Status::InternalError(fmt::format("Invalid Time Type:{}", type->name()));
+        }
+    } else if constexpr (std::is_same_v<ArrowType, arrow::Date32Array>) {
+        multiplier = 24 * 60 * 60; // day => secs
+    } else if constexpr (std::is_same_v<ArrowType, arrow::Date64Array>) {
+        divisor = 1000; //ms => secs
+    }
+
+    for (size_t value_i = array_idx; value_i < array_idx + num_elements; ++value_i) {
+        DateV2Value<DateTimeV2ValueType> v;
+        v.from_unixtime(static_cast<Int64>(concrete_array->Value(value_i)) / divisor * multiplier,
+                        ctz);
+        column_data.emplace_back(binary_cast<DateV2Value<DateTimeV2ValueType>, UInt64>(v));
     }
     return Status::OK();
 }
@@ -351,11 +382,21 @@ Status arrow_column_to_doris_column(const arrow::Array* arrow_column, size_t arr
                     arrow_column, arrow_batch_cur_idx, data_column, num_elements, ctz);
         }
     case arrow::Type::DATE64:
-        return convert_column_with_timestamp_data<arrow::Date64Array>(
-                arrow_column, arrow_batch_cur_idx, data_column, num_elements, ctz);
+        if (which_type.is_date_v2_or_datetime_v2()) {
+            return convert_column_with_datetime_v2_data<arrow::Date64Array>(
+                    arrow_column, arrow_batch_cur_idx, data_column, num_elements, ctz);
+        } else {
+            return convert_column_with_timestamp_data<arrow::Date64Array>(
+                    arrow_column, arrow_batch_cur_idx, data_column, num_elements, ctz);
+        }
     case arrow::Type::TIMESTAMP:
-        return convert_column_with_timestamp_data<arrow::TimestampArray>(
-                arrow_column, arrow_batch_cur_idx, data_column, num_elements, ctz);
+        if (which_type.is_date_v2_or_datetime_v2()) {
+            return convert_column_with_datetime_v2_data<arrow::TimestampArray>(
+                    arrow_column, arrow_batch_cur_idx, data_column, num_elements, ctz);
+        } else {
+            return convert_column_with_timestamp_data<arrow::TimestampArray>(
+                    arrow_column, arrow_batch_cur_idx, data_column, num_elements, ctz);
+        }
     case arrow::Type::DECIMAL:
         return convert_column_with_decimal_data(arrow_column, arrow_batch_cur_idx, data_column,
                                                 num_elements);
diff --git a/be/src/vec/utils/template_helpers.hpp b/be/src/vec/utils/template_helpers.hpp
index 5db9147c89..ebf822513b 100644
--- a/be/src/vec/utils/template_helpers.hpp
+++ b/be/src/vec/utils/template_helpers.hpp
@@ -46,7 +46,8 @@
 #define TIME_TYPE_TO_COLUMN_TYPE(M) \
     M(Date, ColumnInt64)            \
     M(DateTime, ColumnInt64)        \
-    M(DateV2, ColumnUInt32)
+    M(DateV2, ColumnUInt32)         \
+    M(DateTimeV2, ColumnUInt64)
 
 #define COMPLEX_TYPE_TO_COLUMN_TYPE(M) \
     M(BitMap, ColumnBitmap)            \
diff --git a/be/test/olap/delta_writer_test.cpp b/be/test/olap/delta_writer_test.cpp
index 83502b24b6..33c4a48737 100644
--- a/be/test/olap/delta_writer_test.cpp
+++ b/be/test/olap/delta_writer_test.cpp
@@ -463,7 +463,8 @@ TEST_F(TestDeltaWriter, write) {
         DecimalV2Value decimal_value;
         decimal_value.assign_from_double(1.1);
         *(DecimalV2Value*)(tuple->get_slot(slots[9]->tuple_offset())) = decimal_value;
-        ((doris::vectorized::DateV2Value*)(tuple->get_slot(slots[10]->tuple_offset())))
+        ((doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*)(tuple->get_slot(
+                 slots[10]->tuple_offset())))
                 ->from_date_str("2048-11-10", 10);
 
         *(int8_t*)(tuple->get_slot(slots[11]->tuple_offset())) = -127;
@@ -493,7 +494,8 @@ TEST_F(TestDeltaWriter, write) {
 
         *(DecimalV2Value*)(tuple->get_slot(slots[20]->tuple_offset())) = val_decimal;
 
-        ((doris::vectorized::DateV2Value*)(tuple->get_slot(slots[21]->tuple_offset())))
+        ((doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*)(tuple->get_slot(
+                 slots[21]->tuple_offset())))
                 ->from_date_str("2048-11-10", 10);
 
         res = delta_writer->write(tuple);
@@ -595,9 +597,9 @@ TEST_F(TestDeltaWriter, vec_write) {
         decimal_value.assign_from_double(1.1);
         columns[9]->insert_data((const char*)&decimal_value, sizeof(decimal_value));
 
-        doris::vectorized::DateV2Value date_v2;
+        doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> date_v2;
         date_v2.from_date_str("2048-11-10", 10);
-        auto date_v2_int = date_v2.to_date_uint32();
+        auto date_v2_int = date_v2.to_date_int_val();
         columns[10]->insert_data((const char*)&date_v2_int, sizeof(date_v2_int));
 
         int8_t v1 = -127;
@@ -632,7 +634,7 @@ TEST_F(TestDeltaWriter, vec_write) {
         columns[20]->insert_data((const char*)&decimal_value, sizeof(decimal_value));
 
         date_v2.from_date_str("2048-11-10", 10);
-        date_v2_int = date_v2.to_date_uint32();
+        date_v2_int = date_v2.to_date_int_val();
         columns[21]->insert_data((const char*)&date_v2_int, sizeof(date_v2_int));
 
         res = delta_writer->write(&block, {0});
@@ -790,9 +792,9 @@ TEST_F(TestDeltaWriter, vec_sequence_col) {
         int64_t c4_int = c4.to_int64();
         columns[3]->insert_data((const char*)&c4_int, sizeof(c4));
 
-        doris::vectorized::DateV2Value c5;
-        c5.set_time(2022, 6, 6);
-        uint32_t c5_int = c5.to_date_uint32();
+        doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> c5;
+        c5.set_time(2022, 6, 6, 0, 0, 0, 0);
+        uint32_t c5_int = c5.to_date_int_val();
         columns[4]->insert_data((const char*)&c5_int, sizeof(c5));
 
         res = delta_writer->write(&block, {0});
diff --git a/be/test/olap/in_list_predicate_test.cpp b/be/test/olap/in_list_predicate_test.cpp
index 73a77654d1..311c2b8fb2 100644
--- a/be/test/olap/in_list_predicate_test.cpp
+++ b/be/test/olap/in_list_predicate_test.cpp
@@ -44,9 +44,10 @@ static uint32_t timestamp_from_date_v2(const char* date_string) {
     tm time_tm;
     strptime(date_string, "%Y-%m-%d", &time_tm);
 
-    doris::vectorized::DateV2Value value;
-    value.set_time(time_tm.tm_year + 1900, time_tm.tm_mon + 1, time_tm.tm_mday);
-    return binary_cast<doris::vectorized::DateV2Value, uint32_t>(value);
+    doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> value;
+    value.set_time(time_tm.tm_year + 1900, time_tm.tm_mon + 1, time_tm.tm_mday, 0, 0, 0, 0);
+    return binary_cast<doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>,
+                       uint32_t>(value);
 }
 
 static uint64_t timestamp_from_datetime(const std::string& value_string) {
@@ -74,7 +75,8 @@ static std::string to_date_string(uint24_t& date_value) {
 }
 
 static std::string to_date_v2_string(uint32_t& date_value) {
-    auto val = binary_cast<uint32_t, vectorized::DateV2Value>(date_value);
+    auto val = binary_cast<uint32_t, vectorized::DateV2Value<doris::vectorized::DateV2ValueType>>(
+            date_value);
     std::stringstream ss;
     ss << val;
     return ss.str();
diff --git a/be/test/vec/core/block_test.cpp b/be/test/vec/core/block_test.cpp
index ae35d95cb7..546e120245 100644
--- a/be/test/vec/core/block_test.cpp
+++ b/be/test/vec/core/block_test.cpp
@@ -114,10 +114,10 @@ TEST(BlockTest, RowBatchCovertToBlock) {
         memcpy(tuple->get_slot(slot_desc->tuple_offset()), &k7, column_descs[6].size);
 
         slot_desc = tuple_desc->slots()[7];
-        vectorized::DateV2Value k8;
+        vectorized::DateV2Value<doris::vectorized::DateV2ValueType> k8;
         std::string now_date("2020-12-02");
         k8.from_date_str(now_date.c_str(), now_date.size());
-        k8.date_add_interval(time_interval, vectorized::TimeUnit::DAY);
+        k8.date_add_interval(time_interval, vectorized::TimeUnit::DAY, k8);
         memcpy(tuple->get_slot(slot_desc->tuple_offset()), &k8, column_descs[7].size);
 
         tuple_row->set_tuple(0, tuple);
@@ -166,12 +166,12 @@ TEST(BlockTest, RowBatchCovertToBlock) {
         EXPECT_EQ(k7, date_time_value);
 
         larget_int = column8->operator[](i).get<vectorized::UInt32>();
-        vectorized::DateV2Value k8;
+        vectorized::DateV2Value<doris::vectorized::DateV2ValueType> k8;
         memcpy(reinterpret_cast<vectorized::Int128*>(&k8), &larget_int, column_descs[7].size);
-        vectorized::DateV2Value date_v2_value;
+        vectorized::DateV2Value<doris::vectorized::DateV2ValueType> date_v2_value;
         std::string now_date("2020-12-02");
         date_v2_value.from_date_str(now_date.c_str(), now_date.size());
-        date_v2_value.date_add_interval(time_interval, vectorized::TimeUnit::DAY);
+        date_v2_value.date_add_interval(time_interval, vectorized::TimeUnit::DAY, date_v2_value);
 
         EXPECT_EQ(k8, date_v2_value);
 
@@ -475,7 +475,7 @@ TEST(BlockTest, dump_data) {
     auto column_vector_date_v2 = vectorized::ColumnVector<vectorized::UInt32>::create();
     auto& date_v2_data = column_vector_date_v2->get_data();
     for (int i = 0; i < 1024; ++i) {
-        vectorized::DateV2Value value;
+        vectorized::DateV2Value<doris::vectorized::DateV2ValueType> value;
         value.from_date((uint32_t)((2022 << 9) | (6 << 5) | 6));
         date_v2_data.push_back(*reinterpret_cast<vectorized::UInt32*>(&value));
     }
diff --git a/be/test/vec/exprs/vexpr_test.cpp b/be/test/vec/exprs/vexpr_test.cpp
index 473a8aa695..27c76c320f 100644
--- a/be/test/vec/exprs/vexpr_test.cpp
+++ b/be/test/vec/exprs/vexpr_test.cpp
@@ -355,7 +355,7 @@ TEST(TEST_VEXPR, LITERALTEST) {
         EXPECT_EQ(v, dt);
     }
     {
-        vectorized::DateV2Value data_time_value;
+        vectorized::DateV2Value<doris::vectorized::DateV2ValueType> data_time_value;
         const char* date = "20210407";
         data_time_value.from_date_str(date, strlen(date));
         uint32_t dt;
diff --git a/be/test/vec/function/function_test_util.cpp b/be/test/vec/function/function_test_util.cpp
index e5c7dc04ef..956a76e4ff 100644
--- a/be/test/vec/function/function_test_util.cpp
+++ b/be/test/vec/function/function_test_util.cpp
@@ -34,10 +34,10 @@ int64_t str_to_date_time(std::string datetime_str, bool data_time) {
 }
 
 uint32_t str_to_date_v2(std::string datetime_str, std::string datetime_format) {
-    DateV2Value v;
+    DateV2Value<DateV2ValueType> v;
     v.from_date_format_str(datetime_format.c_str(), datetime_format.size(), datetime_str.c_str(),
                            datetime_str.size());
-    return binary_cast<DateV2Value, UInt32>(v);
+    return binary_cast<DateV2Value<DateV2ValueType>, UInt32>(v);
 }
 
 size_t type_index_to_data_type(const std::vector<std::any>& input_types, size_t index,
@@ -216,7 +216,7 @@ bool insert_cell(MutableColumnPtr& column, DataTypePtr type_ptr, const std::any&
     } else if (type.is_date_v2()) {
         static std::string date_time_format("%Y-%m-%d");
         auto datetime_str = std::any_cast<std::string>(cell);
-        DateV2Value v;
+        DateV2Value<DateV2ValueType> v;
         v.from_date_format_str(date_time_format.c_str(), date_time_format.size(),
                                datetime_str.c_str(), datetime_str.size());
         column->insert_data(reinterpret_cast<char*>(&v), 0);
diff --git a/be/test/vec/runtime/vdatetime_value_test.cpp b/be/test/vec/runtime/vdatetime_value_test.cpp
index c445325e58..809705dead 100644
--- a/be/test/vec/runtime/vdatetime_value_test.cpp
+++ b/be/test/vec/runtime/vdatetime_value_test.cpp
@@ -29,31 +29,56 @@ TEST(VDateTimeValueTest, date_v2_to_uint32_test) {
     uint8_t month = 5;
     uint8_t day = 24;
 
-    DateV2Value date_v2;
-    date_v2.set_time(year, month, day);
+    DateV2Value<DateV2ValueType> date_v2;
+    date_v2.set_time(year, month, day, 0, 0, 0, 0);
 
     EXPECT_TRUE(date_v2.year() == year);
     EXPECT_TRUE(date_v2.month() == month);
     EXPECT_TRUE(date_v2.day() == day);
-    EXPECT_TRUE(date_v2.to_date_uint32() == ((year << 9) | (month << 5) | day));
+    EXPECT_TRUE(date_v2.to_date_int_val() == ((year << 9) | (month << 5) | day));
     EXPECT_TRUE(date_v2.hour() == 0);
     EXPECT_TRUE(date_v2.minute() == 0);
     EXPECT_TRUE(date_v2.second() == 0);
 }
 
+TEST(VDateTimeValueTest, datetime_v2_to_uint64_test) {
+    uint16_t year = 2022;
+    uint8_t month = 5;
+    uint8_t day = 24;
+    uint8_t hour = 23;
+    uint8_t minute = 0;
+    uint8_t second = 0;
+    uint32_t microsecond = 999999;
+
+    DateV2Value<DateTimeV2ValueType> datetime_v2;
+    datetime_v2.set_time(year, month, day, hour, minute, second, microsecond);
+
+    EXPECT_TRUE(datetime_v2.year() == year);
+    EXPECT_TRUE(datetime_v2.month() == month);
+    EXPECT_TRUE(datetime_v2.day() == day);
+    EXPECT_TRUE(datetime_v2.to_date_int_val() ==
+                (((uint64_t)year << 46) | ((uint64_t)month << 42) | ((uint64_t)day << 37) |
+                 ((uint64_t)hour << 32) | ((uint64_t)minute << 26) | ((uint64_t)second << 20) |
+                 (uint64_t)microsecond));
+    EXPECT_TRUE(datetime_v2.hour() == hour);
+    EXPECT_TRUE(datetime_v2.minute() == minute);
+    EXPECT_TRUE(datetime_v2.second() == second);
+    EXPECT_TRUE(datetime_v2.microsecond() == microsecond);
+}
+
 TEST(VDateTimeValueTest, date_v2_from_uint32_test) {
     {
         uint16_t year = 2022;
         uint8_t month = 5;
         uint8_t day = 24;
 
-        DateV2Value date_v2;
+        DateV2Value<DateV2ValueType> date_v2;
         date_v2.from_date((uint32_t)((year << 9) | (month << 5) | day));
 
         EXPECT_TRUE(date_v2.year() == year);
         EXPECT_TRUE(date_v2.month() == month);
         EXPECT_TRUE(date_v2.day() == day);
-        EXPECT_TRUE(date_v2.to_date_uint32() == ((year << 9) | (month << 5) | day));
+        EXPECT_TRUE(date_v2.to_date_int_val() == ((year << 9) | (month << 5) | day));
         EXPECT_TRUE(date_v2.hour() == 0);
         EXPECT_TRUE(date_v2.minute() == 0);
         EXPECT_TRUE(date_v2.second() == 0);
@@ -64,25 +89,84 @@ TEST(VDateTimeValueTest, date_v2_from_uint32_test) {
         uint8_t day = 24;
 
         uint32_t ui32 = (uint32_t)((year << 9) | (month << 5) | day);
-        auto date_v2 = (DateV2Value&)ui32;
+        auto date_v2 = (DateV2Value<DateV2ValueType>&)ui32;
 
         EXPECT_TRUE(date_v2.year() == year);
         EXPECT_TRUE(date_v2.month() == month);
         EXPECT_TRUE(date_v2.day() == day);
-        EXPECT_TRUE(date_v2.to_date_uint32() == ((year << 9) | (month << 5) | day));
+        EXPECT_TRUE(date_v2.to_date_int_val() == ((year << 9) | (month << 5) | day));
         EXPECT_TRUE(date_v2.hour() == 0);
         EXPECT_TRUE(date_v2.minute() == 0);
         EXPECT_TRUE(date_v2.second() == 0);
     }
 }
 
+TEST(VDateTimeValueTest, datetime_v2_from_uint64_test) {
+    {
+        uint16_t year = 2022;
+        uint8_t month = 5;
+        uint8_t day = 24;
+        uint8_t hour = 23;
+        uint8_t minute = 0;
+        uint8_t second = 0;
+        uint32_t microsecond = 999999;
+
+        DateV2Value<DateTimeV2ValueType> datetime_v2;
+        datetime_v2.from_datetime((uint64_t)(((uint64_t)year << 46) | ((uint64_t)month << 42) |
+                                             ((uint64_t)day << 37) | ((uint64_t)hour << 32) |
+                                             ((uint64_t)minute << 26) | ((uint64_t)second << 20) |
+                                             (uint64_t)microsecond));
+
+        EXPECT_TRUE(datetime_v2.year() == year);
+        EXPECT_TRUE(datetime_v2.month() == month);
+        EXPECT_TRUE(datetime_v2.day() == day);
+        EXPECT_TRUE(datetime_v2.to_date_int_val() ==
+                    (uint64_t)(((uint64_t)year << 46) | ((uint64_t)month << 42) |
+                               ((uint64_t)day << 37) | ((uint64_t)hour << 32) |
+                               ((uint64_t)minute << 26) | ((uint64_t)second << 20) |
+                               (uint64_t)microsecond));
+        EXPECT_TRUE(datetime_v2.hour() == hour);
+        EXPECT_TRUE(datetime_v2.minute() == minute);
+        EXPECT_TRUE(datetime_v2.second() == second);
+        EXPECT_TRUE(datetime_v2.microsecond() == microsecond);
+    }
+    {
+        uint16_t year = 2022;
+        uint8_t month = 5;
+        uint8_t day = 24;
+        uint8_t hour = 12;
+        uint8_t minute = 0;
+        uint8_t second = 0;
+        uint32_t microsecond = 123000;
+
+        DateV2Value<DateTimeV2ValueType> datetime_v2;
+        datetime_v2.from_datetime((uint64_t)(((uint64_t)year << 46) | ((uint64_t)month << 42) |
+                                             ((uint64_t)day << 37) | ((uint64_t)hour << 32) |
+                                             ((uint64_t)minute << 26) | ((uint64_t)second << 20) |
+                                             (uint64_t)microsecond));
+
+        EXPECT_TRUE(datetime_v2.year() == year);
+        EXPECT_TRUE(datetime_v2.month() == month);
+        EXPECT_TRUE(datetime_v2.day() == day);
+        EXPECT_TRUE(datetime_v2.to_date_int_val() ==
+                    (uint64_t)(((uint64_t)year << 46) | ((uint64_t)month << 42) |
+                               ((uint64_t)day << 37) | ((uint64_t)hour << 32) |
+                               ((uint64_t)minute << 26) | ((uint64_t)second << 20) |
+                               (uint64_t)microsecond));
+        EXPECT_TRUE(datetime_v2.hour() == hour);
+        EXPECT_TRUE(datetime_v2.minute() == minute);
+        EXPECT_TRUE(datetime_v2.second() == second);
+        EXPECT_TRUE(datetime_v2.microsecond() == microsecond);
+    }
+}
+
 TEST(VDateTimeValueTest, date_v2_from_date_format_str_test) {
     uint16_t year = 2022;
     uint8_t month = 5;
     uint8_t day = 24;
 
     {
-        DateV2Value date_v2;
+        DateV2Value<DateV2ValueType> date_v2;
         std::string origin_date = "2022-05-24";
         std::string date_format = "%Y-%m-%d";
         EXPECT_TRUE(date_v2.from_date_format_str(date_format.data(), date_format.size(),
@@ -91,14 +175,14 @@ TEST(VDateTimeValueTest, date_v2_from_date_format_str_test) {
         EXPECT_TRUE(date_v2.year() == year);
         EXPECT_TRUE(date_v2.month() == month);
         EXPECT_TRUE(date_v2.day() == day);
-        EXPECT_TRUE(date_v2.to_date_uint32() == ((year << 9) | (month << 5) | day));
+        EXPECT_TRUE(date_v2.to_date_int_val() == ((year << 9) | (month << 5) | day));
         EXPECT_TRUE(date_v2.hour() == 0);
         EXPECT_TRUE(date_v2.minute() == 0);
         EXPECT_TRUE(date_v2.second() == 0);
     }
 
     {
-        DateV2Value date_v2;
+        DateV2Value<DateV2ValueType> date_v2;
         std::string origin_date = "2022-05-24 10:10:00";
         std::string date_format = "%Y-%m-%d";
         EXPECT_TRUE(date_v2.from_date_format_str(date_format.data(), date_format.size(),
@@ -107,22 +191,92 @@ TEST(VDateTimeValueTest, date_v2_from_date_format_str_test) {
         EXPECT_TRUE(date_v2.year() == year);
         EXPECT_TRUE(date_v2.month() == month);
         EXPECT_TRUE(date_v2.day() == day);
-        EXPECT_TRUE(date_v2.to_date_uint32() == ((year << 9) | (month << 5) | day));
+        EXPECT_TRUE(date_v2.to_date_int_val() == ((year << 9) | (month << 5) | day));
         EXPECT_TRUE(date_v2.hour() == 0);
         EXPECT_TRUE(date_v2.minute() == 0);
         EXPECT_TRUE(date_v2.second() == 0);
     }
 }
 
+TEST(VDateTimeValueTest, datetime_v2_from_date_format_str_test) {
+    uint16_t year = 2022;
+    uint8_t month = 5;
+    uint8_t day = 24;
+
+    {
+        DateV2Value<DateTimeV2ValueType> datetime_v2;
+        std::string origin_date = "2022-05-24 00:00:00";
+        std::string date_format = "%Y-%m-%d %H:%i:%s";
+        EXPECT_TRUE(datetime_v2.from_date_format_str(date_format.data(), date_format.size(),
+                                                     origin_date.data(), origin_date.size()));
+
+        EXPECT_TRUE(datetime_v2.year() == year);
+        EXPECT_TRUE(datetime_v2.month() == month);
+        EXPECT_TRUE(datetime_v2.day() == day);
+        EXPECT_TRUE(datetime_v2.hour() == 0);
+        EXPECT_TRUE(datetime_v2.minute() == 0);
+        EXPECT_TRUE(datetime_v2.second() == 0);
+        EXPECT_TRUE(datetime_v2.microsecond() == 0);
+    }
+
+    {
+        DateV2Value<DateTimeV2ValueType> datetime_v2;
+        std::string origin_date = "2022-05-24 00:00:00";
+        std::string date_format = "%Y-%m-%d %H:%i:%s.%f";
+        EXPECT_TRUE(datetime_v2.from_date_format_str(date_format.data(), date_format.size(),
+                                                     origin_date.data(), origin_date.size()));
+
+        EXPECT_TRUE(datetime_v2.year() == year);
+        EXPECT_TRUE(datetime_v2.month() == month);
+        EXPECT_TRUE(datetime_v2.day() == day);
+        EXPECT_TRUE(datetime_v2.hour() == 0);
+        EXPECT_TRUE(datetime_v2.minute() == 0);
+        EXPECT_TRUE(datetime_v2.second() == 0);
+        EXPECT_TRUE(datetime_v2.microsecond() == 0);
+    }
+
+    {
+        DateV2Value<DateTimeV2ValueType> datetime_v2;
+        std::string origin_date = "2022-05-24 00:00:00.123";
+        std::string date_format = "%Y-%m-%d %H:%i:%s.%f";
+        EXPECT_TRUE(datetime_v2.from_date_format_str(date_format.data(), date_format.size(),
+                                                     origin_date.data(), origin_date.size()));
+
+        EXPECT_TRUE(datetime_v2.year() == year);
+        EXPECT_TRUE(datetime_v2.month() == month);
+        EXPECT_TRUE(datetime_v2.day() == day);
+        EXPECT_TRUE(datetime_v2.hour() == 0);
+        EXPECT_TRUE(datetime_v2.minute() == 0);
+        EXPECT_TRUE(datetime_v2.second() == 0);
+        EXPECT_TRUE(datetime_v2.microsecond() == 123000);
+    }
+
+    {
+        DateV2Value<DateTimeV2ValueType> datetime_v2;
+        std::string origin_date = "2022-05-24 00:00:00.123456";
+        std::string date_format = "%Y-%m-%d %H:%i:%s.%f";
+        EXPECT_TRUE(datetime_v2.from_date_format_str(date_format.data(), date_format.size(),
+                                                     origin_date.data(), origin_date.size()));
+
+        EXPECT_TRUE(datetime_v2.year() == year);
+        EXPECT_TRUE(datetime_v2.month() == month);
+        EXPECT_TRUE(datetime_v2.day() == day);
+        EXPECT_TRUE(datetime_v2.hour() == 0);
+        EXPECT_TRUE(datetime_v2.minute() == 0);
+        EXPECT_TRUE(datetime_v2.second() == 0);
+        EXPECT_TRUE(datetime_v2.microsecond() == 123456);
+    }
+}
+
 TEST(VDateTimeValueTest, date_diff_test) {
     {
-        DateV2Value date_v2_1;
+        DateV2Value<DateV2ValueType> date_v2_1;
         std::string origin_date1 = "2022-05-24";
         std::string date_format1 = "%Y-%m-%d";
         EXPECT_TRUE(date_v2_1.from_date_format_str(date_format1.data(), date_format1.size(),
                                                    origin_date1.data(), origin_date1.size()));
 
-        DateV2Value date_v2_2;
+        DateV2Value<DateV2ValueType> date_v2_2;
         std::string origin_date2 = "2022-06-24";
         std::string date_format2 = "%Y-%m-%d";
         EXPECT_TRUE(date_v2_2.from_date_format_str(date_format2.data(), date_format2.size(),
@@ -137,7 +291,50 @@ TEST(VDateTimeValueTest, date_diff_test) {
     }
 
     {
-        DateV2Value date_v2_1;
+        DateV2Value<DateTimeV2ValueType> date_v2_1;
+        std::string origin_date1 = "2022-05-24 01:00:00";
+        std::string date_format1 = "%Y-%m-%d %H:%i:%s";
+        EXPECT_TRUE(date_v2_1.from_date_format_str(date_format1.data(), date_format1.size(),
+                                                   origin_date1.data(), origin_date1.size()));
+
+        DateV2Value<DateTimeV2ValueType> date_v2_2;
+        std::string origin_date2 = "2022-06-24 01:00:01";
+        std::string date_format2 = "%Y-%m-%d %H:%i:%s";
+        EXPECT_TRUE(date_v2_2.from_date_format_str(date_format2.data(), date_format2.size(),
+                                                   origin_date2.data(), origin_date2.size()));
+
+        EXPECT_TRUE(datetime_diff<TimeUnit::DAY>(date_v2_1, date_v2_2) == 31);
+        EXPECT_TRUE(datetime_diff<TimeUnit::YEAR>(date_v2_1, date_v2_2) == 0);
+        EXPECT_TRUE(datetime_diff<TimeUnit::MONTH>(date_v2_1, date_v2_2) == 1);
+        EXPECT_TRUE(datetime_diff<TimeUnit::HOUR>(date_v2_1, date_v2_2) == 31 * 24);
+        EXPECT_TRUE(datetime_diff<TimeUnit::MINUTE>(date_v2_1, date_v2_2) == 31 * 24 * 60);
+        EXPECT_TRUE(datetime_diff<TimeUnit::SECOND>(date_v2_1, date_v2_2) == 31 * 24 * 60 * 60 + 1);
+    }
+
+    {
+        DateV2Value<DateV2ValueType> date_v2_1;
+        std::string origin_date1 = "2022-05-24";
+        std::string date_format1 = "%Y-%m-%d";
+        EXPECT_TRUE(date_v2_1.from_date_format_str(date_format1.data(), date_format1.size(),
+                                                   origin_date1.data(), origin_date1.size()));
+
+        DateV2Value<DateTimeV2ValueType> date_v2_2;
+        std::string origin_date2 = "2022-06-24 01:00:01";
+        std::string date_format2 = "%Y-%m-%d %H:%i:%s";
+        EXPECT_TRUE(date_v2_2.from_date_format_str(date_format2.data(), date_format2.size(),
+                                                   origin_date2.data(), origin_date2.size()));
+
+        EXPECT_TRUE(datetime_diff<TimeUnit::DAY>(date_v2_1, date_v2_2) == 31);
+        EXPECT_TRUE(datetime_diff<TimeUnit::YEAR>(date_v2_1, date_v2_2) == 0);
+        EXPECT_TRUE(datetime_diff<TimeUnit::MONTH>(date_v2_1, date_v2_2) == 1);
+        EXPECT_TRUE(datetime_diff<TimeUnit::HOUR>(date_v2_1, date_v2_2) == 31 * 24 + 1);
+        EXPECT_TRUE(datetime_diff<TimeUnit::MINUTE>(date_v2_1, date_v2_2) == 31 * 24 * 60 + 60);
+        EXPECT_TRUE(datetime_diff<TimeUnit::SECOND>(date_v2_1, date_v2_2) ==
+                    31 * 24 * 60 * 60 + 3601);
+    }
+
+    {
+        DateV2Value<DateV2ValueType> date_v2_1;
         std::string origin_date1 = "2022-05-24";
         std::string date_format1 = "%Y-%m-%d";
         EXPECT_TRUE(date_v2_1.from_date_format_str(date_format1.data(), date_format1.size(),
@@ -157,6 +354,27 @@ TEST(VDateTimeValueTest, date_diff_test) {
         EXPECT_TRUE(datetime_diff<TimeUnit::SECOND>(date_v2_1, date_v2_2) == 31 * 24 * 60 * 60);
     }
 
+    {
+        DateV2Value<DateTimeV2ValueType> date_v2_1;
+        std::string origin_date1 = "2022-05-24 00:00:00.000";
+        std::string date_format1 = "%Y-%m-%d %H:%i:%s.%f";
+        EXPECT_TRUE(date_v2_1.from_date_format_str(date_format1.data(), date_format1.size(),
+                                                   origin_date1.data(), origin_date1.size()));
+
+        VecDateTimeValue date_v2_2;
+        std::string origin_date2 = "2022-06-24 00:00:00";
+        std::string date_format2 = "%Y-%m-%d %H:%i:%s";
+        EXPECT_TRUE(date_v2_2.from_date_format_str(date_format2.data(), date_format2.size(),
+                                                   origin_date2.data(), origin_date2.size()));
+
+        EXPECT_TRUE(datetime_diff<TimeUnit::DAY>(date_v2_1, date_v2_2) == 31);
+        EXPECT_TRUE(datetime_diff<TimeUnit::YEAR>(date_v2_1, date_v2_2) == 0);
+        EXPECT_TRUE(datetime_diff<TimeUnit::MONTH>(date_v2_1, date_v2_2) == 1);
+        EXPECT_TRUE(datetime_diff<TimeUnit::HOUR>(date_v2_1, date_v2_2) == 31 * 24);
+        EXPECT_TRUE(datetime_diff<TimeUnit::MINUTE>(date_v2_1, date_v2_2) == 31 * 24 * 60);
+        EXPECT_TRUE(datetime_diff<TimeUnit::SECOND>(date_v2_1, date_v2_2) == 31 * 24 * 60 * 60);
+    }
+
     {
         VecDateTimeValue date_v2_1;
         std::string origin_date1 = "2022-05-24 00:00:00";
@@ -164,7 +382,7 @@ TEST(VDateTimeValueTest, date_diff_test) {
         EXPECT_TRUE(date_v2_1.from_date_format_str(date_format1.data(), date_format1.size(),
                                                    origin_date1.data(), origin_date1.size()));
 
-        DateV2Value date_v2_2;
+        DateV2Value<DateV2ValueType> date_v2_2;
         std::string origin_date2 = "2022-06-24";
         std::string date_format2 = "%Y-%m-%d";
         EXPECT_TRUE(date_v2_2.from_date_format_str(date_format2.data(), date_format2.size(),
@@ -179,7 +397,28 @@ TEST(VDateTimeValueTest, date_diff_test) {
     }
 
     {
-        DateV2Value date_v2_1;
+        VecDateTimeValue date_v2_1;
+        std::string origin_date1 = "2022-05-24 00:00:00";
+        std::string date_format1 = "%Y-%m-%d %H:%i:%s";
+        EXPECT_TRUE(date_v2_1.from_date_format_str(date_format1.data(), date_format1.size(),
+                                                   origin_date1.data(), origin_date1.size()));
+
+        DateV2Value<DateTimeV2ValueType> date_v2_2;
+        std::string origin_date2 = "2022-06-24 00:00:00.000000";
+        std::string date_format2 = "%Y-%m-%d %H:%i:%s.%f";
+        EXPECT_TRUE(date_v2_2.from_date_format_str(date_format2.data(), date_format2.size(),
+                                                   origin_date2.data(), origin_date2.size()));
+
+        EXPECT_TRUE(datetime_diff<TimeUnit::DAY>(date_v2_1, date_v2_2) == 31);
+        EXPECT_TRUE(datetime_diff<TimeUnit::YEAR>(date_v2_1, date_v2_2) == 0);
+        EXPECT_TRUE(datetime_diff<TimeUnit::MONTH>(date_v2_1, date_v2_2) == 1);
+        EXPECT_TRUE(datetime_diff<TimeUnit::HOUR>(date_v2_1, date_v2_2) == 31 * 24);
+        EXPECT_TRUE(datetime_diff<TimeUnit::MINUTE>(date_v2_1, date_v2_2) == 31 * 24 * 60);
+        EXPECT_TRUE(datetime_diff<TimeUnit::SECOND>(date_v2_1, date_v2_2) == 31 * 24 * 60 * 60);
+    }
+
+    {
+        DateV2Value<DateV2ValueType> date_v2_1;
         std::string origin_date1 = "2022-05-24";
         std::string date_format1 = "%Y-%m-%d";
         EXPECT_TRUE(date_v2_1.from_date_format_str(date_format1.data(), date_format1.size(),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java
index 68d91c1215..e7849fa20f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java
@@ -58,6 +58,7 @@ import org.apache.doris.rewrite.RewriteEncryptKeyRule;
 import org.apache.doris.rewrite.RewriteFromUnixTimeRule;
 import org.apache.doris.rewrite.RewriteImplicitCastRule;
 import org.apache.doris.rewrite.RewriteInPredicateRule;
+import org.apache.doris.rewrite.RoundLiteralInBinaryPredicatesRule;
 import org.apache.doris.rewrite.mvrewrite.CountDistinctToBitmap;
 import org.apache.doris.rewrite.mvrewrite.CountDistinctToBitmapOrHLLRule;
 import org.apache.doris.rewrite.mvrewrite.CountFieldToSum;
@@ -354,6 +355,7 @@ public class Analyzer {
             // Put it after NormalizeBinaryPredicatesRule, make sure slotRef is on the left and Literal is on the right.
             rules.add(RewriteBinaryPredicatesRule.INSTANCE);
             rules.add(RewriteImplicitCastRule.INSTANCE);
+            rules.add(RoundLiteralInBinaryPredicatesRule.INSTANCE);
             rules.add(FoldConstantsRule.INSTANCE);
             rules.add(RewriteFromUnixTimeRule.INSTANCE);
             rules.add(CompoundPredicateWriteRule.INSTANCE);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java
index e11ebe4d4b..154e724f4f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java
@@ -24,6 +24,7 @@ import org.apache.doris.catalog.Function;
 import org.apache.doris.catalog.FunctionSet;
 import org.apache.doris.catalog.PrimitiveType;
 import org.apache.doris.catalog.ScalarFunction;
+import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.catalog.Type;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Pair;
@@ -306,6 +307,18 @@ public class BinaryPredicate extends Predicate implements Writable {
         }
     }
 
+    private Type dateV2ComparisonResultType(ScalarType t1, ScalarType t2) {
+        if (!t1.isDatetimeV2() && !t2.isDatetimeV2()) {
+            return Type.DATETIMEV2;
+        } else if (t1.isDatetimeV2() && t2.isDatetimeV2()) {
+            return ScalarType.createDatetimeV2Type(Math.max(t1.getScalarScale(), t2.getScalarScale()));
+        } else if (t1.isDatetimeV2()) {
+            return t1;
+        } else {
+            return t2;
+        }
+    }
+
     private Type getCmpType() throws AnalysisException {
         PrimitiveType t1 = getChild(0).getType().getResultType().getPrimitiveType();
         PrimitiveType t2 = getChild(1).getType().getResultType().getPrimitiveType();
@@ -320,7 +333,15 @@ public class BinaryPredicate extends Predicate implements Writable {
         }
 
         if (canCompareDate(getChild(0).getType().getPrimitiveType(), getChild(1).getType().getPrimitiveType())) {
-            return Type.DATETIME;
+            if (!(getChild(0).getType().isDateV2() || getChild(0).getType().isDatetimeV2()
+                    || getChild(1).getType().isDateV2() || getChild(1).getType().isDatetimeV2())) {
+                return Type.DATETIME;
+            } else {
+                Preconditions.checkArgument(getChild(0).getType() instanceof ScalarType
+                        && getChild(1).getType() instanceof ScalarType);
+                return dateV2ComparisonResultType((ScalarType) getChild(0).getType(),
+                        (ScalarType) getChild(1).getType());
+            }
         }
 
         // Following logical is compatible with MySQL:
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 26c2b473e9..46fe04d7e5 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
@@ -173,6 +173,9 @@ public class CastExpr extends Expr {
                 if (toType.getPrimitiveType() == PrimitiveType.DATEV2) {
                     typeName = "datev2_val";
                 }
+                if (toType.getPrimitiveType() == PrimitiveType.DATETIMEV2) {
+                    typeName = "datetimev2_val";
+                }
                 String beSymbol = "doris::" + beClass + "::cast_to_"
                         + typeName;
                 functionSet.addBuiltinBothScalaAndVectorized(ScalarFunction.createBuiltin(getFnName(toType),
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 cc160c989b..72c566067a 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
@@ -397,7 +397,11 @@ public class DateLiteral extends LiteralExpr {
             minute = getOrDefault(dateTime, ChronoField.MINUTE_OF_HOUR, 0);
             second = getOrDefault(dateTime, ChronoField.SECOND_OF_MINUTE, 0);
             microsecond = getOrDefault(dateTime, ChronoField.MICRO_OF_SECOND, 0);
-            this.type = type;
+            if (type.isDatetimeV2()) {
+                this.type = ScalarType.createDatetimeV2Type(6);
+            } else {
+                this.type = type;
+            }
         } catch (Exception ex) {
             throw new AnalysisException("date literal [" + s + "] is invalid: " + ex.getMessage());
         }
@@ -489,16 +493,29 @@ public class DateLiteral extends LiteralExpr {
         if (type.isDate() || type.isDateV2()) {
             return String.format("%04d-%02d-%02d", year, month, day);
         } else if (type.isDatetimeV2()) {
-            String s = String.format("%04d-%02d-%02d %02d:%02d:%02d", year, month, day, hour, minute, second);
-            if (((ScalarType) type).decimalScale() == 0) {
-                return s;
-            }
-            return s + "." + getDecimalNumber();
+            return String.format("%04d-%02d-%02d %02d:%02d:%02d.%06d",
+                    year, month, day, hour, minute, second, microsecond);
         } else {
             return String.format("%04d-%02d-%02d %02d:%02d:%02d", year, month, day, hour, minute, second);
         }
     }
 
+    public void roundCeiling(int newScale) {
+        Preconditions.checkArgument(type.isDatetimeV2());
+        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();
+        }
+        type = ScalarType.createDatetimeV2Type(newScale);
+    }
+
+    public void roundFloor(int newScale) {
+        Preconditions.checkArgument(type.isDatetimeV2());
+        microsecond = Double.valueOf(microsecond / (Math.pow(10, 6 - newScale))).longValue();
+        type = ScalarType.createDatetimeV2Type(newScale);
+    }
+
     public long getDecimalNumber() {
         return Double.valueOf(microsecond / (Math.pow(10, 6 - ((ScalarType) type).decimalScale()))).longValue();
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Function.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Function.java
index 3d01355ca0..0411ba998d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Function.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Function.java
@@ -519,10 +519,11 @@ public class Function implements Writable {
                 return "string_val";
             case DATE:
             case DATETIME:
-            case DATETIMEV2:
                 return "datetime_val";
             case DATEV2:
                 return "datev2_val";
+            case DATETIMEV2:
+                return "datetimev2_val";
             case DECIMALV2:
                 return "decimalv2_val";
             case DECIMAL32:
@@ -568,10 +569,11 @@ public class Function implements Writable {
                 return "StringVal";
             case DATE:
             case DATETIME:
-            case DATETIMEV2:
                 return "DateTimeVal";
             case DATEV2:
                 return "DateV2Val";
+            case DATETIMEV2:
+                return "DateTimeV2Val";
             case DECIMALV2:
                 return "DecimalV2Val";
             case DECIMAL32:
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 a30a44dc3f..cf1bce79a7 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
@@ -1388,19 +1388,6 @@ public class FunctionSet<T> {
                 "",
                 "",
                 true, false, true, true));
-        addBuiltin(AggregateFunction.createBuiltin(FunctionSet.WINDOW_FUNNEL,
-                Lists.newArrayList(Type.BIGINT, Type.STRING, Type.DATEV2, Type.BOOLEAN),
-                Type.INT,
-                Type.VARCHAR,
-                true,
-                "",
-                "",
-                "",
-                "",
-                "",
-                "",
-                "",
-                true, false, true, true));
 
         for (Type t : Type.getSupportedTypes()) {
             if (t.isNull()) {
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 d59adf9a57..6fb59c20c6 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
@@ -440,15 +440,30 @@ public class ScalarType extends Type {
                 return "CHAR(*)";
             }
             return "CHAR(" + len + ")";
-        } else  if (type == PrimitiveType.DECIMALV2 || type.isDecimalV3Type()) {
+        } else  if (type == PrimitiveType.DECIMALV2) {
             if (isWildcardDecimal()) {
                 return "DECIMAL(*,*)";
             }
             return "DECIMAL(" + precision + "," + scale + ")";
+        } else  if (type == PrimitiveType.DECIMAL32) {
+            if (isWildcardDecimal()) {
+                return "DECIMAL32(*,*)";
+            }
+            return "DECIMAL32(" + precision + "," + scale + ")";
+        } else  if (type == PrimitiveType.DECIMAL64) {
+            if (isWildcardDecimal()) {
+                return "DECIMAL64(*,*)";
+            }
+            return "DECIMAL64(" + precision + "," + scale + ")";
+        } else  if (type == PrimitiveType.DECIMAL128) {
+            if (isWildcardDecimal()) {
+                return "DECIMAL128(*,*)";
+            }
+            return "DECIMAL128(" + precision + "," + scale + ")";
         } else  if (type == PrimitiveType.DATETIMEV2) {
-            return "Datetime(" + scale + ")";
+            return "DATETIMEV2(" + scale + ")";
         } else  if (type == PrimitiveType.TIMEV2) {
-            return "Time(" + scale + ")";
+            return "TIMEV2(" + scale + ")";
         } else if (type == PrimitiveType.VARCHAR) {
             if (isWildcardVarchar()) {
                 return "VARCHAR(*)";
@@ -557,7 +572,8 @@ public class ScalarType extends Type {
             case DECIMALV2:
             case DECIMAL32:
             case DECIMAL64:
-            case DECIMAL128: {
+            case DECIMAL128:
+            case DATETIMEV2: {
                 scalarType.setScale(scale);
                 scalarType.setPrecision(precision);
                 break;
@@ -751,10 +767,6 @@ public class ScalarType extends Type {
         if (type.isDecimalV2Type() || type == PrimitiveType.DATETIMEV2 || type == PrimitiveType.TIMEV2) {
             return precision == other.precision && scale == other.scale;
         }
-        if (type == PrimitiveType.DATETIMEV2 || type == PrimitiveType.TIMEV2) {
-            return precision == other.precision && scale == other.scale
-                    && type == ((ScalarType) o).getPrimitiveType();
-        }
         return true;
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/rewrite/RewriteBinaryPredicatesRule.java b/fe/fe-core/src/main/java/org/apache/doris/rewrite/RewriteBinaryPredicatesRule.java
index 2f8b9603e9..a18797b657 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/rewrite/RewriteBinaryPredicatesRule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/rewrite/RewriteBinaryPredicatesRule.java
@@ -27,12 +27,9 @@ import org.apache.doris.analysis.Expr;
 import org.apache.doris.analysis.IntLiteral;
 import org.apache.doris.analysis.LiteralExpr;
 import org.apache.doris.analysis.SlotRef;
-import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.catalog.Type;
 import org.apache.doris.common.AnalysisException;
 
-import java.math.BigDecimal;
-
 /**
  * Rewrite binary predicate.
  */
@@ -116,56 +113,6 @@ public class RewriteBinaryPredicatesRule implements ExprRewriteRule {
         }
     }
 
-    private Expr rewriteDecimalLiteral(Expr expr) {
-        BinaryPredicate.Operator op = ((BinaryPredicate) expr).getOp();
-        Expr expr0 = expr.getChild(0);
-        Expr expr1 = expr.getChild(1);
-        if (expr1.getType().isDecimalV3() && expr1 instanceof DecimalLiteral) {
-            DecimalLiteral literal = (DecimalLiteral) expr1;
-            if (expr0.getType().isDecimalV3()
-                    && ((ScalarType) expr0.getType()).getScalarScale()
-                    < ((ScalarType) expr1.getType()).getScalarScale()) {
-                switch (op) {
-                    case EQ: {
-                        BigDecimal originValue = literal.getValue();
-                        literal.roundCeiling(((ScalarType) expr0.getType()).getScalarScale());
-                        if (literal.getValue().equals(originValue)) {
-                            expr.setChild(1, literal);
-                            return expr;
-                        } else {
-                            return new BoolLiteral(false);
-                        }
-                    }
-                    case NE: {
-                        BigDecimal originValue = literal.getValue();
-                        literal.roundCeiling(((ScalarType) expr0.getType()).getScalarScale());
-                        if (literal.getValue().equals(originValue)) {
-                            expr.setChild(1, literal);
-                            return expr;
-                        } else {
-                            return new BoolLiteral(true);
-                        }
-                    }
-                    case GT:
-                    case LE: {
-                        literal.roundFloor(((ScalarType) expr0.getType()).getScalarScale());
-                        expr.setChild(1, literal);
-                        return expr;
-                    }
-                    case LT:
-                    case GE: {
-                        literal.roundCeiling(((ScalarType) expr0.getType()).getScalarScale());
-                        expr.setChild(1, literal);
-                        return expr;
-                    }
-                    default:
-                        return expr;
-                }
-            }
-        }
-        return expr;
-    }
-
     @Override
     public Expr apply(Expr expr, Analyzer analyzer, ExprRewriter.ClauseType clauseType) throws AnalysisException {
         if (!(expr instanceof BinaryPredicate)) {
@@ -174,11 +121,10 @@ public class RewriteBinaryPredicatesRule implements ExprRewriteRule {
         BinaryPredicate.Operator op = ((BinaryPredicate) expr).getOp();
         Expr expr0 = expr.getChild(0);
         Expr expr1 = expr.getChild(1);
-        if (expr0 instanceof CastExpr && (expr0.getType().isDecimalV2() || expr0.getType().isDecimalV3())
-                && expr0.getChild(0) instanceof SlotRef && expr0.getChild(0).getType().getResultType()
-                == Type.BIGINT && expr1 instanceof DecimalLiteral) {
+        if (expr0 instanceof CastExpr && expr0.getType() == Type.DECIMALV2 && expr0.getChild(0) instanceof SlotRef
+                && expr0.getChild(0).getType().getResultType() == Type.BIGINT && expr1 instanceof DecimalLiteral) {
             return rewriteBigintSlotRefCompareDecimalLiteral(expr0, (DecimalLiteral) expr1, op);
         }
-        return rewriteDecimalLiteral(expr);
+        return expr;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/rewrite/RoundLiteralInBinaryPredicatesRule.java b/fe/fe-core/src/main/java/org/apache/doris/rewrite/RoundLiteralInBinaryPredicatesRule.java
new file mode 100644
index 0000000000..8c0306f8e6
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/rewrite/RoundLiteralInBinaryPredicatesRule.java
@@ -0,0 +1,145 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.rewrite;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.BinaryPredicate;
+import org.apache.doris.analysis.BinaryPredicate.Operator;
+import org.apache.doris.analysis.BoolLiteral;
+import org.apache.doris.analysis.DateLiteral;
+import org.apache.doris.analysis.DecimalLiteral;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.catalog.ScalarType;
+import org.apache.doris.common.AnalysisException;
+
+import java.math.BigDecimal;
+
+/**
+ * Rewrite binary predicate.
+ */
+public class RoundLiteralInBinaryPredicatesRule implements ExprRewriteRule {
+    public static ExprRewriteRule INSTANCE = new RoundLiteralInBinaryPredicatesRule();
+
+    private Expr rewriteDecimalLiteral(Expr expr) {
+        Operator op = ((BinaryPredicate) expr).getOp();
+        Expr expr0 = expr.getChild(0);
+        Expr expr1 = expr.getChild(1);
+        if (expr1.getType().isDecimalV3() && expr1 instanceof DecimalLiteral) {
+            DecimalLiteral literal = (DecimalLiteral) expr1;
+            if (expr0.getType().isDecimalV3()
+                    && ((ScalarType) expr0.getType()).getScalarScale()
+                    < ((ScalarType) expr1.getType()).getScalarScale()) {
+                switch (op) {
+                    case EQ: {
+                        BigDecimal originValue = literal.getValue();
+                        literal.roundCeiling(((ScalarType) expr0.getType()).getScalarScale());
+                        if (literal.getValue().equals(originValue)) {
+                            expr.setChild(1, literal);
+                            return expr;
+                        } else {
+                            return new BoolLiteral(false);
+                        }
+                    }
+                    case NE: {
+                        BigDecimal originValue = literal.getValue();
+                        literal.roundCeiling(((ScalarType) expr0.getType()).getScalarScale());
+                        if (literal.getValue().equals(originValue)) {
+                            expr.setChild(1, literal);
+                            return expr;
+                        } else {
+                            return new BoolLiteral(true);
+                        }
+                    }
+                    case GT:
+                    case LE: {
+                        literal.roundFloor(((ScalarType) expr0.getType()).getScalarScale());
+                        expr.setChild(1, literal);
+                        return expr;
+                    }
+                    case LT:
+                    case GE: {
+                        literal.roundCeiling(((ScalarType) expr0.getType()).getScalarScale());
+                        expr.setChild(1, literal);
+                        return expr;
+                    }
+                    default:
+                        return expr;
+                }
+            }
+        }
+        return expr;
+    }
+
+    private Expr rewriteDateLiteral(Expr expr) {
+        Operator op = ((BinaryPredicate) expr).getOp();
+        Expr expr0 = expr.getChild(0);
+        Expr expr1 = expr.getChild(1);
+        if (expr0.getType().isDatetimeV2() && expr1 instanceof DateLiteral && expr1.getType().isDatetimeV2()) {
+            DateLiteral literal = (DateLiteral) expr1;
+            if (((ScalarType) expr0.getType()).getScalarScale()
+                    < ((ScalarType) expr1.getType()).getScalarScale()) {
+                switch (op) {
+                    case EQ: {
+                        long originValue = literal.getMicrosecond();
+                        literal.roundCeiling(((ScalarType) expr0.getType()).getScalarScale());
+                        if (literal.getMicrosecond() == originValue) {
+                            expr.setChild(1, literal);
+                            return expr;
+                        } else {
+                            return new BoolLiteral(false);
+                        }
+                    }
+                    case NE: {
+                        long originValue = literal.getMicrosecond();
+                        literal.roundCeiling(((ScalarType) expr0.getType()).getScalarScale());
+                        if (literal.getMicrosecond() == originValue) {
+                            expr.setChild(1, literal);
+                            return expr;
+                        } else {
+                            return new BoolLiteral(true);
+                        }
+                    }
+                    case GT:
+                    case LE: {
+                        literal.roundFloor(((ScalarType) expr0.getType()).getScalarScale());
+                        expr.setChild(1, literal);
+                        return expr;
+                    }
+                    case LT:
+                    case GE: {
+                        literal.roundCeiling(((ScalarType) expr0.getType()).getScalarScale());
+                        expr.setChild(1, literal);
+                        return expr;
+                    }
+                    default:
+                        return expr;
+                }
+            }
+        }
+        return expr;
+    }
+
+    @Override
+    public Expr apply(Expr expr, Analyzer analyzer, ExprRewriter.ClauseType clauseType) throws AnalysisException {
+        if (!(expr instanceof BinaryPredicate)) {
+            return expr;
+        }
+        Expr tmpExpr = rewriteDecimalLiteral(expr);
+        return rewriteDateLiteral(tmpExpr);
+    }
+}
diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto
index 28fb5c4aec..e4937fbc7d 100644
--- a/gensrc/proto/internal_service.proto
+++ b/gensrc/proto/internal_service.proto
@@ -381,9 +381,10 @@ enum PColumnType {
     COLUMN_TYPE_DECIMALV2 = 13;
     COLUMN_TYPE_STRING = 14;
     COLUMN_TYPE_DATEV2 = 15;
-    COLUMN_TYPE_DECIMAL32 = 16;
-    COLUMN_TYPE_DECIMAL64 = 17;
-    COLUMN_TYPE_DECIMAL128 = 18;
+    COLUMN_TYPE_DATETIMEV2 = 16;
+    COLUMN_TYPE_DECIMAL32 = 17;
+    COLUMN_TYPE_DECIMAL64 = 18;
+    COLUMN_TYPE_DECIMAL128 = 19;
 }
 
 message PMinMaxFilter {


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