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 <[email protected]>
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: [email protected]
For additional commands, e-mail: [email protected]