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

zclll 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 5b2cbcc7903 [Opt](function) opt of certain time field functions used 
in conjunction with FROM_UNIXTIME. (#57941)
5b2cbcc7903 is described below

commit 5b2cbcc7903dfc5ae3c1cf4b0746c325b4351c2c
Author: linrrarity <[email protected]>
AuthorDate: Mon Jan 5 10:43:17 2026 +0800

    [Opt](function) opt of certain time field functions used in conjunction 
with FROM_UNIXTIME. (#57941)
    
    opt of certain time field functions(`HOUR`, `MINUTE`, `SECOND`, 
`MICROSECOND`) used in conjunction with `FROM_UNIXTIME`
    
    Take `HOUR(FROM_UNIXTIME(ts))` as an example:
    The `hour(from_unixtime(xxx))` function is slow because
    `from_unixtime`needs to extract the full yyyy-MM-dd HH:mm:ss format from
    the timestamp, which is not necessary. By calculating only the required
    fields directly from the timestamp, the process can be significantly
    faster.
    
    Add a function `hour_from_unixtime(ts)` to extract the hour from a unix
    timestamp, which is timezone aware.
    
    Implementation:
    1. Lookup the timezone offset with cctz library
    2. Calculate the hour from local unixtime
    ```cpp
    int64_t local_unixtime = unixtime + timezone.lookup_offset(unixtime);
    int hour = (local_unixtime % (24 * 3600)) / 3600
    ```
    
    Performance:
    
    Before VS After:
    ```text
    -- HOUR
    Doris> SELECT COUNT(HOUR(FROM_UNIXTIME(ts))) FROM test_hour_from_unixtime;
    +--------------------------------+
    | COUNT(HOUR(FROM_UNIXTIME(ts))) |
    +--------------------------------+
    |                      100000000 |
    +--------------------------------+
    1 row in set (9.51 sec)
    
    Doris> SELECT COUNT(HOUR(FROM_UNIXTIME(ts))) FROM test_hour_from_unixtime;
    +--------------------------------+
    | COUNT(HOUR(FROM_UNIXTIME(ts))) |
    +--------------------------------+
    |                      100000000 |
    +--------------------------------+
    1 row in set (0.96 sec)
    
    -- MINUTE
    Doris> SELECT COUNT(MINUTE(FROM_UNIXTIME(ts))) FROM test_hour_from_unixtime;
    +----------------------------------+
    | COUNT(MINUTE(FROM_UNIXTIME(ts))) |
    +----------------------------------+
    |                        100000000 |
    +----------------------------------+
    1 row in set (10.98 sec)
    
    Doris> SELECT COUNT(MINUTE(FROM_UNIXTIME(ts))) FROM test_hour_from_unixtime;
    +----------------------------------+
    | COUNT(MINUTE(FROM_UNIXTIME(ts))) |
    +----------------------------------+
    |                        100000000 |
    +----------------------------------+
    1 row in set (1.00 sec)
    
    -- SECOND
    Doris> SELECT COUNT(SECOND(FROM_UNIXTIME(ts))) FROM test_hour_from_unixtime;
    +----------------------------------+
    | COUNT(SECOND(FROM_UNIXTIME(ts))) |
    +----------------------------------+
    |                        100000000 |
    +----------------------------------+
    1 row in set (10.01 sec)
    
    Doris> SELECT COUNT(SECOND(FROM_UNIXTIME(ts))) FROM test_hour_from_unixtime;
    +----------------------------------+
    | COUNT(SECOND(FROM_UNIXTIME(ts))) |
    +----------------------------------+
    |                        100000000 |
    +----------------------------------+
    1 row in set (0.90 sec)
    
    -- MICROSECOND
    Doris> SELECT COUNT(MICROSECOND(FROM_UNIXTIME(ts))) FROM 
test_hour_from_unixtime;
    +---------------------------------------+
    | COUNT(MICROSECOND(FROM_UNIXTIME(ts))) |
    +---------------------------------------+
    |                             100000000 |
    +---------------------------------------+
    1 row in set (9.75 sec)
    
    Doris> SELECT COUNT(MICROSECOND(FROM_UNIXTIME(ts))) FROM 
test_hour_from_unixtime;
    +---------------------------------------+
    | COUNT(MICROSECOND(FROM_UNIXTIME(ts))) |
    +---------------------------------------+
    |                             100000000 |
    +---------------------------------------+
    1 row in set (1.24 sec)
    ```
---
 be/src/vec/functions/date_time_transforms.h        | 137 +++++++++++++++++++
 .../vec/functions/function_time_value_to_field.cpp |  10 ++
 .../doris/catalog/BuiltinScalarFunctions.java      |   8 ++
 .../rules/expression/ExpressionOptimization.java   |   2 +
 .../rules/expression/ExpressionRuleType.java       |   1 +
 .../rules/SimplifyTimeFieldFromUnixtime.java       | 151 +++++++++++++++++++++
 .../executable/DateTimeExtractAndTransform.java    |  84 ++++++++++++
 .../functions/scalar/HourFromUnixtime.java         |  75 ++++++++++
 .../functions/scalar/MicrosecondFromUnixtime.java  |  75 ++++++++++
 .../functions/scalar/MinuteFromUnixtime.java       |  75 ++++++++++
 .../functions/scalar/SecondFromUnixtime.java       |  75 ++++++++++
 .../expressions/visitor/ScalarFunctionVisitor.java |  20 +++
 .../SimplifyTimeFieldFromUnixtimeTest.java         |  86 ++++++++++++
 .../data/datatype_p0/date/test_from_unixtime.out   |  70 ++++++++++
 .../datatype_p0/date/test_from_unixtime.groovy     |  96 ++++++++++++-
 15 files changed, 964 insertions(+), 1 deletion(-)

diff --git a/be/src/vec/functions/date_time_transforms.h 
b/be/src/vec/functions/date_time_transforms.h
index f494750cf24..b0e73b4aa7f 100644
--- a/be/src/vec/functions/date_time_transforms.h
+++ b/be/src/vec/functions/date_time_transforms.h
@@ -20,13 +20,17 @@
 
 #pragma once
 
+#include <libdivide.h>
+
 #include <cmath>
 #include <cstdint>
 
 #include "common/status.h"
+#include "runtime/define_primitive_type.h"
 #include "runtime/primitive_type.h"
 #include "udf/udf.h"
 #include "util/binary_cast.hpp"
+#include "vec/columns/column_decimal.h"
 #include "vec/columns/column_nullable.h"
 #include "vec/columns/column_string.h"
 #include "vec/columns/column_vector.h"
@@ -427,6 +431,139 @@ struct FromUnixTimeDecimalImpl {
     }
 };
 
+// Base template for optimized time field(HOUR, MINUTE, SECOND, MS) extraction 
from Unix timestamp
+// Uses lookup_offset to avoid expensive civil_second construction
+template <typename Impl>
+class FunctionTimeFieldFromUnixtime : public IFunction {
+public:
+    static constexpr auto name = Impl::name;
+    static FunctionPtr create() { return 
std::make_shared<FunctionTimeFieldFromUnixtime<Impl>>(); }
+
+    String get_name() const override { return name; }
+
+    size_t get_number_of_arguments() const override { return 1; }
+
+    DataTypePtr get_return_type_impl(const ColumnsWithTypeAndName& arguments) 
const override {
+        // microsecond_from_unixtime returns Int32, others 
(hour/minute/second) return Int8
+        if constexpr (Impl::ArgType == PrimitiveType::TYPE_DECIMAL64) {
+            return make_nullable(std::make_shared<DataTypeInt32>());
+        } else {
+            return make_nullable(std::make_shared<DataTypeInt8>());
+        }
+    }
+
+    // (UTC 9999-12-31 23:59:59) - 24 * 3600
+    static const int64_t TIMESTAMP_VALID_MAX = 253402243199L;
+
+    Status execute_impl(FunctionContext* context, Block& block, const 
ColumnNumbers& arguments,
+                        uint32_t result, size_t input_rows_count) const 
override {
+        using ArgColType = PrimitiveTypeTraits<Impl::ArgType>::ColumnType;
+        using ResColType = std::conditional_t<Impl::ArgType == 
PrimitiveType::TYPE_DECIMAL64,
+                                              ColumnInt32, ColumnInt8>;
+        using ResItemType = typename ResColType::value_type;
+        auto res = ResColType::create();
+
+        const auto* ts_col =
+                assert_cast<const 
ArgColType*>(block.get_by_position(arguments[0]).column.get());
+        if constexpr (Impl::ArgType == PrimitiveType::TYPE_DECIMAL64) {
+            // microsecond_from_unixtime only
+            const auto scale = static_cast<int32_t>(ts_col->get_scale());
+
+            for (int i = 0; i < input_rows_count; ++i) {
+                const auto seconds = ts_col->get_intergral_part(i);
+                const auto fraction = ts_col->get_fractional_part(i);
+
+                if (seconds < 0 || seconds > TIMESTAMP_VALID_MAX) {
+                    return Status::InvalidArgument(
+                            "The input value of TimeFiled(from_unixtime()) 
must between 0 and "
+                            "253402243199L");
+                }
+
+                ResItemType value = Impl::extract_field(fraction, scale);
+                res->insert_value(value);
+            }
+        } else {
+            auto ctz = context->state()->timezone_obj();
+            for (int i = 0; i < input_rows_count; ++i) {
+                auto date = ts_col->get_element(i);
+
+                if (date < 0 || date > TIMESTAMP_VALID_MAX) {
+                    return Status::InvalidArgument(
+                            "The input value of TimeFiled(from_unixtime()) 
must between 0 and "
+                            "253402243199L");
+                }
+
+                ResItemType value = Impl::extract_field(date, ctz);
+                res->insert_value(value);
+            }
+        }
+        block.replace_by_position(result, std::move(res));
+        return Status::OK();
+    }
+};
+
+struct HourFromUnixtimeImpl {
+    static constexpr PrimitiveType ArgType = PrimitiveType::TYPE_BIGINT;
+    static constexpr auto name = "hour_from_unixtime";
+
+    static int8_t extract_field(int64_t local_time, const cctz::time_zone& 
ctz) {
+        static const auto epoch = 
std::chrono::time_point_cast<cctz::sys_seconds>(
+                std::chrono::system_clock::from_time_t(0));
+        cctz::time_point<cctz::sys_seconds> t = epoch + 
cctz::seconds(local_time);
+        int offset = ctz.lookup_offset(t).offset;
+        local_time += offset;
+
+        static const libdivide::divider<int64_t> fast_div_3600(3600);
+        static const libdivide::divider<int64_t> fast_div_86400(86400);
+
+        int64_t remainder;
+        if (LIKELY(local_time >= 0)) {
+            remainder = local_time - local_time / fast_div_86400 * 86400;
+        } else {
+            remainder = local_time % 86400;
+            if (remainder < 0) {
+                remainder += 86400;
+            }
+        }
+        return static_cast<int8_t>(remainder / fast_div_3600);
+    }
+};
+
+struct MinuteFromUnixtimeImpl {
+    static constexpr PrimitiveType ArgType = PrimitiveType::TYPE_BIGINT;
+    static constexpr auto name = "minute_from_unixtime";
+
+    static int8_t extract_field(int64_t local_time, const cctz::time_zone& 
/*ctz*/) {
+        static const libdivide::divider<int64_t> fast_div_60(60);
+        static const libdivide::divider<int64_t> fast_div_3600(3600);
+
+        local_time = local_time - local_time / fast_div_3600 * 3600;
+
+        return static_cast<int8_t>(local_time / fast_div_60);
+    }
+};
+
+struct SecondFromUnixtimeImpl {
+    static constexpr PrimitiveType ArgType = PrimitiveType::TYPE_BIGINT;
+    static constexpr auto name = "second_from_unixtime";
+
+    static int8_t extract_field(int64_t local_time, const cctz::time_zone& 
/*ctz*/) {
+        return static_cast<int8_t>(local_time % 60);
+    }
+};
+
+struct MicrosecondFromUnixtimeImpl {
+    static constexpr PrimitiveType ArgType = PrimitiveType::TYPE_DECIMAL64;
+    static constexpr auto name = "microsecond_from_unixtime";
+
+    static int32_t extract_field(int64_t fraction, int scale) {
+        if (scale < 6) {
+            fraction *= common::exp10_i64(6 - scale);
+        }
+        return static_cast<int32_t>(fraction);
+    }
+};
+
 #include "common/compile_check_end.h"
 } // namespace doris::vectorized
 
