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

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


The following commit(s) were added to refs/heads/branch-4.0 by this push:
     new 0143660af11 branch-4.0: [feature](function) The date_add function 
supports DAY_SECOND as interval type #57253 (#57484)
0143660af11 is described below

commit 0143660af115cba780bba44e7f09e0e4f078c2ca
Author: github-actions[bot] 
<41898282+github-actions[bot]@users.noreply.github.com>
AuthorDate: Fri Oct 31 09:53:26 2025 +0800

    branch-4.0: [feature](function) The date_add function supports DAY_SECOND 
as interval type #57253 (#57484)
    
    Cherry-picked from #57253
    
    Co-authored-by: ivin <[email protected]>
---
 .../function_date_or_datetime_computation.h        | 175 +++++++++++++++++----
 .../function_date_or_datetime_computation_v2.cpp   |   3 +
 be/test/vec/function/function_time_test.cpp        |  35 +++++
 .../antlr4/org/apache/doris/nereids/DorisLexer.g4  |   1 +
 .../antlr4/org/apache/doris/nereids/DorisParser.g4 |   2 +-
 .../rules/analysis/DatetimeFunctionBinder.java     |   3 +
 .../functions/executable/DateTimeArithmetic.java   |   9 ++
 .../expressions/functions/scalar/DaySecondAdd.java |  80 ++++++++++
 .../expressions/literal/DateTimeV2Literal.java     |  50 ++++++
 .../trees/expressions/literal/Interval.java        |   1 +
 .../expressions/visitor/ScalarFunctionVisitor.java |   5 +
 .../nereids/rules/expression/FoldConstantTest.java |  16 ++
 .../datetime_functions/test_date_function.out      |   3 +
 .../test_dateadd_with_other_timeunit.out           |  50 ++++++
 .../sql-manual/sql-functions/doc_date_error.groovy |   2 +-
 .../datetime_functions/test_date_function.groovy   |   7 +
 .../test_dateadd_with_other_timeunit.groovy        | 116 ++++++++++++++
 17 files changed, 528 insertions(+), 30 deletions(-)

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 b3698db9087..d61d11932f0 100644
--- a/be/src/vec/functions/function_date_or_datetime_computation.h
+++ b/be/src/vec/functions/function_date_or_datetime_computation.h
@@ -36,6 +36,7 @@
 #include "runtime/runtime_state.h"
 #include "udf/udf.h"
 #include "util/binary_cast.hpp"
+#include "util/string_parser.hpp"
 #include "vec/aggregate_functions/aggregate_function.h"
 #include "vec/columns/column.h"
 #include "vec/columns/column_const.h"
@@ -79,6 +80,7 @@ auto date_time_add(const typename 
PrimitiveTypeTraits<ArgType>::DataType::FieldT
     if (!(ts_value.template date_add_interval<unit>(interval))) [[unlikely]] {
         throw_out_of_bound_date_int<ValueType, 
NativeType>(get_time_unit_name(unit), t, delta);
     }
+
     // here DateValueType = ResultDateValueType
     return binary_cast<ValueType, NativeType>(ts_value);
 }
@@ -124,6 +126,99 @@ ADD_TIME_FUNCTION_IMPL(AddWeeksImpl, weeks_add, WEEK);
 ADD_TIME_FUNCTION_IMPL(AddMonthsImpl, months_add, MONTH);
 ADD_TIME_FUNCTION_IMPL(AddYearsImpl, years_add, YEAR);
 
