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

dataroaring pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/branch-3.0 by this push:
     new eb39ad368f2 [Enhancement-3.0](type) Support cast between time and 
datetime (#53734)
eb39ad368f2 is described below

commit eb39ad368f25cbe397c71724c9147d5162e3f643
Author: zclllyybb <[email protected]>
AuthorDate: Wed Aug 13 10:01:23 2025 +0800

    [Enhancement-3.0](type) Support cast between time and datetime (#53734)
---
 be/src/vec/core/call_on_type_index.h               |   3 +
 be/src/vec/functions/function_cast.h               | 152 +++++++++++++++------
 be/src/vec/runtime/time_value.h                    |  46 +++++++
 be/src/vec/runtime/vdatetime_value.h               |  15 ++
 be/test/runtime/time_value_test.cpp                |  50 +++++++
 .../org/apache/doris/catalog/PrimitiveType.java    |  11 +-
 .../nereids/rules/expression/check/CheckCast.java  |   3 +-
 .../correctness/test_cast_time_to_datetime.out     | Bin 0 -> 197 bytes
 .../correctness/test_cast_time_to_datetime.groovy  |  27 ++++
 .../suites/nereids_syntax_p0/cast.groovy           |   5 +-
 10 files changed, 261 insertions(+), 51 deletions(-)

diff --git a/be/src/vec/core/call_on_type_index.h 
b/be/src/vec/core/call_on_type_index.h
index b651fe2ec47..e64e2003cb5 100644
--- a/be/src/vec/core/call_on_type_index.h
+++ b/be/src/vec/core/call_on_type_index.h
@@ -276,6 +276,9 @@ bool call_on_index_and_number_data_type(TypeIndex number, 
F&& f) {
     case TypeIndex::Float64:
         return f(TypePair<DataTypeNumber<Float64>, T>());
 
+    case TypeIndex::TimeV2:
+        return f(TypePair<DataTypeTimeV2, T>());
+
     case TypeIndex::Decimal32:
         return f(TypePair<DataTypeDecimal<Decimal32>, T>());
     case TypeIndex::Decimal64:
diff --git a/be/src/vec/functions/function_cast.h 
b/be/src/vec/functions/function_cast.h
index 0e7db6e407b..122587c4dc1 100644
--- a/be/src/vec/functions/function_cast.h
+++ b/be/src/vec/functions/function_cast.h
@@ -24,16 +24,13 @@
 #include <fmt/format.h>
 #include <gen_cpp/FrontendService_types.h>
 #include <glog/logging.h>
-#include <stddef.h>
-#include <stdint.h>
 
 #include <algorithm>
-#include <atomic>
 #include <boost/iterator/iterator_facade.hpp>
 #include <cmath>
+#include <cstddef>
 #include <cstdint>
 #include <functional>
-#include <iterator>
 #include <limits>
 #include <memory>
 #include <ostream>
@@ -91,22 +88,16 @@
 #include "vec/data_types/data_type_time_v2.h"
 #include "vec/data_types/serde/data_type_serde.h"
 #include "vec/functions/function.h"
-#include "vec/functions/function_convert_tz.h"
 #include "vec/functions/function_helpers.h"
 #include "vec/io/reader_buffer.h"
+#include "vec/runtime/time_value.h"
 #include "vec/runtime/vdatetime_value.h"
-#include "vec/utils/util.hpp"
 
 class DateLUTImpl;
 
-namespace doris {
-namespace vectorized {
+namespace doris::vectorized {
 template <typename T>
 class ColumnDecimal;
-} // namespace vectorized
-} // namespace doris
-
-namespace doris::vectorized {
 /** Type conversion functions.
   * toType - conversion in "natural way";
   */
@@ -423,11 +414,58 @@ struct ConvertImpl {
                         }
                     } else {
                         if constexpr (IsDateTimeV2Type<FromDataType>) {
-                            static_cast_set(
-                                    vec_to[i],
-                                    reinterpret_cast<const 
DateV2Value<DateTimeV2ValueType>&>(
-                                            vec_from[i])
-                                            .to_int64());
+                            if constexpr (std::is_same_v<ToDataType, 
DataTypeTimeV2>) {
+                                // datetimev2 to timev2
+                                auto dtmv2 = binary_cast<UInt64, 
DateV2Value<DateTimeV2ValueType>>(
+                                        col_from->get_data()[i]);
+
+                                const auto* type = assert_cast<const 
DataTypeDateTimeV2*>(
+                                        
block.get_by_position(arguments[0]).type.get());
+                                auto scale = type->get_scale();
+                                const auto* to_type = assert_cast<const 
DataTypeTimeV2*>(
+                                        
block.get_by_position(result).type.get());
+                                UInt32 to_scale = to_type->get_scale();
+
+                                uint32_t hour = dtmv2.hour();
+                                uint32_t minute = dtmv2.minute();
+                                uint32_t second = dtmv2.second();
+                                uint32_t microseconds = dtmv2.microsecond();
+                                if (to_scale < scale) { // need to round
+                                    // e.g. scale reduce to 4, means we need 
to round the last 2 digits
+                                    // 999956: 56 > 100/2, then round up to 
1000000
+                                    uint32_t divisor = common::exp10_i64(6 - 
to_scale);
+                                    uint32_t remainder = microseconds % 
divisor;
+                                    microseconds = (microseconds / divisor) * 
divisor;
+                                    if (remainder >= divisor / 2) {
+                                        // do rounding up
+                                        microseconds += divisor;
+                                    }
+                                }
+
+                                // carry on if microseconds >= 1000000
+                                if (microseconds >= 1000000) {
+                                    microseconds -= 1000000;
+                                    second += 1;
+                                    if (second >= 60) {
+                                        second -= 60;
+                                        minute += 1;
+                                        if (minute >= 60) {
+                                            minute -= 60;
+                                            hour += 1;
+                                        }
+                                    }
+                                }
+
+                                auto time = TimeValue::limit_with_bound(
+                                        TimeValue::make_time(hour, minute, 
second, microseconds));
+                                col_to->get_data()[i] = time;
+                            } else {
+                                static_cast_set(
+                                        vec_to[i],
+                                        reinterpret_cast<const 
DateV2Value<DateTimeV2ValueType>&>(
+                                                vec_from[i])
+                                                .to_int64());
+                            }
                         } else {
                             static_cast_set(vec_to[i],
                                             reinterpret_cast<const 
DateV2Value<DateV2ValueType>&>(
@@ -498,8 +536,8 @@ struct ConvertImplToTimeType {
     using FromFieldType = typename FromDataType::FieldType;
     using ToFieldType = typename ToDataType::FieldType;
 
-    static Status execute(Block& block, const ColumnNumbers& arguments, size_t 
result,
-                          size_t /*input_rows_count*/) {
+    static Status execute(FunctionContext* context, Block& block, const 
ColumnNumbers& arguments,
+                          uint32_t result, size_t /*input_rows_count*/) {
         const ColumnWithTypeAndName& named_from = 
block.get_by_position(arguments[0]);
 
         using ColVecFrom =
@@ -527,29 +565,53 @@ struct ConvertImplToTimeType {
             col_null_map_to = ColumnUInt8::create(size, 0);
             auto& vec_null_map_to = col_null_map_to->get_data();
 
-            UInt32 from_precision = 0;
-            UInt32 from_scale = 0;
-            UInt32 to_precision = NumberTraits::max_ascii_len<Int64>();
-            if constexpr (IsDecimalNumber<FromFieldType>) {
-                const auto& from_decimal_type = assert_cast<const 
FromDataType&>(*named_from.type);
-                from_precision = from_decimal_type.get_precision();
-                from_scale = from_decimal_type.get_scale();
+            if constexpr (std::is_same_v<FromDataType, DataTypeTimeV2>) {
+                DateValueType current_date_value;
+                
current_date_value.from_unixtime(context->state()->timestamp_ms() / 1000,
+                                                 
context->state()->timezone_obj());
+                uint32_t scale = 0;
+                // Only DateTimeV2 has scale
+                if (std::is_same_v<ToDataType, DataTypeDateTimeV2>) {
+                    scale = 
remove_nullable(block.get_by_position(result).type)->get_scale();
+                }
+                // According to MySQL rules, when casting time type to 
date/datetime,
+                // the current date is added to the time
+                // So here we need to clear the time part
+                current_date_value.reset_time_part();
+                for (size_t i = 0; i < size; ++i) {
+                    auto& date_value = 
reinterpret_cast<DateValueType&>(vec_to[i]);
+                    date_value = current_date_value;
+                    int64_t microsecond = TimeValue::round_time(vec_from[i], 
scale);
+                    // Only TimeV2 type needs microseconds
+                    if constexpr (IsTimeV2Type<ToDataType>) {
+                        vec_null_map_to[i] = !date_value.template 
date_add_interval<MICROSECOND>(
+                                TimeInterval {MICROSECOND, microsecond, 
false});
+                    } else {
+                        vec_null_map_to[i] =
+                                !date_value.template 
date_add_interval<SECOND>(TimeInterval {
+                                        SECOND, microsecond / 
TimeValue::ONE_SECOND_MICROSECONDS,
+                                        false});
+                    }
+
+                    // DateType of VecDateTimeValue should cast to date
+                    if constexpr (IsDateType<ToDataType>) {
+                        date_value.cast_to_date();
+                    } else if constexpr (IsDateTimeType<ToDataType>) {
+                        date_value.to_datetime();
+                    }
+                }
+            } else {
+                for (size_t i = 0; i < size; ++i) {
+                    auto& date_value = 
reinterpret_cast<DateValueType&>(vec_to[i]);
+                    vec_null_map_to[i] = 
!date_value.from_date_int64(int64_t(vec_from[i]));
+                    // DateType of VecDateTimeValue should cast to date
+                    if constexpr (IsDateType<ToDataType>) {
+                        date_value.cast_to_date();
+                    } else if constexpr (IsDateTimeType<ToDataType>) {
+                        date_value.to_datetime();
+                    }
+                }
             }
-            bool narrow_integral = to_precision < (from_precision - 
from_scale);
-            std::visit(
-                    [&](auto narrow_integral) {
-                        for (size_t i = 0; i < size; ++i) {
-                            auto& date_value = 
reinterpret_cast<DateValueType&>(vec_to[i]);
-                            vec_null_map_to[i] = 
!date_value.from_date_int64(int64_t(vec_from[i]));
-                            // DateType of VecDateTimeValue should cast to date
-                            if constexpr (IsDateType<ToDataType>) {
-                                date_value.cast_to_date();
-                            } else if constexpr (IsDateTimeType<ToDataType>) {
-                                date_value.to_datetime();
-                            }
-                        }
-                    },
-                    make_bool_variant(narrow_integral));
             block.get_by_position(result).column =
                     ColumnNullable::create(std::move(col_to), 
std::move(col_null_map_to));
         } else {
@@ -1687,10 +1749,10 @@ public:
 };
 
 template <typename ToDataType, typename Name>
-class FunctionConvertToTimeType : public IFunction {
+class FunctionConvertFromDatelikeType : public IFunction {
 public:
     static constexpr auto name = Name::name;
-    static FunctionPtr create() { return 
std::make_shared<FunctionConvertToTimeType>(); }
+    static FunctionPtr create() { return 
std::make_shared<FunctionConvertFromDatelikeType>(); }
 
     String get_name() const override { return name; }
 
@@ -1715,7 +1777,7 @@ public:
             using RightDataType = typename Types::RightType;
 
             ret_status = ConvertImplToTimeType<LeftDataType, RightDataType, 
Name>::execute(
-                    block, arguments, result, input_rows_count);
+                    context, block, arguments, result, input_rows_count);
             return true;
         };
 
@@ -1788,7 +1850,7 @@ private:
                            
check_and_get_data_type<DataTypeDate>(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();
+            function = FunctionConvertFromDatelikeType<DataType, 
NameCast>::create();
         } else {
             function = FunctionTo<DataType>::Type::create();
         }
diff --git a/be/src/vec/runtime/time_value.h b/be/src/vec/runtime/time_value.h
index d94e62b977c..8e1624b9780 100644
--- a/be/src/vec/runtime/time_value.h
+++ b/be/src/vec/runtime/time_value.h
@@ -22,6 +22,7 @@
 #include "runtime/define_primitive_type.h"
 #include "runtime/primitive_type.h"
 #include "util/date_func.h"
+#include "vec/common/int_exp.h"
 #include "vec/data_types/data_type_time.h"
 
 namespace doris {
@@ -34,10 +35,43 @@ public:
     constexpr static int64_t ONE_HOUR_MICROSECONDS = 60 * 
ONE_MINUTE_MICROSECONDS;
     constexpr static int64_t ONE_MINUTE_SECONDS = 60;
     constexpr static int64_t ONE_HOUR_SECONDS = 60 * ONE_MINUTE_SECONDS;
+    constexpr static uint32_t MICROS_SCALE = 6;
+    constexpr static int64_t MAX_TIME =
+            3024000LL * ONE_SECOND_MICROSECONDS - 1; // 840:00:00 - 1ms -> 
838:59:59.999999
 
     using TimeType = typename PrimitiveTypeTraits<TYPE_TIMEV2>::CppType;
     using ColumnTime = vectorized::DataTypeTimeV2::ColumnType;
 
+    static int64_t round_time(TimeType value, uint32_t scale) {
+        auto time = (int64_t)value;
+        DCHECK(scale <= MICROS_SCALE);
+        int64_t factor = common::exp10_i64(6 - scale);
+        int64_t rounded_value = (time >= 0) ? (time + factor / 2) / factor * 
factor
+                                            : (time - factor / 2) / factor * 
factor;
+        return rounded_value;
+    }
+
+    // Construct time based on hour/minute/second/microsecond
+    template <bool CHECK = false>
+    static TimeType make_time(int64_t hour, int64_t minute, int64_t second, 
int64_t microsecond = 0,
+                              bool negative = false) {
+        if constexpr (CHECK) {
+            // the max time value is 838:59:59.999999
+            if (std::abs(hour) > 838 || std::abs(minute) >= 60 || 
std::abs(second) >= 60 ||
+                std::abs(microsecond) >= 1000000) [[unlikely]] {
+                throw Exception(ErrorCode::INVALID_ARGUMENT,
+                                "Invalid time value: hour={}, minute={}, 
second={}, microsecond={}",
+                                hour, minute, second, microsecond);
+            }
+        }
+        DCHECK(hour >= 0 && minute >= 0 && second >= 0 && microsecond >= 0)
+                << "Hour, minute, second and microsecond must be non-negative 
but got " << hour
+                << ":" << minute << ":" << second << "." << microsecond;
+        int64_t value = (hour * ONE_HOUR_MICROSECONDS) + (minute * 
ONE_MINUTE_MICROSECONDS) +
+                        (second * ONE_SECOND_MICROSECONDS) + microsecond;
+        return static_cast<TimeType>(negative ? -value : value);
+    }
+
     // refer to https://dev.mysql.com/doc/refman/5.7/en/time.html
     // the time value between '-838:59:59' and '838:59:59'
     /// TODO: Why is the time type stored as double? Can we directly use int64 
and remove the time limit?
@@ -67,6 +101,18 @@ public:
     static int32_t second(TimeType time) {
         return (check_over_max_time(time) / ONE_SECOND_MICROSECONDS) % 
ONE_MINUTE_SECONDS;
     }
+
+    // refer to https://dev.mysql.com/doc/refman/5.7/en/time.html
+    // the time value between '-838:59:59' and '838:59:59'
+    static TimeType limit_with_bound(TimeType time) {
+        if (time > MAX_TIME) {
+            return MAX_TIME;
+        }
+        if (time < -MAX_TIME) {
+            return -MAX_TIME;
+        }
+        return time;
+    }
 };
 
 } // namespace doris
diff --git a/be/src/vec/runtime/vdatetime_value.h 
b/be/src/vec/runtime/vdatetime_value.h
index 11dfb6e796b..0e815dbf3e0 100644
--- a/be/src/vec/runtime/vdatetime_value.h
+++ b/be/src/vec/runtime/vdatetime_value.h
@@ -442,6 +442,12 @@ public:
         return _hour * SECOND_PER_HOUR + _minute * SECOND_PER_MINUTE + _second;
     }
 
+    void reset_time_part() {
+        _hour = 0;
+        _minute = 0;
+        _second = 0;
+    }
+
     bool check_loss_accuracy_cast_to_date() {
         auto loss_accuracy = _hour != 0 || _minute != 0 || _second != 0;
         cast_to_date();
@@ -934,6 +940,15 @@ public:
         return hour() * SECOND_PER_HOUR + minute() * SECOND_PER_MINUTE + 
second();
     }
 
+    void reset_time_part() {
+        if constexpr (is_datetime) {
+            date_v2_value_.hour_ = 0;
+            date_v2_value_.minute_ = 0;
+            date_v2_value_.second_ = 0;
+            date_v2_value_.microsecond_ = 0;
+        }
+    }
+
     int64_t time_part_to_microsecond() const {
         return time_part_to_seconds() * 1000 * 1000 + microsecond();
     }
diff --git a/be/test/runtime/time_value_test.cpp 
b/be/test/runtime/time_value_test.cpp
new file mode 100644
index 00000000000..d38698131f2
--- /dev/null
+++ b/be/test/runtime/time_value_test.cpp
@@ -0,0 +1,50 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cctz/civil_time.h>
+#include <cctz/time_zone.h>
+#include <gtest/gtest.h>
+#include <vec/runtime/time_value.h>
+
+namespace doris {
+
+TEST(TimeValueTest, make_time) {
+    int64_t hour = 1;
+    int64_t minute = 2;
+    int64_t second = 3;
+    TimeValue::TimeType time = TimeValue::make_time(hour, minute, second);
+    EXPECT_EQ(time, 3723000000);
+}
+
+TEST(TimeValueTest, round_time) {
+    //  01:02:03.500000 -> 01:02:04.000000
+    EXPECT_EQ(TimeValue::round_time(TimeValue::make_time(1, 2, 3, 500000), 0),
+              TimeValue::make_time(1, 2, 4));
+
+    //  01:02:03.499999 -> 01:01:03.000000
+    EXPECT_EQ(TimeValue::round_time(TimeValue::make_time(1, 2, 3, 499999), 0),
+              TimeValue::make_time(1, 2, 3));
+
+    //  -01:02:03.500000 -> -01:01:04.000000
+    EXPECT_EQ(TimeValue::round_time(-TimeValue::make_time(1, 2, 3, 500000), 0),
+              -TimeValue::make_time(1, 2, 4));
+
+    //  -01:02:03.499999 -> -01:01:03.000000
+    EXPECT_EQ(TimeValue::round_time(-TimeValue::make_time(1, 2, 3, 499999), 0),
+              -TimeValue::make_time(1, 2, 3));
+}
+} // namespace doris
\ No newline at end of file
diff --git 
a/fe/fe-common/src/main/java/org/apache/doris/catalog/PrimitiveType.java 
b/fe/fe-common/src/main/java/org/apache/doris/catalog/PrimitiveType.java
index fbd9bf6924e..3c1327e1e40 100644
--- a/fe/fe-common/src/main/java/org/apache/doris/catalog/PrimitiveType.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/PrimitiveType.java
@@ -403,6 +403,7 @@ public enum PrimitiveType {
         builder.put(DATETIMEV2, DATETIME);
         builder.put(DATETIMEV2, DATEV2);
         builder.put(DATETIMEV2, DATETIMEV2);
+        builder.put(DATETIMEV2, TIMEV2);
         builder.put(DATETIMEV2, DECIMALV2);
         builder.put(DATETIMEV2, DECIMAL32);
         builder.put(DATETIMEV2, DECIMAL64);
@@ -620,13 +621,21 @@ public enum PrimitiveType {
         builder.put(TIME, DOUBLE);
         builder.put(TIME, VARCHAR);
         builder.put(TIME, STRING);
+        builder.put(TIME, DATE);
+        builder.put(TIME, DATETIME);
+        builder.put(TIME, DATEV2);
+        builder.put(TIME, DATETIMEV2);
 
-        //TIMEV2
+        // TIMEV2
         builder.put(TIMEV2, TIME);
         builder.put(TIMEV2, TIMEV2);
         builder.put(TIMEV2, DOUBLE);
         builder.put(TIMEV2, VARCHAR);
         builder.put(TIMEV2, STRING);
+        builder.put(TIMEV2, DATE);
+        builder.put(TIMEV2, DATETIME);
+        builder.put(TIMEV2, DATEV2);
+        builder.put(TIMEV2, DATETIMEV2);
 
         implicitCastMap = builder.build();
     }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/check/CheckCast.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/check/CheckCast.java
index e76d7ef344d..2f74144be08 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/check/CheckCast.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/check/CheckCast.java
@@ -125,7 +125,8 @@ public class CheckCast implements 
ExpressionPatternRuleFactory {
             return false;
         }
         if (targetType.isTimeLikeType() && !(originalType.isIntegralType()
-                || originalType.isStringLikeType() || 
originalType.isFloatLikeType())) {
+                || originalType.isStringLikeType() || 
originalType.isFloatLikeType()
+                || originalType.isDateTimeType() || 
originalType.isDateTimeV2Type())) {
             return false;
         }
         return true;
diff --git a/regression-test/data/correctness/test_cast_time_to_datetime.out 
b/regression-test/data/correctness/test_cast_time_to_datetime.out
new file mode 100644
index 00000000000..a498c5649f9
Binary files /dev/null and 
b/regression-test/data/correctness/test_cast_time_to_datetime.out differ
diff --git 
a/regression-test/suites/correctness/test_cast_time_to_datetime.groovy 
b/regression-test/suites/correctness/test_cast_time_to_datetime.groovy
new file mode 100644
index 00000000000..bfdd40e0726
--- /dev/null
+++ b/regression-test/suites/correctness/test_cast_time_to_datetime.groovy
@@ -0,0 +1,27 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite("test_cast_time_to_datetime") {
+    qt_sql "select cast(cast('0000-01-01 12:12:12' as datetime) as time);"
+    qt_sql "select cast(cast('2000-02-03 12:12:12' as datetime) as time);"
+    qt_sql "select cast(cast('2000-02-03 12:12:12.123456' as datetime(6)) as 
time(4));"
+    qt_sql "select cast(cast(cast('2020-12-12 12:12:12' as time) as datetime) 
as time);"
+    qt_sql "select time_to_sec(cast('2002-05-30 10:10:20' as datetime));"
+    def res = sql "select date_format(cast(cast('2000-02-03 12:12:12.123456' 
as datetime(6)) as time(4)), '%b %e %Y %l:%i%p');"
+    // check final 7 char of res[0][0] is 12:12PM
+    assertEquals("12:12PM", res[0][0].substring(res[0][0].length() - 7))
+}
\ No newline at end of file
diff --git a/regression-test/suites/nereids_syntax_p0/cast.groovy 
b/regression-test/suites/nereids_syntax_p0/cast.groovy
index 5354648801a..5e4377d0d21 100644
--- a/regression-test/suites/nereids_syntax_p0/cast.groovy
+++ b/regression-test/suites/nereids_syntax_p0/cast.groovy
@@ -244,10 +244,7 @@ suite("cast") {
         exception "cannot cast"
     }
     // datetime
-    test {
-        sql """select cast(k11 as time) ct from test order by ct;"""
-        exception "cannot cast"
-    }
+    sql """select cast(k11 as time) ct from test order by ct;"""
 
     sql "select cast(1 as signed)"
     sql "select cast(1 as signed int)"


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to