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

lihaopeng 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 32d4b08989a [opt](function) optimize from_unixtime/date_format by 
specially format str (#40821)
32d4b08989a is described below

commit 32d4b08989a4359342bdc796e1fabfbf422656c0
Author: Mryange <59914473+mrya...@users.noreply.github.com>
AuthorDate: Wed Sep 18 11:36:19 2024 +0800

    [opt](function) optimize from_unixtime/date_format by specially format str 
(#40821)
    
    ```
     mysql [test]>select count(date_format(a, 'yyyyMMdd')) from date_format_tmp;
    +-----------------------------------+
    | count(date_format(a, 'yyyyMMdd')) |
    +-----------------------------------+
    |                          16000000 |
    +-----------------------------------+
    1 row in set (0.53 sec)
    
    
    mysql [test]>select count(date_format(a, 'yyyyMMdd')) from date_format_tmp;
    +-----------------------------------+
    | count(date_format(a, 'yyyyMMdd')) |
    +-----------------------------------+
    |                          16000000 |
    +-----------------------------------+
    1 row in set (0.28 sec)
    ```
---
 be/src/vec/functions/date_format_type.h            | 156 +++++++++++++++++++++
 be/src/vec/functions/date_time_transforms.h        | 104 +++++++++-----
 .../functions/function_datetime_string_to_string.h | 140 ++++++++++++++----
 .../datetime_functions/test_date_function.out      |  15 ++
 .../datetime_functions/test_date_function.groovy   |   9 +-
 5 files changed, 357 insertions(+), 67 deletions(-)

diff --git a/be/src/vec/functions/date_format_type.h 
b/be/src/vec/functions/date_format_type.h
new file mode 100644
index 00000000000..071ecf44853
--- /dev/null
+++ b/be/src/vec/functions/date_format_type.h
@@ -0,0 +1,156 @@
+// 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.
+
+#pragma once
+
+#include <variant>
+
+#include "vec/common/string_ref.h"
+
+namespace doris::vectorized::time_format_type {
+// Used to optimize commonly used date formats.
+
+inline StringRef rewrite_specific_format(const char* raw_str, size_t str_size) 
{
+    const static std::string specific_format_strs[3] = {"%Y%m%d", "%Y-%m-%d", 
"%Y-%m-%d %H:%i:%s"};
+    const static std::string specific_format_rewrite[3] = {"yyyyMMdd", 
"yyyy-MM-dd",
+                                                           "yyyy-MM-dd 
HH:mm:ss"};
+    for (int i = 0; i < 3; i++) {
+        const StringRef specific_format {specific_format_strs[i].data(),
+                                         specific_format_strs[i].size()};
+        if (specific_format == StringRef {raw_str, str_size}) {
+            return {specific_format_rewrite[i].data(), 
specific_format_rewrite[i].size()};
+        }
+    }
+    return {raw_str, str_size};
+}
+
+template <typename T>
+void put_year(T y, char* buf, int& i) {
+    int t = y / 100;
+    buf[i++] = t / 10 + '0';
+    buf[i++] = t % 10 + '0';
+
+    t = y % 100;
+    buf[i++] = t / 10 + '0';
+    buf[i++] = t % 10 + '0';
+}
+
+template <typename T>
+void put_other(T m, char* buf, int& i) {
+    buf[i++] = m / 10 + '0';
+    buf[i++] = m % 10 + '0';
+}
+
+// NoneImpl indicates that no specific optimization has been applied, and the 
general logic is used for processing.
+struct NoneImpl {};
+
+struct yyyyMMddImpl {
+    template <typename DateType>
+    size_t static date_to_str(const DateType& date_value, char* buf) {
+        int i = 0;
+        put_year(date_value.year(), buf, i);
+        put_other(date_value.month(), buf, i);
+        put_other(date_value.day(), buf, i);
+        return i;
+    }
+};
+
+struct yyyy_MM_ddImpl {
+    template <typename DateType>
+    size_t static date_to_str(const DateType& date_value, char* buf) {
+        int i = 0;
+        put_year(date_value.year(), buf, i);
+        buf[i++] = '-';
+        put_other(date_value.month(), buf, i);
+        buf[i++] = '-';
+        put_other(date_value.day(), buf, i);
+        return i;
+    }
+};
+
+struct yyyy_MM_dd_HH_mm_ssImpl {
+    template <typename DateType>
+    size_t static date_to_str(const DateType& date_value, char* buf) {
+        int i = 0;
+        put_year(date_value.year(), buf, i);
+        buf[i++] = '-';
+        put_other(date_value.month(), buf, i);
+        buf[i++] = '-';
+        put_other(date_value.day(), buf, i);
+        buf[i++] = ' ';
+        put_other(date_value.hour(), buf, i);
+        buf[i++] = ':';
+        put_other(date_value.minute(), buf, i);
+        buf[i++] = ':';
+        put_other(date_value.second(), buf, i);
+        return i;
+    }
+};
+
+struct yyyy_MMImpl {
+    template <typename DateType>
+    size_t static date_to_str(const DateType& date_value, char* buf) {
+        int i = 0;
+        put_year(date_value.year(), buf, i);
+        buf[i++] = '-';
+        put_other(date_value.month(), buf, i);
+        return i;
+    }
+};
+struct yyyyMMImpl {
+    template <typename DateType>
+    size_t static date_to_str(const DateType& date_value, char* buf) {
+        int i = 0;
+        put_year(date_value.year(), buf, i);
+        put_other(date_value.month(), buf, i);
+        return i;
+    }
+};
+
+struct yyyyImpl {
+    template <typename DateType>
+    size_t static date_to_str(const DateType& date_value, char* buf) {
+        int i = 0;
+        put_year(date_value.year(), buf, i);
+        return i;
+    }
+};
+
+using FormatImplVariant = std::variant<NoneImpl, yyyyMMddImpl, yyyy_MM_ddImpl,
+                                       yyyy_MM_dd_HH_mm_ssImpl, yyyy_MMImpl, 
yyyyMMImpl, yyyyImpl>;
+
+const static std::string default_format = "yyyy-MM-dd HH:mm:ss";
+const static auto default_impl = yyyy_MM_dd_HH_mm_ssImpl {};
+inline FormatImplVariant string_to_impl(const std::string& format) {
+    if (format == "yyyyMMdd" || format == "%Y%m%d") {
+        return yyyyMMddImpl {};
+    } else if (format == "yyyy-MM-dd" || format == "%Y-%m-%d") {
+        return yyyy_MM_ddImpl {};
+    } else if (format == "yyyy-MM-dd HH:mm:ss" || format == "%Y-%m-%d 
%H:%i:%s") {
+        return yyyy_MM_dd_HH_mm_ssImpl {};
+    } else if (format == "yyyy-MM") {
+        return yyyy_MMImpl {};
+    } else if (format == "yyyyMM") {
+        return yyyyMMImpl {};
+    } else if (format == "yyyy") {
+        return yyyyImpl {};
+    } else {
+        return NoneImpl {};
+    }
+}
+
+} // namespace doris::vectorized::time_format_type
diff --git a/be/src/vec/functions/date_time_transforms.h 
b/be/src/vec/functions/date_time_transforms.h
index 266c9b5d272..84824d74ff1 100644
--- a/be/src/vec/functions/date_time_transforms.h
+++ b/be/src/vec/functions/date_time_transforms.h
@@ -33,6 +33,7 @@
 #include "vec/core/types.h"
 #include "vec/data_types/data_type_date_time.h"
 #include "vec/data_types/data_type_string.h"
+#include "vec/functions/date_format_type.h"
 #include "vec/runtime/vdatetime_value.h"
 #include "vec/utils/util.hpp"
 
@@ -184,34 +185,44 @@ struct DateFormatImpl {
 
     static constexpr auto name = "date_format";
 
-    static inline auto execute(const FromType& t, StringRef format, 
ColumnString::Chars& res_data,
-                               size_t& offset) {
-        const auto& dt = (DateType&)t;
-        if (format.size > 128) {
-            return std::pair {offset, true};
-        }
-        char buf[100 + SAFE_FORMAT_STRING_MARGIN];
-        if (!dt.to_format_string_conservative(format.data, format.size, buf,
-                                              100 + 
SAFE_FORMAT_STRING_MARGIN)) {
-            return std::pair {offset, true};
-        }
+    template <typename Impl>
+    static inline bool execute(const FromType& t, StringRef format, 
ColumnString::Chars& res_data,
+                               size_t& offset, const cctz::time_zone& 
time_zone) {
+        if constexpr (std::is_same_v<Impl, time_format_type::NoneImpl>) {
+            // Handle non-special formats.
+            const auto& dt = (DateType&)t;
+            char buf[100 + SAFE_FORMAT_STRING_MARGIN];
+            if (!dt.to_format_string_conservative(format.data, format.size, 
buf,
+                                                  100 + 
SAFE_FORMAT_STRING_MARGIN)) {
+                return true;
+            }
+
+            auto len = strlen(buf);
+            res_data.insert(buf, buf + len);
+            offset += len;
+            return false;
+        } else {
+            const auto& dt = (DateType&)t;
 
-        auto len = strlen(buf);
-        res_data.insert(buf, buf + len);
-        offset += len;
-        return std::pair {offset, false};
+            if (!dt.is_valid_date()) {
+                return true;
+            }
+
+            // No buffer is needed here because these specially optimized 
formats have fixed lengths,
+            // and sufficient memory has already been reserved.
+            auto len = Impl::date_to_str(dt, (char*)res_data.data() + offset);
+            offset += len;
+
+            return false;
+        }
     }
 
     static DataTypes get_variadic_argument_types() {
-        return std::vector<DataTypePtr> {
-                std::dynamic_pointer_cast<const IDataType>(
-                        std::make_shared<typename 
DateTraits<ArgType>::DateType>()),
-                std::dynamic_pointer_cast<const IDataType>(
-                        std::make_shared<vectorized::DataTypeString>())};
+        return std::vector<DataTypePtr> {std::make_shared<typename 
DateTraits<ArgType>::DateType>(),
+                                         
std::make_shared<vectorized::DataTypeString>()};
     }
 };
 
-// TODO: This function should be depend on arguments not always nullable
 template <typename DateType>
 struct FromUnixTimeImpl {
     using FromType = Int64;
@@ -220,24 +231,45 @@ struct FromUnixTimeImpl {
     static const int64_t TIMESTAMP_VALID_MAX = 32536771199;
     static constexpr auto name = "from_unixtime";
 
-    static inline auto execute(FromType val, StringRef format, 
ColumnString::Chars& res_data,
+    template <typename Impl>
+    static inline bool execute(const FromType& val, StringRef format, 
ColumnString::Chars& res_data,
                                size_t& offset, const cctz::time_zone& 
time_zone) {
-        DateType dt;
-        if (format.size > 128 || val < 0 || val > TIMESTAMP_VALID_MAX) {
-            return std::pair {offset, true};
-        }
-        dt.from_unixtime(val, time_zone);
+        if constexpr (std::is_same_v<Impl, time_format_type::NoneImpl>) {
+            DateType dt;
+            if (val < 0 || val > TIMESTAMP_VALID_MAX) {
+                return true;
+            }
+            dt.from_unixtime(val, time_zone);
 
-        char buf[100 + SAFE_FORMAT_STRING_MARGIN];
-        if (!dt.to_format_string_conservative(format.data, format.size, buf,
-                                              100 + 
SAFE_FORMAT_STRING_MARGIN)) {
-            return std::pair {offset, true};
-        }
+            char buf[100 + SAFE_FORMAT_STRING_MARGIN];
+            if (!dt.to_format_string_conservative(format.data, format.size, 
buf,
+                                                  100 + 
SAFE_FORMAT_STRING_MARGIN)) {
+                return true;
+            }
 
-        auto len = strlen(buf);
-        res_data.insert(buf, buf + len);
-        offset += len;
-        return std::pair {offset, false};
+            auto len = strlen(buf);
+            res_data.insert(buf, buf + len);
+            offset += len;
+            return false;
+
+        } else {
+            DateType dt;
+            if (val < 0 || val > TIMESTAMP_VALID_MAX) {
+                return true;
+            }
+            dt.from_unixtime(val, time_zone);
+
+            if (!dt.is_valid_date()) {
+                return true;
+            }
+
+            // No buffer is needed here because these specially optimized 
formats have fixed lengths,
+            // and sufficient memory has already been reserved.
+            auto len = Impl::date_to_str(dt, (char*)res_data.data() + offset);
+            offset += len;
+
+            return false;
+        }
     }
 };
 
diff --git a/be/src/vec/functions/function_datetime_string_to_string.h 
b/be/src/vec/functions/function_datetime_string_to_string.h
index 41eba51301c..80fe6cf1f41 100644
--- a/be/src/vec/functions/function_datetime_string_to_string.h
+++ b/be/src/vec/functions/function_datetime_string_to_string.h
@@ -21,6 +21,7 @@
 
 #include <memory>
 #include <utility>
+#include <variant>
 
 #include "common/status.h"
 #include "vec/aggregate_functions/aggregate_function.h"
@@ -29,6 +30,7 @@
 #include "vec/columns/column_string.h"
 #include "vec/columns/column_vector.h"
 #include "vec/columns/columns_number.h"
+#include "vec/common/assert_cast.h"
 #include "vec/common/string_ref.h"
 #include "vec/core/block.h"
 #include "vec/core/column_numbers.h"
@@ -38,6 +40,7 @@
 #include "vec/data_types/data_type.h"
 #include "vec/data_types/data_type_nullable.h"
 #include "vec/data_types/data_type_string.h"
+#include "vec/functions/date_format_type.h"
 #include "vec/functions/date_time_transforms.h"
 #include "vec/functions/function.h"
 #include "vec/runtime/vdatetime_value.h"
@@ -66,6 +69,57 @@ public:
         return {};
     }
 
+    struct FormatState {
+        std::string format_str;
+        // Check if the format string is null or exceeds the length limit.
+        bool is_valid = true;
+        time_format_type::FormatImplVariant format_type;
+    };
+
+    Status open(FunctionContext* context, FunctionContext::FunctionStateScope 
scope) override {
+        if (scope == FunctionContext::THREAD_LOCAL) {
+            return Status::OK();
+        }
+        std::shared_ptr<FormatState> state = std::make_shared<FormatState>();
+        DCHECK((context->get_num_args() == 1) || (context->get_num_args() == 
2));
+        context->set_function_state(scope, state);
+        if (context->get_num_args() == 1) {
+            // default argument
+            state->format_str = time_format_type::default_format;
+            state->format_type = time_format_type::default_impl;
+            return IFunction::open(context, scope);
+        }
+
+        const auto* column_string = context->get_constant_col(1);
+
+        if (column_string == nullptr) {
+            return Status::InvalidArgument(
+                    "The second parameter of the function {} must be a 
constant.", get_name());
+        }
+
+        auto string_vale = column_string->column_ptr->get_data_at(0);
+        if (string_vale.data == nullptr) {
+            // func(col , null);
+            state->is_valid = false;
+            return IFunction::open(context, scope);
+        }
+
+        string_vale = string_vale.trim();
+        auto format_str =
+                time_format_type::rewrite_specific_format(string_vale.data, 
string_vale.size);
+        if (format_str.size > 128) {
+            //  exceeds the length limit.
+            state->is_valid = false;
+            return IFunction::open(context, scope);
+        }
+
+        // Preprocess special format strings.
+        state->format_str = format_str;
+        state->format_type = 
time_format_type::string_to_impl(state->format_str);
+
+        return IFunction::open(context, scope);
+    }
+
     DataTypePtr get_return_type_impl(const ColumnsWithTypeAndName& arguments) 
const override {
         return make_nullable(std::make_shared<DataTypeString>());
     }
@@ -78,42 +132,68 @@ public:
         const ColumnPtr source_col = 
block.get_by_position(arguments[0]).column;
 
         const auto* nullable_column = 
check_and_get_column<ColumnNullable>(source_col.get());
-        const auto* sources = check_and_get_column<ColumnVector<typename 
Transform::FromType>>(
+        const auto* sources = assert_cast<const ColumnVector<typename 
Transform::FromType>*>(
                 nullable_column ? 
nullable_column->get_nested_column_ptr().get()
                                 : source_col.get());
 
-        if (sources) {
-            auto col_res = ColumnString::create();
-            ColumnUInt8::MutablePtr col_null_map_to;
-            col_null_map_to = ColumnUInt8::create();
-            auto& vec_null_map_to = col_null_map_to->get_data();
-
-            if (arguments.size() == 2) {
-                const IColumn& source_col1 = 
*block.get_by_position(arguments[1]).column;
-                StringRef formatter =
-                        source_col1.get_data_at(0); // for both ColumnString 
or ColumnConst.
-                TransformerToStringTwoArgument<Transform>::vector_constant(
-                        context, sources->get_data(), formatter, 
col_res->get_chars(),
-                        col_res->get_offsets(), vec_null_map_to);
-            } else { //default argument
-                TransformerToStringTwoArgument<Transform>::vector_constant(
-                        context, sources->get_data(), StringRef("%Y-%m-%d 
%H:%i:%s"),
-                        col_res->get_chars(), col_res->get_offsets(), 
vec_null_map_to);
-            }
+        auto col_res = ColumnString::create();
+        ColumnUInt8::MutablePtr col_null_map_to;
+        col_null_map_to = ColumnUInt8::create();
+        auto& vec_null_map_to = col_null_map_to->get_data();
 
-            if (nullable_column) {
-                const auto& origin_null_map = 
nullable_column->get_null_map_column().get_data();
-                for (int i = 0; i < origin_null_map.size(); ++i) {
-                    vec_null_map_to[i] |= origin_null_map[i];
-                }
+        RETURN_IF_ERROR(vector_constant(context, sources->get_data(), 
col_res->get_chars(),
+                                        col_res->get_offsets(), 
vec_null_map_to));
+
+        if (nullable_column) {
+            // input column is nullable
+            const auto& origin_null_map = 
nullable_column->get_null_map_column().get_data();
+            for (int i = 0; i < origin_null_map.size(); ++i) {
+                vec_null_map_to[i] |= origin_null_map[i];
             }
-            block.get_by_position(result).column =
-                    ColumnNullable::create(std::move(col_res), 
std::move(col_null_map_to));
-        } else {
-            return Status::InternalError("Illegal column {} of first argument 
of function {}",
-                                         
block.get_by_position(arguments[0]).column->get_name(),
-                                         name);
         }
+
+        block.get_by_position(result).column =
+                ColumnNullable::create(std::move(col_res), 
std::move(col_null_map_to));
+
+        return Status::OK();
+    }
+
+    Status vector_constant(FunctionContext* context,
+                           const PaddedPODArray<typename Transform::FromType>& 
ts,
+                           ColumnString::Chars& res_data, 
ColumnString::Offsets& res_offsets,
+                           PaddedPODArray<UInt8>& null_map) const {
+        auto* format_state = reinterpret_cast<FormatState*>(
+                context->get_function_state(FunctionContext::FRAGMENT_LOCAL));
+        if (!format_state) {
+            return Status::RuntimeError("funciton context for function '{}' 
must have FormatState;",
+                                        get_name());
+        }
+
+        StringRef format(format_state->format_str);
+
+        const auto len = ts.size();
+
+        if (!format_state->is_valid) {
+            res_offsets.resize_fill(len, 0);
+            null_map.resize_fill(len, true);
+            return Status::OK();
+        }
+        res_offsets.resize(len);
+        res_data.reserve(len * format.size + len);
+        null_map.resize_fill(len, false);
+
+        std::visit(
+                [&](auto type) {
+                    using Impl = decltype(type);
+                    size_t offset = 0;
+                    for (int i = 0; i < len; ++i) {
+                        null_map[i] = Transform::template execute<Impl>(
+                                ts[i], format, res_data, offset, 
context->state()->timezone_obj());
+                        res_offsets[i] = offset;
+                    }
+                    res_data.resize(offset);
+                },
+                format_state->format_type);
         return Status::OK();
     }
 };
diff --git 
a/regression-test/data/nereids_p0/sql_functions/datetime_functions/test_date_function.out
 
b/regression-test/data/nereids_p0/sql_functions/datetime_functions/test_date_function.out
index 8c256e42d57..2aef8a1257a 100644
--- 
a/regression-test/data/nereids_p0/sql_functions/datetime_functions/test_date_function.out
+++ 
b/regression-test/data/nereids_p0/sql_functions/datetime_functions/test_date_function.out
@@ -147,6 +147,12 @@
 -- !sql --
 2009-10-04
 
+-- !sql_date_format_long --
+\N
+
+-- !sql_date_format_long --
+\N
+
 -- !sql --
 2008-11-30T23:59:59
 
@@ -476,6 +482,12 @@ February
 -- !sql --
 1      2022-08-01 17:00:31
 
+-- !sql --
+1      \N
+
+-- !sql --
+1      \N
+
 -- !sql --
 true
 
@@ -494,6 +506,9 @@ true
 -- !sql_date_format_long --
 \N
 
+-- !sql_date_format_long --
+\N
+
 -- !sql --
 \N
 
diff --git 
a/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function.groovy
 
b/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function.groovy
index 0a986f249e5..ae748997839 100644
--- 
a/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function.groovy
+++ 
b/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function.groovy
@@ -248,6 +248,9 @@ suite("test_date_function") {
     sql """ truncate table ${tableName} """
     sql """ insert into ${tableName} values ("2009-10-04 22:23:00") """
     qt_sql """ select date_format(test_datetime, 'yyyy-MM-dd') from 
${tableName}; """
+    qt_sql_date_format_long """ select date_format(test_datetime, '%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') from ${tableName};"""
+    qt_sql_date_format_long """ select 
date_format(non_nullable(test_datetime), '%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') from ${tableName};"""
+  
     sql """ truncate table ${tableName} """
 
     sql """ insert into ${tableName} values ("2010-11-30 23:59:59") """
@@ -465,7 +468,9 @@ suite("test_date_function") {
     qt_sql """ SELECT id,FROM_UNIXTIME(update_time,"%Y-%m-%d %H:%i:%s") FROM 
${tableName} WHERE FROM_UNIXTIME(update_time,"%Y-%m-%d %H:%i:%s") <= 
'2022-08-01 00:00:00' ORDER BY id; """
     qt_sql """ SELECT id,FROM_UNIXTIME(update_time,"%Y-%m-%d %H:%i:%s") FROM 
${tableName} WHERE FROM_UNIXTIME(update_time,"%Y-%m-%d %H:%i:%s") LIKE 
'2022-08-01 00:00:00' ORDER BY id; """
     qt_sql """ SELECT id,FROM_UNIXTIME(update_time,"%Y-%m-%d %H:%i:%s") FROM 
${tableName} WHERE FROM_UNIXTIME(update_time,"%Y-%m-%d %H:%i:%s") = '2022-08-01 
17:00:31' ORDER BY id; """
-
+    qt_sql """ SELECT id,FROM_UNIXTIME(update_time,null) FROM ${tableName} 
WHERE FROM_UNIXTIME(update_time,"%Y-%m-%d %H:%i:%s") = '2022-08-01 17:00:31' 
ORDER BY id; """
+    qt_sql """ SELECT id,FROM_UNIXTIME(update_time,'%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') FROM ${tableName} WHERE FROM_UNIXTIME(update_time,"%Y-%m-%d 
%H:%i:%s") = '2022-08-01 17:00:31' ORDER BY id; """
+    
     qt_sql """SELECT CURDATE() = CURRENT_DATE();"""
     qt_sql """SELECT unix_timestamp(CURDATE()) = 
unix_timestamp(CURRENT_DATE());"""
 
@@ -475,6 +480,8 @@ suite("test_date_function") {
     qt_sql """ select date_format('2025-01-01', '%X %V'); """
     qt_sql """ select date_format('2022-08-04', '%X %V %w'); """
     qt_sql_date_format_long """ select date_format(cast('2011-06-24' as 
DATETIMEV2(0)), '%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') """
+    qt_sql_date_format_long """ select date_format(null, '%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') """
+    
     qt_sql """ select STR_TO_DATE('Tue Jul 12 20:00:45 CST 2022', '%a %b %e 
%H:%i:%s %Y'); """
     qt_sql """ select STR_TO_DATE('Tue Jul 12 20:00:45 CST 2022', '%a %b %e %T 
CST %Y'); """
     qt_sql """ select STR_TO_DATE('2018-4-2 15:3:28','%Y-%m-%d %H:%i:%s'); """


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

Reply via email to