+template <PrimitiveType PType>
+struct AddDaySecondImpl {
+    static constexpr PrimitiveType ArgPType = PType;
+    static constexpr PrimitiveType ReturnType = PType;
+    static constexpr PrimitiveType IntervalPType = PrimitiveType ::TYPE_STRING;
+    using InputNativeType = typename PrimitiveTypeTraits<PType>::DataType 
::FieldType;
+    using ReturnNativeType = InputNativeType;
+    using IntervalDataType = typename 
PrimitiveTypeTraits<IntervalPType>::DataType;
+    using IntervalNativeType = IntervalDataType::FieldType; // string
+    using ConvertedType = typename 
PrimitiveTypeTraits<TYPE_BIGINT>::DataType::FieldType;
+
+    static constexpr auto name = "day_second_add";
+    static constexpr auto is_nullable = false;
+
+    static inline ReturnNativeType execute(const InputNativeType& t, 
IntervalNativeType delta) {
+        long seconds = parse_time_string_to_seconds(delta);
+        return date_time_add<TimeUnit::SECOND, PType, ConvertedType>(t, 
seconds);
+    }
+
+    static DataTypes get_variadic_argument_types() {
+        return {std ::make_shared<typename 
PrimitiveTypeTraits<PType>::DataType>(),
+                std ::make_shared<typename 
PrimitiveTypeTraits<IntervalPType>::DataType>()};
+    }
+
+    static long parse_time_string_to_seconds(IntervalNativeType time_str_ref) {
+        bool is_negative = false;
+        auto time_str = StringRef {time_str_ref.data(), 
time_str_ref.length()}.trim();
+        // string format: "d h:m:s"
+        size_t space_pos = time_str.find_first_of(' ');
+        if (space_pos == std::string::npos) {
+            throw Exception(ErrorCode::INVALID_ARGUMENT,
+                            "Invalid time format, missing space in '{}'",
+                            std::string_view {time_str.data, time_str.size});
+        }
+        // day
+        StringRef days_sub = time_str.substring(0, space_pos).trim();
+        StringParser::ParseResult success;
+        int days = StringParser::string_to_int_internal<int32_t, 
true>(days_sub.data, days_sub.size,
+                                                                       
&success);
+        if (success != StringParser::PARSE_SUCCESS) {
+            throw Exception(ErrorCode::INVALID_ARGUMENT, "Invalid days format 
in '{}'",
+                            std::string_view {time_str.data, time_str.size});
+        }
+        if (days < 0) {
+            is_negative = true;
+        }
+
+        // hour:minute:second
+        StringRef time_hour_str = time_str.substring(space_pos + 1);
+        size_t colon1 = time_hour_str.find_first_of(':');
+        if (colon1 == std::string::npos) {
+            throw Exception(ErrorCode::INVALID_ARGUMENT, "Invalid time format, 
missing ':' in '{}'",
+                            std::string_view {time_str.data, time_str.size});
+        }
+        size_t colon2_rel = time_hour_str.substring(colon1 + 
1).find_first_of(':');
+        size_t colon2 =
+                (colon2_rel != std::string::npos) ? colon1 + 1 + colon2_rel : 
std::string::npos;
+        if (colon2 == std::string::npos) {
+            throw Exception(ErrorCode::INVALID_ARGUMENT, "Invalid time format, 
missing ':' in '{}'",
+                            std::string_view {time_str.data, time_str.size});
+        }
+        StringRef hours_sub = time_hour_str.substring(0, colon1).trim();
+        int hours = StringParser::string_to_int_internal<int32_t, 
true>(hours_sub.data,
+                                                                        
hours_sub.size, &success);
+        if (success != StringParser::PARSE_SUCCESS) {
+            throw Exception(ErrorCode::INVALID_ARGUMENT, "Invalid hours format 
in '{}'",
+                            std::string_view {time_str.data, time_str.size});
+        }
+        StringRef minutes_sub = time_hour_str.substring(colon1 + 1, colon2 - 
colon1 - 1).trim();
+        int minutes = StringParser::string_to_int_internal<int32_t, true>(
+                minutes_sub.data, minutes_sub.size, &success);
+        if (success != StringParser::PARSE_SUCCESS) {
+            throw Exception(ErrorCode::INVALID_ARGUMENT, "Invalid minutes 
format in '{}'",
+                            std::string_view {time_str.data, time_str.size});
+        }
+        StringRef seconds_sub = time_hour_str.substring(colon2 + 1).trim();
+        int seconds = StringParser::string_to_int_internal<int32_t, true>(
+                seconds_sub.data, seconds_sub.size, &success);
+        if (success != StringParser::PARSE_SUCCESS) {
+            throw Exception(ErrorCode::INVALID_ARGUMENT, "Invalid seconds 
format in '{}'",
+                            std::string_view {time_str.data, time_str.size});
+        }
+
+        long part0 = days * 24 * 3600;
+        // NOTE: Compatible with MySQL
+        long part1 = std::abs(hours) * 3600 + std::abs(minutes) * 60 + 
std::abs(seconds);
+        if (is_negative) {
+            part1 *= -1;
+        }
+        return part0 + part1;
+    }
+};
+
 template <PrimitiveType PType>
 struct AddQuartersImpl {
     static constexpr PrimitiveType ArgPType = PType;
@@ -580,14 +675,32 @@ public:
             // vector-const
             if (const auto* nest_col1_const = 
check_and_get_column<ColumnConst>(*nest_col1)) {
                 rconst = true;
-                const auto col1_inside_const =
-                        assert_cast<const 
IntervalColumnType&>(nest_col1_const->get_data_column());
-                Op::vector_constant(sources->get_data(), res_col->get_data(),
-                                    col1_inside_const.get_data()[0], nullmap0, 
nullmap1);
+                if constexpr (Transform::IntervalPType == TYPE_STRING) {
+                    Op::vector_constant(sources->get_data(), 
res_col->get_data(),
+                                        
nest_col1_const->get_data_at(0).to_string(), nullmap0,
+                                        nullmap1);
+                } else {
+                    const auto col1_inside_const = assert_cast<const 
IntervalColumnType&>(
+                            nest_col1_const->get_data_column());
+                    Op::vector_constant(sources->get_data(), 
res_col->get_data(),
+                                        col1_inside_const.get_data()[0], 
nullmap0, nullmap1);
+                }
             } else { // vector-vector
-                const auto concrete_col1 = assert_cast<const 
IntervalColumnType&>(*nest_col1);
-                Op::vector_vector(sources->get_data(), 
concrete_col1.get_data(),
-                                  res_col->get_data(), nullmap0, nullmap1);
+                if constexpr (Transform::IntervalPType != TYPE_STRING) {
+                    const auto concrete_col1 = assert_cast<const 
IntervalColumnType&>(*nest_col1);
+                    Op::vector_vector(sources->get_data(), 
concrete_col1.get_data(),
+                                      res_col->get_data(), nullmap0, nullmap1);
+                } else {
+                    const auto* nest_col1_string = 
check_and_get_column<ColumnString>(*nest_col1);
+                    if (nest_col1_string->size() == 1) {
+                        rconst = true;
+                        Op::vector_constant(sources->get_data(), 
res_col->get_data(),
+                                            
nest_col1_string->get_data_at(0).to_string(), nullmap0,
+                                            nullmap1);
+                    } else {
+                        return Status::NotSupported("Do not support 
vector-vector for string type");
+                    }
+                }
             }
 
             // update result nullmap with inputs
@@ -608,28 +721,33 @@ public:
         } else if (const auto* sources_const =
                            
check_and_get_column_const<ColumnVector<Transform::ArgPType>>(
                                    src_nested_col.get())) {
-            // const-vector
-            const auto col0_inside_const = assert_cast<const 
ColumnVector<Transform::ArgPType>&>(
-                    sources_const->get_data_column());
-            const ColumnPtr nested_col1 = remove_nullable(col1);
-            const auto concrete_col1 = assert_cast<const 
IntervalColumnType&>(*nested_col1);
-            Op::constant_vector(col0_inside_const.get_data()[0], 
res_col->get_data(),
-                                concrete_col1.get_data(), nullmap0, nullmap1);
-
-            // update result nullmap with inputs
-            if (result_nullable) {
-                auto null_map = ColumnBool::create(input_rows_count, 0);
-                NullMap& result_null_map = 
assert_cast<ColumnBool&>(*null_map).get_data();
-                if (nullmap0) {
-                    VectorizedUtils::update_null_map(result_null_map, 
*nullmap0, true);
-                }
-                if (nullmap1) { // no const-const here. default impl deal it.
-                    VectorizedUtils::update_null_map(result_null_map, 
*nullmap1);
+            if constexpr (Transform::IntervalPType != TYPE_STRING) {
+                // const-vector
+                const auto col0_inside_const =
+                        assert_cast<const ColumnVector<Transform::ArgPType>&>(
+                                sources_const->get_data_column());
+                const ColumnPtr nested_col1 = remove_nullable(col1);
+                const auto concrete_col1 = assert_cast<const 
IntervalColumnType&>(*nested_col1);
+                Op::constant_vector(col0_inside_const.get_data()[0], 
res_col->get_data(),
+                                    concrete_col1.get_data(), nullmap0, 
nullmap1);
+
+                // update result nullmap with inputs
+                if (result_nullable) {
+                    auto null_map = ColumnBool::create(input_rows_count, 0);
+                    NullMap& result_null_map = 
assert_cast<ColumnBool&>(*null_map).get_data();
+                    if (nullmap0) {
+                        VectorizedUtils::update_null_map(result_null_map, 
*nullmap0, true);
+                    }
+                    if (nullmap1) { // no const-const here. default impl deal 
it.
+                        VectorizedUtils::update_null_map(result_null_map, 
*nullmap1);
+                    }
+                    block.get_by_position(result).column =
+                            ColumnNullable::create(std::move(res_col), 
std::move(null_map));
+                } else {
+                    block.replace_by_position(result, std::move(res_col));
                 }
-                block.get_by_position(result).column =
-                        ColumnNullable::create(std::move(res_col), 
std::move(null_map));
             } else {
-                block.replace_by_position(result, std::move(res_col));
+                return Status::NotSupported("Do not support const-vector for 
string type");
             }
         } else { // no const-const here. default impl deal it.
             return Status::InternalError(
@@ -855,7 +973,8 @@ struct CurrentTimeImpl {
                                             dtv.microsecond());
             } else {
                 return Status::InvalidArgument(
-                        "The precision in function CURTIME should be between 0 
and 6, but got {}",
+                        "The precision in function CURTIME should be between 0 
and 6, but got "
+                        "{}",
                         precision);
             }
         } else {
diff --git a/be/src/vec/functions/function_date_or_datetime_computation_v2.cpp 
b/be/src/vec/functions/function_date_or_datetime_computation_v2.cpp
index ee6b92aa45c..460b5d30755 100644
--- a/be/src/vec/functions/function_date_or_datetime_computation_v2.cpp
+++ b/be/src/vec/functions/function_date_or_datetime_computation_v2.cpp
@@ -53,6 +53,8 @@ using FunctionDatetimeV2AddMonths =
 using FunctionDatetimeV2AddQuarters =
         FunctionDateOrDateTimeComputation<AddQuartersImpl<TYPE_DATETIMEV2>>;
 using FunctionDatetimeV2AddYears = 
FunctionDateOrDateTimeComputation<AddYearsImpl<TYPE_DATETIMEV2>>;
+using FunctionDatetimeV2AddDaySecond =
+        FunctionDateOrDateTimeComputation<AddDaySecondImpl<TYPE_DATETIMEV2>>;
 
 using FunctionDatetimeV2SubMicroseconds =
         
FunctionDateOrDateTimeComputation<SubtractMicrosecondsImpl<TYPE_DATETIMEV2>>;
@@ -116,6 +118,7 @@ void 
register_function_date_time_computation_v2(SimpleFunctionFactory& factory)
     factory.register_function<FunctionDatetimeV2AddMonths>();
     factory.register_function<FunctionDatetimeV2AddYears>();
     factory.register_function<FunctionDatetimeV2AddQuarters>();
+    factory.register_function<FunctionDatetimeV2AddDaySecond>();
 
     factory.register_function<FunctionSubDaysV2>();
     factory.register_function<FunctionSubMonthsV2>();
diff --git a/be/test/vec/function/function_time_test.cpp 
b/be/test/vec/function/function_time_test.cpp
index d51c45513c3..78dca0b2ba2 100644
--- a/be/test/vec/function/function_time_test.cpp
+++ b/be/test/vec/function/function_time_test.cpp
@@ -1081,6 +1081,41 @@ TEST(VTimestampFunctionsTest, weeks_sub_v2_test) {
     }
 }
 
+TEST(VTimestampFunctionsTest, day_second_add_v2_test) {
+    std::string func_name = "day_second_add";
+
+    InputTypeSet input_types = {PrimitiveType::TYPE_DATETIMEV2,
+                                Consted {PrimitiveType::TYPE_STRING}};
+
+    {
+        DataSet data_set = {
+                {{std::string("2020-10-23 00:00:11.123"), std::string("1 
0:0:1")},
+                 std::string("2020-10-24 00:00:12.123")},
+        };
+
+        static_cast<void>(
+                check_function<DataTypeDateTimeV2, true>(func_name, 
input_types, data_set));
+    }
+    {
+        DataSet data_set = {
+                {{std::string("2020-05-23 00:00:11.123"), std::string("1 
10:11:12")},
+                 std::string("2020-05-24 10:11:23.123")},
+        };
+
+        static_cast<void>(
+                check_function<DataTypeDateTimeV2, true>(func_name, 
input_types, data_set));
+    }
+    {
+        DataSet data_set = {
+                {{std::string("2020-05-23 00:00:11.123"), std::string("10 
0:0:1")},
+                 std::string("2020-06-02 00:00:12.123")},
+        };
+
+        static_cast<void>(
+                check_function<DataTypeDateTimeV2, true>(func_name, 
input_types, data_set));
+    }
+}
+
 TEST(VTimestampFunctionsTest, to_days_v2_test) {
     std::string func_name = "to_days";
 
diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 
b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4
index fa9f119258c..521ef91040b 100644
--- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4
+++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4
@@ -176,6 +176,7 @@ DATEV2: 'DATEV2';
 DATETIMEV1: 'DATETIMEV1';
 DATEV1: 'DATEV1';
 DAY: 'DAY';
+DAY_SECOND: 'DAY_SECOND';
 DAYS: 'DAYS';
 DECIMAL: 'DECIMAL';
 DECIMALV2: 'DECIMALV2';
diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 
b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
index d85b645f0a4..c4649068c24 100644
--- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
+++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
@@ -1737,7 +1737,7 @@ interval
     ;
 
 unitIdentifier
-       : YEAR | QUARTER | MONTH | WEEK | DAY | HOUR | MINUTE | SECOND
+       : YEAR | QUARTER | MONTH | WEEK | DAY | HOUR | MINUTE | SECOND | 
DAY_SECOND
     ;
 
 dataTypeWithNullable
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/DatetimeFunctionBinder.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/DatetimeFunctionBinder.java
index 7f57b09b356..8847beeada6 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/DatetimeFunctionBinder.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/DatetimeFunctionBinder.java
@@ -33,6 +33,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayRangeYea
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DateDiff;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DayCeil;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DayFloor;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.DaySecondAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DaysAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DaysDiff;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DaysSub;
@@ -299,6 +300,8 @@ public class DatetimeFunctionBinder {
                 return new MinutesAdd(timestamp, amount);
             case SECOND:
                 return new SecondsAdd(timestamp, amount);
+            case DAY_SECOND:
+                return new DaySecondAdd(timestamp, amount);
             default:
                 throw new AnalysisException("Unsupported time stamp add time 
unit: " + unit
                         + ", supported time unit: 
YEAR/QUARTER/MONTH/WEEK/DAY/HOUR/MINUTE/SECOND");
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeArithmetic.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeArithmetic.java
index f49daa1a386..07bd85d8d95 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeArithmetic.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeArithmetic.java
@@ -26,6 +26,7 @@ import 
org.apache.doris.nereids.trees.expressions.literal.DateTimeV2Literal;
 import org.apache.doris.nereids.trees.expressions.literal.DateV2Literal;
 import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral;
 import org.apache.doris.nereids.trees.expressions.literal.TimeV2Literal;
+import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral;
 
 import java.time.LocalDateTime;
 import java.time.temporal.ChronoUnit;
@@ -125,6 +126,14 @@ public class DateTimeArithmetic {
         return date.plusDays(day.getValue());
     }
 
+    /**
+     * datetime arithmetic function day_second-add.
+     */
+    @ExecFunction(name = "day_second_add")
+    public static Expression daysAdd(DateTimeV2Literal date, VarcharLiteral 
daySecond) {
+        return date.plusDaySecond(daySecond);
+    }
+
     /**
      * datetime arithmetic function hours-add.
      */
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaySecondAdd.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaySecondAdd.java
new file mode 100644
index 00000000000..6a7889982aa
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/DaySecondAdd.java
@@ -0,0 +1,80 @@
+// 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.nereids.trees.expressions.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import 
org.apache.doris.nereids.trees.expressions.functions.ComputeSignatureForDateArithmetic;
+import 
org.apache.doris.nereids.trees.expressions.functions.DateAddSubMonotonic;
+import 
org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import 
org.apache.doris.nereids.trees.expressions.functions.PropagateNullableOnDateOrTimeLikeV2Args;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DateTimeV2Type;
+import org.apache.doris.nereids.types.VarcharType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'day_second_add'.
+ */
+public class DaySecondAdd extends ScalarFunction
+        implements BinaryExpression, ExplicitlyCastableSignature,
+        ComputeSignatureForDateArithmetic, 
PropagateNullableOnDateOrTimeLikeV2Args, DateAddSubMonotonic {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(DateTimeV2Type.SYSTEM_DEFAULT).args(DateTimeV2Type.SYSTEM_DEFAULT,
+                VarcharType.SYSTEM_DEFAULT));
+
+    public DaySecondAdd(Expression arg0, Expression arg1) {
+        super("day_second_add", arg0, arg1);
+    }
+
+    /** constructor for withChildren and reuse signature */
+    private DaySecondAdd(ScalarFunctionParams functionParams) {
+        super(functionParams);
+    }
+
+    @Override
+    public DaySecondAdd withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 2);
+        return new DaySecondAdd(getFunctionParams(children));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitDaySecondAdd(this, context);
+    }
+
+    @Override
+    public Expression withConstantArgs(Expression literal) {
+        return new DaySecondAdd(literal, child(1));
+    }
+
+    @Override
+    public FunctionSignature computeSignature(FunctionSignature signature) {
+        return super.computeSignature(signature);
+    }
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java
index 0af78154b60..52d03b319c6 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java
@@ -213,6 +213,56 @@ public class DateTimeV2Literal extends DateTimeLiteral {
         return fromJavaDateType(toJavaDateType().plusDays(days), 
getDataType().getScale());
     }
 
+    /**
+     * plusDaySecond
+     */
+    public Expression plusDaySecond(VarcharLiteral daySecond) {
+        String stringValue = daySecond.getStringValue().trim();
+
+        if (!stringValue.matches("[0-9:\\-\\s]+")) {
+            return new NullLiteral(dataType);
+        }
+
+        String[] split = stringValue.split("\\s+");
+        if (split.length != 2) {
+            return new NullLiteral(dataType);
+        }
+
+        String day = split[0];
+        String[] hourMinuteSecond = split[1].split(":");
+
+        if (hourMinuteSecond.length != 3) {
+            return new NullLiteral(dataType);
+        }
+
+        try {
+            long days = Long.parseLong(day);
+            boolean dayPositive = days >= 0;
+
+            long hours = Long.parseLong(hourMinuteSecond[0]);
+            long minutes = Long.parseLong(hourMinuteSecond[1]);
+            long seconds = Long.parseLong(hourMinuteSecond[2]);
+
+            if (dayPositive) {
+                hours = Math.abs(hours);
+                minutes = Math.abs(minutes);
+                seconds = Math.abs(seconds);
+            } else {
+                hours = -Math.abs(hours);
+                minutes = -Math.abs(minutes);
+                seconds = -Math.abs(seconds);
+            }
+
+            return fromJavaDateType(toJavaDateType()
+                .plusDays(days)
+                .plusHours(hours)
+                .plusMinutes(minutes)
+                .plusSeconds(seconds), getDataType().getScale());
+        } catch (NumberFormatException e) {
+            return new NullLiteral(dataType);
+        }
+    }
+
     public Expression plusMonths(long months) {
         return fromJavaDateType(toJavaDateType().plusMonths(months), 
getDataType().getScale());
     }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/Interval.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/Interval.java
index 1fe7dfcd1a4..110eb225dea 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/Interval.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/Interval.java
@@ -74,6 +74,7 @@ public class Interval extends Expression implements 
UnaryExpression, AlwaysNotNu
      * Supported time unit.
      */
     public enum TimeUnit {
+        DAY_SECOND("DAY_SECOND", false, 900),
         YEAR("YEAR", false, 800),
         MONTH("MONTH", false, 700),
         QUARTER("QUARTER", false, 600), //TODO: need really support quarter
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
index 4cb7f7bbf8a..b62b490404a 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
@@ -179,6 +179,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.DayName;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DayOfMonth;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DayOfWeek;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DayOfYear;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.DaySecondAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DaysAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DaysDiff;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DaysSub;
@@ -1127,6 +1128,10 @@ public interface ScalarFunctionVisitor<R, C> {
         return visitScalarFunction(daysAdd, context);
     }
 
+    default R visitDaySecondAdd(DaySecondAdd daySecondAdd, C context) {
+        return visitScalarFunction(daySecondAdd, context);
+    }
+
     default R visitDaysSub(DaysSub daysSub, C context) {
         return visitScalarFunction(daysSub, context);
     }
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/FoldConstantTest.java
 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/FoldConstantTest.java
index 66f9b5f3f1f..dc89e8b5034 100644
--- 
a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/FoldConstantTest.java
+++ 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/FoldConstantTest.java
@@ -60,6 +60,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.Cot;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Csc;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DateFormat;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DateTrunc;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.DaySecondAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Degrees;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Dexp;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Dlog10;
@@ -400,6 +401,21 @@ class FoldConstantTest extends ExpressionRewriteTestHelper 
{
                         StringLiteral.of("MON"));
         rewritten = executor.rewrite(nextDay, context);
         Assertions.assertEquals(new DateV2Literal("2020-06-01"), rewritten);
+
+        DaySecondAdd daySecondAdd = new DaySecondAdd(
+                DateTimeV2Literal.fromJavaDateType(LocalDateTime.of(1, 1, 1, 
1, 1, 1), 0),
+                new VarcharLiteral("1 1:1:1"));
+        rewritten = executor.rewrite(daySecondAdd, context);
+        Assertions.assertEquals(new DateTimeV2Literal("0001-01-02 02:02:02"), 
rewritten);
+        // fail to fold, because the result is out of range
+        daySecondAdd = new 
DaySecondAdd(DateTimeV2Literal.fromJavaDateType(LocalDateTime.of(9999, 12, 31, 
23, 59, 1), 0),
+                new VarcharLiteral("1 1:1:1"));
+        rewritten = executor.rewrite(daySecondAdd, context);
+        Assertions.assertEquals(daySecondAdd, rewritten);
+        daySecondAdd = new 
DaySecondAdd(DateTimeV2Literal.fromJavaDateType(LocalDateTime.of(0, 1, 1, 0, 1, 
1), 0),
+                new VarcharLiteral("-1 -1:1:1"));
+        rewritten = executor.rewrite(daySecondAdd, context);
+        Assertions.assertEquals(daySecondAdd, rewritten);
     }
 
     @Test
diff --git 
a/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_function.out
 
b/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_function.out
index 8b853fb0494..e38c206d81c 100644
--- 
a/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_function.out
+++ 
b/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_function.out
@@ -174,6 +174,9 @@
 -- !sql --
 2010-12-01T00:00:01
 
+-- !sql --
+2010-12-02T00:00
+
 -- !sql --
 22:23:00
 
diff --git 
a/regression-test/data/query_p0/sql_functions/datetime_functions/test_dateadd_with_other_timeunit.out
 
b/regression-test/data/query_p0/sql_functions/datetime_functions/test_dateadd_with_other_timeunit.out
new file mode 100644
index 00000000000..3c7b1efbf0e
--- /dev/null
+++ 
b/regression-test/data/query_p0/sql_functions/datetime_functions/test_dateadd_with_other_timeunit.out
@@ -0,0 +1,50 @@
+-- This file is automatically generated. You should know what you did if you 
want to edit this
+-- !sql --
+2025-10-25T02:03:04.456700
+2025-10-30T11:11:11
+
+-- !sql --
+2025-10-25T02:03:04.456700
+2025-10-30T11:11:11
+
+-- !sql --
+2025-10-25T01:01:01
+2025-10-30T01:01:01
+
+-- !sql --
+2025-10-25T01:01:01
+2025-10-30T01:01:01
+
+-- !sql --
+2025-10-30T01:01:01
+
+-- !sql --
+2025-10-30T01:01:01
+
+-- !sql --
+2025-10-27T22:58:59
+
+-- !sql --
+2025-10-27T22:58:59
+
+-- !sql --
+2025-10-27T22:58:59
+
+-- !sql --
+2025-10-27T22:58:59
+
+-- !sql --
+\N
+
+-- !sql --
+\N
+
+-- !sql --
+\N
+
+-- !sql --
+\N
+
+-- !sql --
+\N
+
diff --git 
a/regression-test/suites/doc/sql-manual/sql-functions/doc_date_error.groovy 
b/regression-test/suites/doc/sql-manual/sql-functions/doc_date_error.groovy
index ccb459ce11d..79c207eeaf0 100644
--- a/regression-test/suites/doc/sql-manual/sql-functions/doc_date_error.groovy
+++ b/regression-test/suites/doc/sql-manual/sql-functions/doc_date_error.groovy
@@ -37,7 +37,7 @@ suite("doc_date_error") {
     }
     test {
         sql """select DATE_ADD('2023-12-31 23:00:00', INTERVAL 2 sa);"""
-        exception "mismatched input 'sa' expecting {'.', '[', 'AND', 
'BETWEEN', 'COLLATE', 'DAY', 'DIV', 'HOUR', 'IN', 'IS', 'LIKE', 'MATCH', 
'MATCH_ALL', 'MATCH_ANY', 'MATCH_PHRASE', 'MATCH_PHRASE_EDGE', 
'MATCH_PHRASE_PREFIX', 'MATCH_REGEXP', 'MINUTE', 'MONTH', 'NOT', 'OR', 
'QUARTER', 'REGEXP', 'RLIKE', 'SECOND', 'WEEK', 'XOR', 'YEAR', EQ, '<=>', NEQ, 
'<', LTE, '>', GTE, '+', '-', '*', '/', '%', '&', '&&', '|', '||', '^'}(line 1, 
pos 50)"
+        exception "mismatched input 'sa' expecting"
     }
 
     // date_ceil out of range
diff --git 
a/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_function.groovy
 
b/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_function.groovy
index 0278e85119e..542b8778d63 100644
--- 
a/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_function.groovy
+++ 
b/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_function.groovy
@@ -251,6 +251,13 @@ suite("test_date_function") {
     qt_sql """ select date_add(test_datetime, INTERVAL 2 HOUR) result from 
${tableName}; """
     qt_sql """ select date_add(test_datetime, INTERVAL 2 MINUTE) result from 
${tableName}; """
     qt_sql """ select date_add(test_datetime, INTERVAL 2 SECOND) result from 
${tableName}; """
+    qt_sql """ select date_add(test_datetime, INTERVAL '1 00:00:01' 
DAY_SECOND) result from ${tableName}; """
+
+    test {
+        sql """ select date_add(test_datetime, INTERVAL '1 xx:00:01' 
DAY_SECOND) result from ${tableName}; """
+        // check exception message contains
+        exception "Invalid hours format"
+    }
 
     explain {
         sql """select * from ${tableName} where test_datetime >= 
date_add('2024-01-16',INTERVAL 1 day);"""
diff --git 
a/regression-test/suites/query_p0/sql_functions/datetime_functions/test_dateadd_with_other_timeunit.groovy
 
b/regression-test/suites/query_p0/sql_functions/datetime_functions/test_dateadd_with_other_timeunit.groovy
new file mode 100644
index 00000000000..5494695db12
--- /dev/null
+++ 
b/regression-test/suites/query_p0/sql_functions/datetime_functions/test_dateadd_with_other_timeunit.groovy
@@ -0,0 +1,116 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+import java.text.SimpleDateFormat
+
+suite("test_dateadd_with_other_timeunit") {
+    def tableName = "test_date_add_with_other_timeunit"
+
+    sql """ DROP TABLE IF EXISTS ${tableName} """
+    sql """
+            CREATE TABLE IF NOT EXISTS ${tableName} (
+                test_datetime datetime(4) NULL COMMENT "",
+                test_date date NULL COMMENT ""
+            ) ENGINE=OLAP
+            DUPLICATE KEY(test_datetime)
+            COMMENT "OLAP"
+            DISTRIBUTED BY HASH(test_datetime) BUCKETS 1
+            PROPERTIES (
+                "replication_allocation" = "tag.location.default: 1",
+                "in_memory" = "false",
+                "storage_format" = "V2"
+            )
+        """
+    sql """ insert into ${tableName} values ("2025-10-29 10:10:10", 
"2025-10-29"), ("2025-10-24 01:02:03.4567", "2025-10-24"); """
+    
+    testFoldConst """ select date_add("2025-10-29 10:10:10", INTERVAL "1 
1:1:1" DAY_SECOND); """
+    testFoldConst """ select date_add("2025-10-29 10:10:10", INTERVAL "1 
-1:1:1" DAY_SECOND); """
+    testFoldConst """ select date_add("2025-10-29 10:10:10", INTERVAL "1 
-1:-1:1" DAY_SECOND); """
+    testFoldConst """ select date_add("2025-10-29 10:10:10", INTERVAL "1 
-1:-1:-1" DAY_SECOND); """
+    testFoldConst """ select date_add("2025-10-29 10:10:10", INTERVAL "-1 
1:1:1" DAY_SECOND); """
+    testFoldConst """ select date_add("2025-10-29 10:10:10", INTERVAL "-1 
-1:1:1" DAY_SECOND); """
+    testFoldConst """ select date_add("2025-10-29 10:10:10", INTERVAL "-1 
-1:-1:1" DAY_SECOND); """
+    testFoldConst """ select date_add("2025-10-29 10:10:10", INTERVAL "-1 
-1:-1:-1" DAY_SECOND); """
+
+    testFoldConst """ select date_add("2025-10-29", INTERVAL "1 1:1:1" 
DAY_SECOND); """
+    testFoldConst """ select date_add("2025-10-29", INTERVAL "1 -1:1:1" 
DAY_SECOND); """
+    testFoldConst """ select date_add("2025-10-29", INTERVAL "1 -1:-1:1" 
DAY_SECOND); """
+    testFoldConst """ select date_add("2025-10-29", INTERVAL "1 -1:-1:-1" 
DAY_SECOND); """
+    testFoldConst """ select date_add("2025-10-29", INTERVAL "-1 1:1:1" 
DAY_SECOND); """
+    testFoldConst """ select date_add("2025-10-29", INTERVAL "-1 -1:1:1" 
DAY_SECOND); """
+    testFoldConst """ select date_add("2025-10-29", INTERVAL "-1 -1:-1:1" 
DAY_SECOND); """
+    testFoldConst """ select date_add("2025-10-29", INTERVAL "-1 -1:-1:-1" 
DAY_SECOND); """
+
+    qt_sql """ select date_add(test_datetime, INTERVAL "1 1:1:1" DAY_SECOND) 
result from ${tableName}; """
+    qt_sql """ select date_add(test_datetime, INTERVAL " 1  1 : 1 : 1 " 
DAY_SECOND) result from ${tableName}; """
+
+    qt_sql """ select date_add(test_date, INTERVAL "1 1:1:1" DAY_SECOND) 
result from ${tableName}; """
+    qt_sql """ select date_add(test_date, INTERVAL " 1  1 : 1 : 1 " 
DAY_SECOND) result from ${tableName}; """
+
+    qt_sql """ select date_add("2025-10-29", INTERVAL "1 1:1:1" DAY_SECOND); 
"""
+    qt_sql """ select date_add("2025-10-29", INTERVAL "1 1:1:-1" DAY_SECOND); 
"""
+    qt_sql """ select date_add("2025-10-29", INTERVAL "-1 1:1:1" DAY_SECOND); 
"""
+    qt_sql """ select date_add("2025-10-29", INTERVAL "-1 -1:1:1" DAY_SECOND); 
"""
+    qt_sql """ select date_add("2025-10-29", INTERVAL "-1 -1:-1:1" 
DAY_SECOND); """
+    qt_sql """ select date_add("2025-10-29", INTERVAL "-1 -1:-1:-1" 
DAY_SECOND); """
+    qt_sql """ select date_add("2025-10-29 10:10:10", INTERVAL "-1 -1:1:1 43" 
DAY_SECOND); """
+    qt_sql """ select date_add("2025-10-29 10:10:10", INTERVAL "-1 -1:1:1xxx" 
DAY_SECOND); """
+    qt_sql """ select date_add("2025-10-29 10:10:10", INTERVAL "1 1:1:1.1234" 
DAY_SECOND); """
+    qt_sql """ select date_add("2025-10-29", INTERVAL "-1 -1:1:1 34" 
DAY_SECOND); """
+    qt_sql """ select date_add("2025-10-29", INTERVAL "-1 -1:1:1xx" 
DAY_SECOND); """
+
+    test {
+        sql """ select date_add(test_datetime, INTERVAL '1' DAY_SECOND) result 
from ${tableName}; """
+        exception "Invalid time format"
+    }
+
+    test {
+        sql """ select date_add(test_datetime, INTERVAL '1 2' DAY_SECOND) 
result from ${tableName}; """
+        exception "Invalid time format"
+    }
+
+    test {
+        sql """ select date_add(test_datetime, INTERVAL '1 2:3' DAY_SECOND) 
result from ${tableName}; """
+        exception "Invalid time format"
+    }
+
+    test {
+        sql """ select date_add(test_datetime, INTERVAL '1 2:3:4.5678' 
DAY_SECOND) result from ${tableName}; """
+        exception "Invalid seconds format"
+    }
+
+    test {
+        sql """ select date_add(test_datetime, INTERVAL 'xx 00:00:01' 
DAY_SECOND) result from ${tableName}; """
+        exception "Invalid days format"
+    }
+
+    test {
+        sql """ select date_add(test_datetime, INTERVAL '1 xx:00:01' 
DAY_SECOND) result from ${tableName}; """
+        exception "Invalid hours format"
+    }
+
+    test {
+        sql """ select date_add(test_datetime, INTERVAL '1 00:xx:01' 
DAY_SECOND) result from ${tableName}; """
+        exception "Invalid minutes format"
+    }
+
+    test {
+        sql """ select date_add(test_datetime, INTERVAL '1 00:00:xx' 
DAY_SECOND) result from ${tableName}; """
+        exception "Invalid seconds format"
+    }
+
+}


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


Reply via email to