diff --git a/be/src/vec/functions/function_time_value_to_field.cpp 
b/be/src/vec/functions/function_time_value_to_field.cpp
index e018b1b5edd..5374f004619 100644
--- a/be/src/vec/functions/function_time_value_to_field.cpp
+++ b/be/src/vec/functions/function_time_value_to_field.cpp
@@ -21,6 +21,7 @@
 #include "common/status.h"
 #include "vec/data_types/data_type_number.h"
 #include "vec/data_types/data_type_time.h"
+#include "vec/functions/date_time_transforms.h"
 #include "vec/functions/function.h"
 #include "vec/functions/function_date_or_datetime_computation.h"
 #include "vec/functions/simple_function_factory.h"
@@ -89,11 +90,20 @@ struct MicroImpl {
     static inline auto execute(const TimeValue::TimeType& t) { return 
TimeValue::microsecond(t); }
 };
 
+using FunctionHourFromUnixtime = 
FunctionTimeFieldFromUnixtime<HourFromUnixtimeImpl>;
+using FunctionMinuteFromUnixtime = 
FunctionTimeFieldFromUnixtime<MinuteFromUnixtimeImpl>;
+using FunctionSecondFromUnixtime = 
FunctionTimeFieldFromUnixtime<SecondFromUnixtimeImpl>;
+using FunctionMicrosecondFromUnixtime = 
FunctionTimeFieldFromUnixtime<MicrosecondFromUnixtimeImpl>;
+
 void register_function_time_value_field(SimpleFunctionFactory& factory) {
     factory.register_function<FunctionTimeValueToField<DataTypeInt32, 
HourImpl>>();
     factory.register_function<FunctionTimeValueToField<DataTypeInt8, 
MintuImpl>>();
     factory.register_function<FunctionTimeValueToField<DataTypeInt8, 
SecondImpl>>();
     factory.register_function<FunctionTimeValueToField<DataTypeInt32, 
MicroImpl>>();
+    factory.register_function<FunctionHourFromUnixtime>();
+    factory.register_function<FunctionMinuteFromUnixtime>();
+    factory.register_function<FunctionSecondFromUnixtime>();
+    factory.register_function<FunctionMicrosecondFromUnixtime>();
 }
 #include "common/compile_check_end.h"
 } // namespace doris::vectorized
