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]