\ No newline at end of file
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
index 7e62f190c9c..c8c6bde4f94 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
@@ -238,6 +238,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.HllToBase64;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Hour;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HourCeil;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HourFloor;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.HourFromUnixtime;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursDiff;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursSub;
@@ -335,6 +336,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MicroSecondsA
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MicroSecondsDiff;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MicroSecondsSub;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Microsecond;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MicrosecondFromUnixtime;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MilliSecondTimestamp;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MilliSecondsAdd;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MilliSecondsDiff;
@@ -342,6 +344,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MilliSecondsS
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Minute;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MinuteCeil;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MinuteFloor;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MinuteFromUnixtime;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MinutesAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MinutesDiff;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MinutesSub;
@@ -419,6 +422,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.SecToTime;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Second;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondCeil;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondFloor;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.SecondFromUnixtime;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.SecondTimestamp;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsDiff;
@@ -776,6 +780,7 @@ public class BuiltinScalarFunctions implements 
FunctionHelper {
             scalar(Hour.class, "hour"),
             scalar(HourCeil.class, "hour_ceil"),
             scalar(HourFloor.class, "hour_floor"),
+            scalar(HourFromUnixtime.class, "hour_from_unixtime"),
             scalar(HoursAdd.class, "hours_add"),
             scalar(HoursDiff.class, "hours_diff"),
             scalar(HoursSub.class, "hours_sub"),
@@ -877,6 +882,7 @@ public class BuiltinScalarFunctions implements 
FunctionHelper {
             scalar(Md5.class, "md5"),
             scalar(Md5Sum.class, "md5sum"),
             scalar(Microsecond.class, "microsecond"),
+            scalar(MicrosecondFromUnixtime.class, "microsecond_from_unixtime"),
             scalar(MicroSecondsAdd.class, "microseconds_add"),
             scalar(MicroSecondsDiff.class, "microseconds_diff"),
             scalar(MicroSecondsSub.class, "microseconds_sub"),
@@ -886,6 +892,7 @@ public class BuiltinScalarFunctions implements 
FunctionHelper {
             scalar(Minute.class, "minute"),
             scalar(MinuteCeil.class, "minute_ceil"),
             scalar(MinuteFloor.class, "minute_floor"),
+            scalar(MinuteFromUnixtime.class, "minute_from_unixtime"),
             scalar(MinutesAdd.class, "minutes_add"),
             scalar(MinutesDiff.class, "minutes_diff"),
             scalar(MinutesSub.class, "minutes_sub"),
@@ -964,6 +971,7 @@ public class BuiltinScalarFunctions implements 
FunctionHelper {
             scalar(Second.class, "second"),
             scalar(SecondCeil.class, "second_ceil"),
             scalar(SecondFloor.class, "second_floor"),
+            scalar(SecondFromUnixtime.class, "second_from_unixtime"),
             scalar(SecondsAdd.class, "seconds_add"),
             scalar(SecondsDiff.class, "seconds_diff"),
             scalar(SecondsSub.class, "seconds_sub"),
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java
index b5b806aba39..a75bc0ecce1 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java
@@ -35,6 +35,7 @@ import 
org.apache.doris.nereids.rules.expression.rules.SimplifyConflictCompound;
 import org.apache.doris.nereids.rules.expression.rules.SimplifyInPredicate;
 import org.apache.doris.nereids.rules.expression.rules.SimplifyRange;
 import org.apache.doris.nereids.rules.expression.rules.SimplifySelfComparison;
+import 
org.apache.doris.nereids.rules.expression.rules.SimplifyTimeFieldFromUnixtime;
 import org.apache.doris.nereids.rules.expression.rules.TopnToMax;
 
 import com.google.common.collect.ImmutableList;
@@ -56,6 +57,7 @@ public class ExpressionOptimization extends ExpressionRewrite 
{
                     // compound predicates
                     SimplifyRange.INSTANCE,
                     SimplifyConflictCompound.INSTANCE,
+                    SimplifyTimeFieldFromUnixtime.INSTANCE,
                     DistinctPredicatesRule.INSTANCE,
                     ExtractCommonFactorRule.INSTANCE,
 
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRuleType.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRuleType.java
index d44f1bf41c6..bcd19f46a44 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRuleType.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRuleType.java
@@ -57,6 +57,7 @@ public enum ExpressionRuleType {
     SIMPLIFY_COMPARISON_PREDICATE,
     SIMPLIFY_CONDITIONAL_FUNCTION,
     SIMPLIFY_CONFLICT_COMPOUND,
+    SIMPLIFY_DATETIME_FUNCTION,
     SIMPLIFY_EQUAL_BOOLEAN_LITERAL,
     SIMPLIFY_IN_PREDICATE,
     SIMPLIFY_NOT_EXPR,
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyTimeFieldFromUnixtime.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyTimeFieldFromUnixtime.java
new file mode 100644
index 00000000000..5797f55df95
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyTimeFieldFromUnixtime.java
@@ -0,0 +1,151 @@
+// 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.rules.expression.rules;
+
+import org.apache.doris.nereids.rules.expression.ExpressionMatchingContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
+import org.apache.doris.nereids.rules.expression.ExpressionRuleType;
+import org.apache.doris.nereids.trees.expressions.Cast;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.FromUnixtime;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Hour;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.HourFromUnixtime;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Microsecond;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MicrosecondFromUnixtime;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Minute;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MinuteFromUnixtime;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Second;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.SecondFromUnixtime;
+import org.apache.doris.nereids.types.BigIntType;
+import org.apache.doris.nereids.types.DataType;
+import org.apache.doris.nereids.types.DecimalV3Type;
+import org.apache.doris.nereids.util.TypeCoercionUtils;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * Simplify time extraction functions with from_unixtime():
+ * - hour(from_unixtime(ts)) -> hour_from_unixtime(ts)
+ * - minute(from_unixtime(ts)) -> minute_from_unixtime(ts)
+ * - second(from_unixtime(ts)) -> second_from_unixtime(ts)
+ * - microsecond(from_unixtime(ts)) -> microsecond_from_unixtime(ts)
+ */
+public class SimplifyTimeFieldFromUnixtime implements 
ExpressionPatternRuleFactory {
+    public static final SimplifyTimeFieldFromUnixtime INSTANCE = new 
SimplifyTimeFieldFromUnixtime();
+
+    @Override
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesType(Hour.class)
+                        .thenApply(SimplifyTimeFieldFromUnixtime::rewriteHour)
+                        .toRule(ExpressionRuleType.SIMPLIFY_DATETIME_FUNCTION),
+                matchesType(Minute.class)
+                        
.thenApply(SimplifyTimeFieldFromUnixtime::rewriteMinute)
+                        .toRule(ExpressionRuleType.SIMPLIFY_DATETIME_FUNCTION),
+                matchesType(Second.class)
+                        
.thenApply(SimplifyTimeFieldFromUnixtime::rewriteSecond)
+                        .toRule(ExpressionRuleType.SIMPLIFY_DATETIME_FUNCTION),
+                matchesType(Microsecond.class)
+                        
.thenApply(SimplifyTimeFieldFromUnixtime::rewriteMicrosecond)
+                        .toRule(ExpressionRuleType.SIMPLIFY_DATETIME_FUNCTION)
+        );
+    }
+
+    private static Expression rewriteHour(ExpressionMatchingContext<Hour> ctx) 
{
+        Hour hour = ctx.expr;
+        Expression nestedChild = removeCast(hour.child());
+        if (!(nestedChild instanceof FromUnixtime && nestedChild.arity() == 
1)) {
+            return hour;
+        }
+
+        FromUnixtime fromUnixtime = (FromUnixtime) nestedChild;
+        Expression tsArg = fromUnixtime.child(0);
+        if (!tsArg.getDataType().isNumericType() && !tsArg.isNullLiteral()) {
+            return hour;
+        }
+        Expression bigintArg = TypeCoercionUtils.castIfNotSameType(tsArg, 
BigIntType.INSTANCE);
+        Expression rewritten = new HourFromUnixtime(bigintArg);
+        return TypeCoercionUtils.ensureSameResultType(hour, rewritten, 
ctx.rewriteContext);
+    }
+
+    private static Expression rewriteMinute(ExpressionMatchingContext<Minute> 
ctx) {
+        Minute minute = ctx.expr;
+        Expression nestedChild = removeCast(minute.child());
+        if (!(nestedChild instanceof FromUnixtime && nestedChild.arity() == 
1)) {
+            return minute;
+        }
+
+        FromUnixtime fromUnixtime = (FromUnixtime) nestedChild;
+        Expression tsArg = fromUnixtime.child(0);
+        if (!tsArg.getDataType().isNumericType() && !tsArg.isNullLiteral()) {
+            return minute;
+        }
+        Expression bigintArg = TypeCoercionUtils.castIfNotSameType(tsArg, 
BigIntType.INSTANCE);
+        Expression rewritten = new MinuteFromUnixtime(bigintArg);
+        return TypeCoercionUtils.ensureSameResultType(minute, rewritten, 
ctx.rewriteContext);
+    }
+
+    private static Expression rewriteSecond(ExpressionMatchingContext<Second> 
ctx) {
+        Second second = ctx.expr;
+        Expression nestedChild = removeCast(second.child());
+        if (!(nestedChild instanceof FromUnixtime && nestedChild.arity() == 
1)) {
+            return second;
+        }
+
+        FromUnixtime fromUnixtime = (FromUnixtime) nestedChild;
+        Expression tsArg = fromUnixtime.child(0);
+        if (!tsArg.getDataType().isNumericType() && !tsArg.isNullLiteral()) {
+            return second;
+        }
+        Expression bigintArg = TypeCoercionUtils.castIfNotSameType(tsArg, 
BigIntType.INSTANCE);
+        Expression rewritten = new SecondFromUnixtime(bigintArg);
+        return TypeCoercionUtils.ensureSameResultType(second, rewritten, 
ctx.rewriteContext);
+    }
+
+    private static Expression 
rewriteMicrosecond(ExpressionMatchingContext<Microsecond> ctx) {
+        Microsecond microsecond = ctx.expr;
+        Expression nestedChild = removeCast(microsecond.child());
+        if (!(nestedChild instanceof FromUnixtime && nestedChild.arity() == 
1)) {
+            return microsecond;
+        }
+
+        FromUnixtime fromUnixtime = (FromUnixtime) nestedChild;
+        Expression tsArg = fromUnixtime.child(0);
+        if (!tsArg.getDataType().isNumericType() && !tsArg.isNullLiteral()) {
+            return microsecond;
+        }
+        Expression decimalArg = TypeCoercionUtils.castIfNotSameType(tsArg,
+                                        DecimalV3Type.createDecimalV3Type(18, 
6));
+        Expression rewritten = new MicrosecondFromUnixtime(decimalArg);
+        return TypeCoercionUtils.ensureSameResultType(microsecond, rewritten, 
ctx.rewriteContext);
+    }
+
+    private static Expression removeCast(Expression expr) {
+        Expression current = expr;
+        if (current instanceof Cast) {
+            DataType nestedType = current.getDataType();
+            if (nestedType.isDateTimeType() || nestedType.isDateTimeV2Type()) {
+                current = ((Cast) current).child();
+            }
+        }
+        return current;
+    }
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeExtractAndTransform.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeExtractAndTransform.java
index ea6551687ba..d63b935dedc 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeExtractAndTransform.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeExtractAndTransform.java
@@ -102,6 +102,9 @@ public class DateTimeExtractAndTransform {
         DAY_OF_WEEK.put("SUNDAY", 7);
     }
 
+    // Maximum valid timestamp value (UTC 9999-12-31 23:59:59 - 24 * 3600 for 
all timezones)
+    private static final long TIMESTAMP_VALID_MAX = 32536771199L;
+
     /**
      * datetime arithmetic function date-v2
      */
@@ -1334,4 +1337,85 @@ public class DateTimeExtractAndTransform {
         }
         return year * 100 + month % 12 + 1;
     }
+
+    /**
+     * date extract function hour_from_unixtime
+     */
+    @ExecFunction(name = "hour_from_unixtime")
+    public static Expression hourFromUnixtime(BigIntLiteral unixTime) {
+        long epochSecond = unixTime.getValue();
+        if (epochSecond < 0 || epochSecond > TIMESTAMP_VALID_MAX) {
+            throw new AnalysisException("Function hour_from_unixtime out of 
range(between 0 and "
+                            + TIMESTAMP_VALID_MAX + "): " + epochSecond);
+        }
+
+        ZoneId timeZone = DateUtils.getTimeZone();
+        ZonedDateTime zonedDateTime = 
Instant.ofEpochSecond(epochSecond).atZone(timeZone);
+        return new TinyIntLiteral((byte) zonedDateTime.getHour());
+    }
+
+    /**
+     * date extract function minute_from_unixtime
+     */
+    @ExecFunction(name = "minute_from_unixtime")
+    public static Expression minuteFromUnixtime(BigIntLiteral unixTime) {
+        long localTime = unixTime.getValue();
+        if (localTime < 0 || localTime > TIMESTAMP_VALID_MAX) {
+            throw new AnalysisException("Function minute_from_unixtime out of 
range(between 0 and "
+                    + TIMESTAMP_VALID_MAX + "): " + localTime);
+        }
+
+        localTime = localTime - (localTime / 3600) * 3600;
+
+        byte minute = (byte) (localTime / 60);
+        return new TinyIntLiteral(minute);
+    }
+
+    /**
+     * date extract function second_from_unixtime
+     */
+    @ExecFunction(name = "second_from_unixtime")
+    public static Expression secondFromUnixtime(BigIntLiteral unixTime) {
+        long localTime = unixTime.getValue();
+        if (localTime < 0 || localTime > TIMESTAMP_VALID_MAX) {
+            throw new AnalysisException("Function second_from_unixtime out of 
range(between 0 and "
+                    + TIMESTAMP_VALID_MAX + "): " + localTime);
+        }
+
+        long remainder;
+        if (localTime >= 0) {
+            remainder = localTime % 60;
+        } else {
+            remainder = localTime % 60;
+            if (remainder < 0) {
+                remainder += 60;
+            }
+        }
+        return new TinyIntLiteral((byte) remainder);
+    }
+
+    /**
+     * date extract function microsecond_from_unixtime
+     */
+    @ExecFunction(name = "microsecond_from_unixtime")
+    public static Expression microsecondFromUnixtime(DecimalV3Literal 
unixTime) {
+        BigDecimal value = unixTime.getValue();
+
+        long seconds = value.longValue();
+        if (seconds < 0 || seconds > TIMESTAMP_VALID_MAX) {
+            throw new AnalysisException("Function microsecond_from_unixtime 
out of range(between 0 and "
+                    + TIMESTAMP_VALID_MAX + "): " + seconds);
+        }
+
+        DecimalV3Type dataType = (DecimalV3Type) unixTime.getDataType();
+        int scale = dataType.getScale();
+
+        BigDecimal fractional = value.remainder(BigDecimal.ONE);
+        long fraction = fractional.movePointRight(scale).longValue();
+
+        if (scale < 6) {
+            fraction *= (long) Math.pow(10, 6 - scale);
+        }
+        return new IntegerLiteral((int) fraction);
+    }
 }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HourFromUnixtime.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HourFromUnixtime.java
new file mode 100644
index 00000000000..ac024cc7cc4
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/HourFromUnixtime.java
@@ -0,0 +1,75 @@
+// 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.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.BigIntType;
+import org.apache.doris.nereids.types.TinyIntType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * Scalar Function 'hour_from_unixtime'
+ * Optimized version of `HOUR(FROM_UNIXTIME(ts))`
+ */
+public class HourFromUnixtime extends ScalarFunction
+        implements UnaryExpression, ExplicitlyCastableSignature, 
PropagateNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(TinyIntType.INSTANCE).args(BigIntType.INSTANCE)
+    );
+
+    /**
+     * constructor with 1 argument.
+     */
+    public HourFromUnixtime(Expression arg) {
+        super("hour_from_unixtime", arg);
+    }
+
+    /** constructor for withChildren and reuse signature */
+    private HourFromUnixtime(ScalarFunctionParams functionParams) {
+        super(functionParams);
+    }
+
+    /**
+     * withChildren.
+     */
+    @Override
+    public HourFromUnixtime withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new HourFromUnixtime(getFunctionParams(children));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitHourFromUnixtime(this, context);
+    }
+
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MicrosecondFromUnixtime.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MicrosecondFromUnixtime.java
new file mode 100644
index 00000000000..afb7d77ecfa
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MicrosecondFromUnixtime.java
@@ -0,0 +1,75 @@
+// 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.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DecimalV3Type;
+import org.apache.doris.nereids.types.IntegerType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * Scalar Function 'microsecond_from_unixtime'
+ * Optimized version of `MICROSECOND(FROM_UNIXTIME(ts))`
+ */
+public class MicrosecondFromUnixtime extends ScalarFunction
+        implements UnaryExpression, ExplicitlyCastableSignature, 
PropagateNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(IntegerType.INSTANCE).args(DecimalV3Type.createDecimalV3Type(18,
 6))
+    );
+
+    /**
+     * constructor with 1 argument.
+     */
+    public MicrosecondFromUnixtime(Expression arg) {
+        super("microsecond_from_unixtime", arg);
+    }
+
+    /** constructor for withChildren and reuse signature */
+    private MicrosecondFromUnixtime(ScalarFunctionParams functionParams) {
+        super(functionParams);
+    }
+
+    /**
+     * withChildren.
+     */
+    @Override
+    public MicrosecondFromUnixtime withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new MicrosecondFromUnixtime(getFunctionParams(children));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitMicrosecondFromUnixtime(this, context);
+    }
+
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinuteFromUnixtime.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinuteFromUnixtime.java
new file mode 100644
index 00000000000..6bc00aee47e
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MinuteFromUnixtime.java
@@ -0,0 +1,75 @@
+// 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.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.BigIntType;
+import org.apache.doris.nereids.types.TinyIntType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * Scalar Function 'minute_from_unixtime'
+ * Optimized version of `MINUTE(FROM_UNIXTIME(ts))`
+ */
+public class MinuteFromUnixtime extends ScalarFunction
+        implements UnaryExpression, ExplicitlyCastableSignature, 
PropagateNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(TinyIntType.INSTANCE).args(BigIntType.INSTANCE)
+    );
+
+    /**
+     * constructor with 1 argument.
+     */
+    public MinuteFromUnixtime(Expression arg) {
+        super("minute_from_unixtime", arg);
+    }
+
+    /** constructor for withChildren and reuse signature */
+    private MinuteFromUnixtime(ScalarFunctionParams functionParams) {
+        super(functionParams);
+    }
+
+    /**
+     * withChildren.
+     */
+    @Override
+    public MinuteFromUnixtime withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new MinuteFromUnixtime(getFunctionParams(children));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitMinuteFromUnixtime(this, context);
+    }
+
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondFromUnixtime.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondFromUnixtime.java
new file mode 100644
index 00000000000..3f4398229f7
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SecondFromUnixtime.java
@@ -0,0 +1,75 @@
+// 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.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.BigIntType;
+import org.apache.doris.nereids.types.TinyIntType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * Scalar Function 'second_from_unixtime'
+ * Optimized version of `SECOND(FROM_UNIXTIME(ts))`
+ */
+public class SecondFromUnixtime extends ScalarFunction
+        implements UnaryExpression, ExplicitlyCastableSignature, 
PropagateNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(TinyIntType.INSTANCE).args(BigIntType.INSTANCE)
+    );
+
+    /**
+     * constructor with 1 argument.
+     */
+    public SecondFromUnixtime(Expression arg) {
+        super("second_from_unixtime", arg);
+    }
+
+    /** constructor for withChildren and reuse signature */
+    private SecondFromUnixtime(ScalarFunctionParams functionParams) {
+        super(functionParams);
+    }
+
+    /**
+     * withChildren.
+     */
+    @Override
+    public SecondFromUnixtime withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new SecondFromUnixtime(getFunctionParams(children));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitSecondFromUnixtime(this, context);
+    }
+
+}
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 5a556add837..c1db9cbd288 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
@@ -246,6 +246,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.HllToBase64;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Hour;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HourCeil;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HourFloor;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.HourFromUnixtime;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursDiff;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursSub;
@@ -342,12 +343,14 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MicroSecondsA
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MicroSecondsDiff;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MicroSecondsSub;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Microsecond;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MicrosecondFromUnixtime;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MilliSecondsAdd;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MilliSecondsDiff;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MilliSecondsSub;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Minute;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MinuteCeil;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MinuteFloor;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MinuteFromUnixtime;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MinuteSecondAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MinutesAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MinutesDiff;
@@ -424,6 +427,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.Sec;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Second;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondCeil;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondFloor;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.SecondFromUnixtime;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.SecondMicrosecondAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsDiff;
@@ -1441,6 +1445,22 @@ public interface ScalarFunctionVisitor<R, C> {
         return visitScalarFunction(hourFloor, context);
     }
 
+    default R visitHourFromUnixtime(HourFromUnixtime hourFromUnixtime, C 
context) {
+        return visitScalarFunction(hourFromUnixtime, context);
+    }
+
+    default R visitMinuteFromUnixtime(MinuteFromUnixtime minuteFromUnixtime, C 
context) {
+        return visitScalarFunction(minuteFromUnixtime, context);
+    }
+
+    default R visitSecondFromUnixtime(SecondFromUnixtime secondFromUnixtime, C 
context) {
+        return visitScalarFunction(secondFromUnixtime, context);
+    }
+
+    default R visitMicrosecondFromUnixtime(MicrosecondFromUnixtime 
microsecondFromUnixtime, C context) {
+        return visitScalarFunction(microsecondFromUnixtime, context);
+    }
+
     default R visitHoursDiff(HoursDiff hoursDiff, C context) {
         return visitScalarFunction(hoursDiff, context);
     }
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyTimeFieldFromUnixtimeTest.java
 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyTimeFieldFromUnixtimeTest.java
new file mode 100644
index 00000000000..a1812ef088f
--- /dev/null
+++ 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyTimeFieldFromUnixtimeTest.java
@@ -0,0 +1,86 @@
+// 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.rules.expression;
+
+import 
org.apache.doris.nereids.rules.expression.rules.SimplifyTimeFieldFromUnixtime;
+import org.apache.doris.nereids.trees.expressions.Expression;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Maps;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.Map;
+
+/**
+ * Tests for {@link SimplifyTimeFieldFromUnixtime}.
+ */
+public class SimplifyTimeFieldFromUnixtimeTest extends 
ExpressionRewriteTestHelper {
+    public SimplifyTimeFieldFromUnixtimeTest() {
+        executor = new ExpressionRuleExecutor(ImmutableList.of(
+            bottomUp(SimplifyTimeFieldFromUnixtime.INSTANCE)));
+    }
+
+    @Test
+    public void testRewriteSimple() {
+        assertRewriteAfterTypeCoercion("hour(from_unixtime(IA))", 
"hour_from_unixtime(IA)");
+        assertRewriteAfterTypeCoercion("minute(from_unixtime(IA))", 
"minute_from_unixtime(IA)");
+        assertRewriteAfterTypeCoercion("second(from_unixtime(IA))", 
"second_from_unixtime(IA)");
+        
assertRewriteAfterTypeCoercion("microsecond(from_unixtime(DECIMAL_V3_A))", 
"microsecond_from_unixtime(cast(DECIMAL_V3_A as DECIMALV3(18, 6)))");
+    }
+
+    @Test
+    public void testRewriteWithCast() {
+        assertRewriteAfterTypeCoercion("hour(cast(from_unixtime(IA) as 
datetime))",
+                "hour_from_unixtime(IA)");
+        assertRewriteAfterTypeCoercion("minute(cast(from_unixtime(IA) as 
datetime))",
+                "minute_from_unixtime(IA)");
+        assertRewriteAfterTypeCoercion("second(cast(from_unixtime(IA) as 
datetime))",
+                "second_from_unixtime(IA)");
+        
assertRewriteAfterTypeCoercion("microsecond(cast(from_unixtime(DECIMAL_V3_A) as 
datetimev2(6)))",
+                "microsecond_from_unixtime(cast(DECIMAL_V3_A as DECIMALV3(18, 
6)))");
+    }
+
+    @Test
+    public void testNoRewriteOnFormattedCall() {
+        Map<String, org.apache.doris.nereids.trees.expressions.Slot> memo = 
Maps.newHashMap();
+        Expression expression = replaceUnboundSlot(
+                PARSER.parseExpression("hour(from_unixtime(IA, 
'yyyy-MM-dd'))"), memo);
+        expression = typeCoercion(expression);
+        Expression rewritten = executor.rewrite(expression, context);
+        Assertions.assertEquals(expression, rewritten);
+
+        expression = replaceUnboundSlot(
+                PARSER.parseExpression("minute(from_unixtime(IA, 
'yyyy-MM-dd'))"), memo);
+        expression = typeCoercion(expression);
+        rewritten = executor.rewrite(expression, context);
+        Assertions.assertEquals(expression, rewritten);
+
+        expression = replaceUnboundSlot(
+                PARSER.parseExpression("second(from_unixtime(IA, 
'yyyy-MM-dd'))"), memo);
+        expression = typeCoercion(expression);
+        rewritten = executor.rewrite(expression, context);
+        Assertions.assertEquals(expression, rewritten);
+
+        expression = replaceUnboundSlot(
+                
PARSER.parseExpression("microsecond(from_unixtime(DECIMAL_V3_A, 
'yyyy-MM-dd'))"), memo);
+        expression = typeCoercion(expression);
+        rewritten = executor.rewrite(expression, context);
+        Assertions.assertEquals(expression, rewritten);
+    }
+}
diff --git a/regression-test/data/datatype_p0/date/test_from_unixtime.out 
b/regression-test/data/datatype_p0/date/test_from_unixtime.out
index fea5f5e0b31..83bd7766cdc 100644
--- a/regression-test/data/datatype_p0/date/test_from_unixtime.out
+++ b/regression-test/data/datatype_p0/date/test_from_unixtime.out
@@ -22,3 +22,73 @@
 -- !sql8 --
 3001-01-19 00:00:00
 
+-- !hour_from_unixtime1 --
+0
+11
+16
+
+-- !hour_from_unixtime2 --
+0
+
+-- !hour_from_unixtime3 --
+7
+
+-- !hour_from_unixtime4 --
+0
+
+-- !hour_from_unixtime5 --
+0
+
+-- !hour_from_unixtime6 --
+16
+
+-- !minute_from_unixtime1 --
+1
+33
+0
+
+-- !minute_from_unixtime2 --
+0
+
+-- !minute_from_unixtime3 --
+48
+
+-- !minute_from_unixtime4 --
+19
+
+-- !minute_from_unixtime5 --
+0
+
+-- !second_from_unixtime1 --
+40
+20
+0
+
+-- !second_from_unixtime2 --
+0
+
+-- !second_from_unixtime3 --
+34
+
+-- !second_from_unixtime4 --
+5
+
+-- !second_from_unixtime5 --
+0
+
+-- !microsecond_from_unixtime1 --
+123000
+1000
+123457
+
+-- !microsecond_from_unixtime2 --
+0
+
+-- !microsecond_from_unixtime3 --
+123456
+
+-- !microsecond_from_unixtime4 --
+140000
+
+-- !microsecond_from_unixtime5 --
+0
diff --git a/regression-test/suites/datatype_p0/date/test_from_unixtime.groovy 
b/regression-test/suites/datatype_p0/date/test_from_unixtime.groovy
index 5c5612f9a75..e5e7c3f9ce4 100644
--- a/regression-test/suites/datatype_p0/date/test_from_unixtime.groovy
+++ b/regression-test/suites/datatype_p0/date/test_from_unixtime.groovy
@@ -56,4 +56,98 @@ suite("test_from_unixtime") {
     // qt_sql10 "select from_unixtime(-7629445119491449);"
 
     // qt_long "select from_unixtime(1196440219, '%f %V %f %l %V %I %S %p %w 
%r %j %f %l %I %D %w %j %D %e %s %V %f %D %M %s %X %U %v %c %u %x %r %j %a %h 
%s %m %a %v %u %b');"
-}
+
+    // HOUR(FROM_UNIXTIME(ts)) -> hour_from_unixtime(ts)
+    sql "set debug_skip_fold_constant = true;"
+    qt_hour_from_unixtime1 "SELECT HOUR(FROM_UNIXTIME(k0)) FROM test1 ORDER BY 
k0;"
+    qt_hour_from_unixtime2 "SELECT HOUR(FROM_UNIXTIME(114514, 'yyyy-MM-dd'));"
+    qt_hour_from_unixtime3 "SELECT HOUR(FROM_UNIXTIME(114514, 'yyyy-MM-dd 
HH:mm:ss'));"
+    qt_hour_from_unixtime4 "SELECT HOUR(FROM_UNIXTIME(1145.14));"
+    qt_hour_from_unixtime5 "SELECT HOUR(FROM_UNIXTIME(32536771200));"
+    sql "set time_zone = '-08:00'"
+    qt_hour_from_unixtime6 "SELECT HOUR(FROM_UNIXTIME(3));"
+    test {
+        sql """ SELECT HOUR(FROM_UNIXTIME(-1)); """
+        exception "The input value of TimeFiled(from_unixtime()) must between 
0 and 253402243199"
+    }
+    explain {
+        sql """ SELECT HOUR(FROM_UNIXTIME(k0)) FROM test1; """
+        contains "hour_from_unixtime"
+    }
+    // should only applicable to the `from_unixtime` function with a single 
parameter.
+    explain {
+        sql """SELECT HOUR(FROM_UNIXTIME(k0, 'yyyy-MM-dd HH:mm:ss')) FROM 
test1;"""
+        notContains "hour_from_unixtime"
+    }
+    testFoldConst("SELECT HOUR_FROM_UNIXTIME(1145.14);")
+    testFoldConst("SELECT HOUR_FROM_UNIXTIME(NULL);")
+    testFoldConst("SELECT HOUR_FROM_UNIXTIME(32536771200);")
+    sql "set time_zone = '+00:00'"
+
+    // MINUTE(FROM_UNIXTIME(ts)) -> minute_from_unixtime(ts)
+    qt_minute_from_unixtime1 "SELECT MINUTE(FROM_UNIXTIME(k0)) FROM test1 
ORDER BY k0;"
+    qt_minute_from_unixtime2 "SELECT MINUTE(FROM_UNIXTIME(114514, 
'yyyy-MM-dd'));"
+    qt_minute_from_unixtime3 "SELECT MINUTE(FROM_UNIXTIME(114514, 'yyyy-MM-dd 
HH:mm:ss'));"
+    qt_minute_from_unixtime4 "SELECT MINUTE(FROM_UNIXTIME(1145.14));"
+    qt_minute_from_unixtime5 "SELECT MINUTE(FROM_UNIXTIME(32536771200));"
+    test {
+        sql """ SELECT MINUTE(FROM_UNIXTIME(-1)); """
+        exception "The input value of TimeFiled(from_unixtime()) must between 
0 and 253402243199"
+    }
+    explain {
+        sql """ SELECT MINUTE(FROM_UNIXTIME(k0)) FROM test1; """
+        contains "minute_from_unixtime"
+    }
+    explain {
+        sql """SELECT MINUTE(FROM_UNIXTIME(k0, 'yyyy-MM-dd HH:mm:ss')) FROM 
test1;"""
+        notContains "minute_from_unixtime"
+    }
+    testFoldConst("SELECT MINUTE_FROM_UNIXTIME(1145.14);")
+    testFoldConst("SELECT MINUTE_FROM_UNIXTIME(NULL);")
+    testFoldConst("SELECT MINUTE_FROM_UNIXTIME(32536771200);")
+
+    // SECOND(FROM_UNIXTIME(ts)) -> second_from_unixtime(ts)
+    qt_second_from_unixtime1 "SELECT SECOND(FROM_UNIXTIME(k0)) FROM test1 
ORDER BY k0;"
+    qt_second_from_unixtime2 "SELECT SECOND(FROM_UNIXTIME(114514, 
'yyyy-MM-dd'));"
+    qt_second_from_unixtime3 "SELECT SECOND(FROM_UNIXTIME(114514, 'yyyy-MM-dd 
HH:mm:ss'));"
+    qt_second_from_unixtime4 "SELECT SECOND(FROM_UNIXTIME(1145.14));"
+    qt_second_from_unixtime5 "SELECT SECOND(FROM_UNIXTIME(32536771200));"
+    test {
+        sql """ SELECT SECOND(FROM_UNIXTIME(-1)); """
+        exception "The input value of TimeFiled(from_unixtime()) must between 
0 and 253402243199"
+    }
+    explain {
+        sql """ SELECT SECOND(FROM_UNIXTIME(k0)) FROM test1; """
+        contains "second_from_unixtime"
+    }
+    explain {
+        sql """SELECT SECOND(FROM_UNIXTIME(k0, 'yyyy-MM-dd HH:mm:ss')) FROM 
test1;"""
+        notContains "second_from_unixtime"
+    }
+    testFoldConst("SELECT SECOND_FROM_UNIXTIME(1145.14);")
+    testFoldConst("SELECT SECOND_FROM_UNIXTIME(NULL);")
+    testFoldConst("SELECT SECOND_FROM_UNIXTIME(32536771200);")
+
+    // MICROSECOND(FROM_UNIXTIME(ts)) -> microsecond_from_unixtime(ts)
+    qt_microsecond_from_unixtime1 "SELECT MICROSECOND(FROM_UNIXTIME(k1)) FROM 
test1 ORDER BY k1;"
+    qt_microsecond_from_unixtime2 "SELECT 
MICROSECOND(FROM_UNIXTIME(114514.123456, 'yyyy-MM-dd'));"
+    qt_microsecond_from_unixtime3 "SELECT 
MICROSECOND(FROM_UNIXTIME(114514.123456, '%Y-%m-%d %H:%i:%s.%f'));"
+    qt_microsecond_from_unixtime4 "SELECT MICROSECOND(FROM_UNIXTIME(1145.14));"
+    qt_microsecond_from_unixtime5 "SELECT 
MICROSECOND(FROM_UNIXTIME(32536771200));"
+    test {
+        sql """ SELECT MICROSECOND(FROM_UNIXTIME(-1)); """
+        exception "The input value of TimeFiled(from_unixtime()) must between 
0 and 253402243199"
+    }
+    explain {
+        sql """ SELECT MICROSECOND(FROM_UNIXTIME(k1)) FROM test1; """
+        contains "microsecond_from_unixtime"
+    }
+    explain {
+        sql """SELECT MICROSECOND(FROM_UNIXTIME(k1, 'yyyy-MM-dd HH:mm:ss')) 
FROM test1;"""
+        notContains "microsecond_from_unixtime"
+    }
+    testFoldConst("SELECT MICROSECOND_FROM_UNIXTIME(1145.14);")
+    testFoldConst("SELECT MICROSECOND_FROM_UNIXTIME(NULL);")
+    testFoldConst("SELECT MICROSECOND_FROM_UNIXTIME(32536771200);")
+    sql "set debug_skip_fold_constant = false;"
+}
\ No newline at end of file


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

Reply via email to