mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Updated function dateName before merge
This commit is contained in:
parent
5191fecb09
commit
ac2f9dd15a
@ -1,24 +1,19 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <common/DateLUTImpl.h>
|
||||
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
|
||||
#include <Functions/DateTimeTransforms.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/FunctionsConversion.h>
|
||||
#include <Functions/IFunctionImpl.h>
|
||||
#include <Functions/castTypeToEither.h>
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <common/DateLUTImpl.h>
|
||||
|
||||
#include <type_traits>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
@ -32,10 +27,25 @@ namespace ErrorCodes
|
||||
namespace
|
||||
{
|
||||
|
||||
template <typename DataType> struct ActionValueTypeMap {};
|
||||
template <> struct ActionValueTypeMap<DataTypeDate> { using ActionValueType = UInt16; };
|
||||
template <> struct ActionValueTypeMap<DataTypeDateTime> { using ActionValueType = UInt32; };
|
||||
template <> struct ActionValueTypeMap<DataTypeDateTime64> { using ActionValueType = Int64; };
|
||||
template <typename DataType> struct DataTypeToTimeTypeMap {};
|
||||
|
||||
template <> struct DataTypeToTimeTypeMap<DataTypeDate>
|
||||
{
|
||||
using TimeType = UInt16;
|
||||
};
|
||||
|
||||
template <> struct DataTypeToTimeTypeMap<DataTypeDateTime>
|
||||
{
|
||||
using TimeType = UInt32;
|
||||
};
|
||||
|
||||
template <> struct DataTypeToTimeTypeMap<DataTypeDateTime64>
|
||||
{
|
||||
using TimeType = Int64;
|
||||
};
|
||||
|
||||
template <typename DataType>
|
||||
using DateTypeToTimeType = typename DataTypeToTimeTypeMap<DataType>::TimeType;
|
||||
|
||||
class FunctionDateNameImpl : public IFunction
|
||||
{
|
||||
@ -61,24 +71,30 @@ public:
|
||||
"Number of arguments for function {} doesn't match: passed {}",
|
||||
getName(),
|
||||
toString(arguments.size()));
|
||||
|
||||
if (!WhichDataType(arguments[0].type).isString())
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of 1 argument of function {}. Must be string",
|
||||
arguments[0].type->getName(),
|
||||
getName());
|
||||
if (!WhichDataType(arguments[1].type).isDateOrDateTime())
|
||||
|
||||
WhichDataType first_argument_type(arguments[1].type);
|
||||
|
||||
if (!(first_argument_type.isDate() || first_argument_type.isDateTime() || first_argument_type.isDateTime64()))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of 2 argument of function {}. Must be a date or a date with time",
|
||||
arguments[1].type->getName(),
|
||||
getName());
|
||||
|
||||
if (arguments.size() == 3 && !WhichDataType(arguments[2].type).isString())
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of 3 argument of function {}. Must be string",
|
||||
arguments[2].type->getName(),
|
||||
getName());
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
@ -89,8 +105,9 @@ public:
|
||||
{
|
||||
ColumnPtr res;
|
||||
|
||||
if (!((res = executeType<DataTypeDate>(arguments, result_type)) || (res = executeType<DataTypeDateTime>(arguments, result_type))
|
||||
|| (res = executeType<DataTypeDateTime64>(arguments, result_type))))
|
||||
if (!((res = executeType<DataTypeDate>(arguments, result_type))
|
||||
|| (res = executeType<DataTypeDateTime>(arguments, result_type))
|
||||
|| (res = executeType<DataTypeDateTime64>(arguments, result_type))))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of function {], must be Date or DateTime.",
|
||||
@ -107,24 +124,15 @@ public:
|
||||
if (!times)
|
||||
return nullptr;
|
||||
|
||||
const ColumnConst * datepart_column = checkAndGetColumnConst<ColumnString>(arguments[0].column.get());
|
||||
if (!datepart_column)
|
||||
const ColumnConst * date_part_column = checkAndGetColumnConst<ColumnString>(arguments[0].column.get());
|
||||
if (!date_part_column)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of first ('datepart') argument of function {}. Must be constant string.",
|
||||
arguments[0].column->getName(),
|
||||
getName());
|
||||
|
||||
using T = typename ActionValueTypeMap<DataType>::ActionValueType;
|
||||
auto datepart_writer = DatePartWriter<T>();
|
||||
String datepart = datepart_column->getValue<String>();
|
||||
|
||||
if (!datepart_writer.isCorrectDatePart(datepart))
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Illegal value {} of first ('format') argument of function {}. Check documentation",
|
||||
datepart,
|
||||
getName());
|
||||
String date_part = date_part_column->getValue<String>();
|
||||
|
||||
const DateLUTImpl * time_zone_tmp;
|
||||
if (std::is_same_v<DataType, DataTypeDateTime64> || std::is_same_v<DataType, DataTypeDateTime>)
|
||||
@ -132,203 +140,210 @@ public:
|
||||
else
|
||||
time_zone_tmp = &DateLUT::instance();
|
||||
|
||||
const auto & vec = times->getData();
|
||||
const auto & times_data = times->getData();
|
||||
const DateLUTImpl & time_zone = *time_zone_tmp;
|
||||
|
||||
UInt32 scale [[maybe_unused]] = 0;
|
||||
if constexpr (std::is_same_v<DataType, DataTypeDateTime64>)
|
||||
{
|
||||
scale = vec.getScale();
|
||||
scale = times_data.getScale();
|
||||
}
|
||||
|
||||
auto col_res = ColumnString::create();
|
||||
auto & dst_data = col_res->getChars();
|
||||
auto & dst_offsets = col_res->getOffsets();
|
||||
dst_data.resize(vec.size() * (9 /* longest possible word 'Wednesday' */ + 1 /* zero terminator */));
|
||||
dst_offsets.resize(vec.size());
|
||||
auto result_column = ColumnString::create();
|
||||
auto & result_column_data = result_column->getChars();
|
||||
auto & result_column_offsets = result_column->getOffsets();
|
||||
|
||||
auto * begin = reinterpret_cast<char *>(dst_data.data());
|
||||
auto * pos = begin;
|
||||
/* longest possible word 'Wednesday' with zero terminator */
|
||||
static constexpr size_t longest_word_length = 9 + 1;
|
||||
|
||||
for (size_t i = 0; i < vec.size(); ++i)
|
||||
result_column_data.resize_fill(times_data.size() * longest_word_length);
|
||||
result_column_offsets.resize(times_data.size());
|
||||
|
||||
auto * begin = reinterpret_cast<char *>(result_column_data.data());
|
||||
|
||||
WriteBuffer buffer(begin, result_column_data.size());
|
||||
|
||||
using TimeType = DateTypeToTimeType<DataType>;
|
||||
callOnDatePartWriter<TimeType>(date_part, [&](const auto & writer)
|
||||
{
|
||||
if constexpr (std::is_same_v<DataType, DataTypeDateTime64>)
|
||||
for (size_t i = 0; i < times_data.size(); ++i)
|
||||
{
|
||||
// since right now LUT does not support Int64-values and not format instructions for subsecond parts,
|
||||
// treat DatTime64 values just as DateTime values by ignoring fractional and casting to UInt32.
|
||||
const auto c = DecimalUtils::split(vec[i], scale);
|
||||
datepart_writer.writeDatePart(pos, datepart, static_cast<UInt32>(c.whole), time_zone);
|
||||
if constexpr (std::is_same_v<DataType, DataTypeDateTime64>)
|
||||
{
|
||||
const auto components = DecimalUtils::split(times_data[i], scale);
|
||||
writer.write(buffer, static_cast<Int64>(components.whole), time_zone);
|
||||
}
|
||||
else
|
||||
{
|
||||
writer.write(buffer, times_data[i], time_zone);
|
||||
}
|
||||
|
||||
/// Null terminator
|
||||
++buffer.position();
|
||||
result_column_offsets[i] = buffer.position() - begin;
|
||||
}
|
||||
else
|
||||
{
|
||||
datepart_writer.writeDatePart(pos, datepart, vec[i], time_zone);
|
||||
}
|
||||
dst_offsets[i] = pos - begin;
|
||||
++pos;
|
||||
}
|
||||
dst_data.resize(pos - begin);
|
||||
return col_res;
|
||||
});
|
||||
|
||||
result_column_data.resize(buffer.position() - begin);
|
||||
|
||||
return result_column;
|
||||
}
|
||||
|
||||
private:
|
||||
|
||||
template <typename Time>
|
||||
class DatePartWriter
|
||||
struct YearWriter
|
||||
{
|
||||
public:
|
||||
void writeDatePart(char *& target, const String & datepart, Time source, const DateLUTImpl & timezone)
|
||||
static void write(WriteBuffer & buffer, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
datepart_functions.at(datepart)(target, source, timezone);
|
||||
}
|
||||
|
||||
bool isCorrectDatePart(const String & datepart) { return datepart_functions.find(datepart) != datepart_functions.end(); }
|
||||
|
||||
private:
|
||||
const std::unordered_map<String, void (*)(char *&, Time, const DateLUTImpl &)> datepart_functions = {
|
||||
{"year", writeYear},
|
||||
{"quarter", writeQuarter},
|
||||
{"month", writeMonth},
|
||||
{"dayofyear", writeDayOfYear},
|
||||
{"day", writeDay},
|
||||
{"week", writeWeek},
|
||||
{"weekday", writeWeekday},
|
||||
{"hour", writeHour},
|
||||
{"minute", writeMinute},
|
||||
{"second", writeSecond},
|
||||
};
|
||||
|
||||
static inline void writeYear(char *& target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber(target, ToYearImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static inline void writeQuarter(char *& target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber(target, ToQuarterImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static inline void writeMonth(char *& target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
const auto month = ToMonthImpl::execute(source, timezone);
|
||||
static constexpr std::string_view monthnames[]
|
||||
= {"January",
|
||||
"February",
|
||||
"March",
|
||||
"April",
|
||||
"May",
|
||||
"June",
|
||||
"July",
|
||||
"August",
|
||||
"September",
|
||||
"October",
|
||||
"November",
|
||||
"December"};
|
||||
writeString(target, monthnames[month - 1]);
|
||||
}
|
||||
|
||||
static inline void writeDayOfYear(char *& target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber(target, ToDayOfYearImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static inline void writeDay(char *& target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber(target, ToDayOfMonthImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static inline void writeWeek(char *& target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber(target, ToISOWeekImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static inline void writeWeekday(char *& target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
const auto day = ToDayOfWeekImpl::execute(source, timezone);
|
||||
static constexpr std::string_view daynames[] = {"Monday", "Tuesday", "Wednesday", "Thursday", "Friday", "Saturday", "Sunday"};
|
||||
writeString(target, daynames[day - 1]);
|
||||
}
|
||||
|
||||
static inline void writeHour(char *& target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber(target, ToHourImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static inline void writeMinute(char *& target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber(target, ToMinuteImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static inline void writeSecond(char *& target, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeNumber(target, ToSecondImpl::execute(source, timezone));
|
||||
}
|
||||
|
||||
static inline void writeString(char *& target, const std::string_view & value)
|
||||
{
|
||||
size_t size = value.size() + 1; /// With zero terminator
|
||||
memcpy(target, value.data(), size);
|
||||
target += size;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static inline void writeNumber(char *& target, T value)
|
||||
{
|
||||
if (value < 10)
|
||||
{
|
||||
*target = value + '0';
|
||||
target += 2;
|
||||
*target = '\0';
|
||||
}
|
||||
else if (value < 100)
|
||||
{
|
||||
writeNumber2(target, value);
|
||||
target += 3;
|
||||
*target = '\0';
|
||||
}
|
||||
else if (value < 1000)
|
||||
{
|
||||
writeNumber3(target, value);
|
||||
target += 4;
|
||||
*target = '\0';
|
||||
}
|
||||
else if (value < 10000)
|
||||
{
|
||||
writeNumber4(target, value);
|
||||
target += 5;
|
||||
*target = '\0';
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
"Illegal value of second ('datetime') argument of function dateName. Check documentation.",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static inline void writeNumber2(char * p, T v)
|
||||
{
|
||||
memcpy(p, &digits100[v * 2], 2);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static inline void writeNumber3(char * p, T v)
|
||||
{
|
||||
writeNumber2(p, v / 10);
|
||||
p[2] = v % 10 + '0';
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static inline void writeNumber4(char * p, T v)
|
||||
{
|
||||
writeNumber2(p, v / 100);
|
||||
writeNumber2(p + 2, v % 100);
|
||||
writeText(ToYearImpl::execute(source, timezone), buffer);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Time>
|
||||
struct QuarterWriter
|
||||
{
|
||||
static inline void write(WriteBuffer & buffer, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeText(ToQuarterImpl::execute(source, timezone), buffer);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Time>
|
||||
struct MonthWriter
|
||||
{
|
||||
static inline void write(WriteBuffer & buffer, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
const auto month = ToMonthImpl::execute(source, timezone);
|
||||
static constexpr std::string_view month_names[] =
|
||||
{
|
||||
"January",
|
||||
"February",
|
||||
"March",
|
||||
"April",
|
||||
"May",
|
||||
"June",
|
||||
"July",
|
||||
"August",
|
||||
"September",
|
||||
"October",
|
||||
"November",
|
||||
"December"
|
||||
};
|
||||
|
||||
writeText(month_names[month - 1], buffer);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Time>
|
||||
struct WeekWriter
|
||||
{
|
||||
static inline void write(WriteBuffer & buffer, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeText(ToISOWeekImpl::execute(source, timezone), buffer);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Time>
|
||||
struct DayOfYearWriter
|
||||
{
|
||||
static inline void write(WriteBuffer & buffer, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeText(ToDayOfYearImpl::execute(source, timezone), buffer);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Time>
|
||||
struct DayWriter
|
||||
{
|
||||
static inline void write(WriteBuffer & buffer, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeText(ToDayOfMonthImpl::execute(source, timezone), buffer);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Time>
|
||||
struct WeekDayWriter
|
||||
{
|
||||
static inline void write(WriteBuffer & buffer, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
const auto day = ToDayOfWeekImpl::execute(source, timezone);
|
||||
static constexpr std::string_view day_names[] =
|
||||
{
|
||||
"Monday",
|
||||
"Tuesday",
|
||||
"Wednesday",
|
||||
"Thursday",
|
||||
"Friday",
|
||||
"Saturday",
|
||||
"Sunday"
|
||||
};
|
||||
|
||||
writeText(day_names[day - 1], buffer);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Time>
|
||||
struct HourWriter
|
||||
{
|
||||
static inline void write(WriteBuffer & buffer, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeText(ToHourImpl::execute(source, timezone), buffer);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Time>
|
||||
struct MinuteWriter
|
||||
{
|
||||
static inline void write(WriteBuffer & buffer, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeText(ToMinuteImpl::execute(source, timezone), buffer);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Time>
|
||||
struct SecondWriter
|
||||
{
|
||||
static inline void write(WriteBuffer & buffer, Time source, const DateLUTImpl & timezone)
|
||||
{
|
||||
writeText(ToSecondImpl::execute(source, timezone), buffer);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Time, typename Call>
|
||||
void callOnDatePartWriter(const String & date_part, Call && call) const
|
||||
{
|
||||
if (date_part == "year")
|
||||
std::forward<Call>(call)(YearWriter<Time>());
|
||||
else if (date_part == "quarter")
|
||||
std::forward<Call>(call)(QuarterWriter<Time>());
|
||||
else if (date_part == "month")
|
||||
std::forward<Call>(call)(MonthWriter<Time>());
|
||||
else if (date_part == "week")
|
||||
std::forward<Call>(call)(WeekWriter<Time>());
|
||||
else if (date_part == "dayofyear")
|
||||
std::forward<Call>(call)(DayOfYearWriter<Time>());
|
||||
else if (date_part == "day")
|
||||
std::forward<Call>(call)(DayWriter<Time>());
|
||||
else if (date_part == "weekday")
|
||||
std::forward<Call>(call)(WeekDayWriter<Time>());
|
||||
else if (date_part == "hour")
|
||||
std::forward<Call>(call)(HourWriter<Time>());
|
||||
else if (date_part == "minute")
|
||||
std::forward<Call>(call)(MinuteWriter<Time>());
|
||||
else if (date_part == "second")
|
||||
std::forward<Call>(call)(SecondWriter<Time>());
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid date part {} for function {}", date_part, getName());
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
void registerFunctionDateName(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionDateNameImpl>(FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -880,6 +880,7 @@ inline std::enable_if_t<std::is_floating_point_v<T>, void>
|
||||
writeText(const T & x, WriteBuffer & buf) { writeFloatText(x, buf); }
|
||||
|
||||
inline void writeText(const String & x, WriteBuffer & buf) { writeString(x.c_str(), x.size(), buf); }
|
||||
inline void writeText(const std::string_view & x, WriteBuffer & buf) { writeString(x.data(), x.size(), buf); }
|
||||
|
||||
/// Implemented as template specialization (not function overload) to avoid preference over templates on arithmetic types above.
|
||||
template <> inline void writeText<bool>(const bool & x, WriteBuffer & buf) { writeBoolText(x, buf); }
|
||||
|
@ -1,47 +1,12 @@
|
||||
2021
|
||||
2
|
||||
April
|
||||
104
|
||||
14
|
||||
15
|
||||
Wednesday
|
||||
11
|
||||
22
|
||||
33
|
||||
2021
|
||||
2
|
||||
April
|
||||
104
|
||||
14
|
||||
15
|
||||
Wednesday
|
||||
11
|
||||
22
|
||||
33
|
||||
2021
|
||||
2
|
||||
April
|
||||
104
|
||||
14
|
||||
15
|
||||
Wednesday
|
||||
11 8
|
||||
Monday
|
||||
Tuesday
|
||||
Wednesday
|
||||
Thursday
|
||||
Friday
|
||||
Saturday
|
||||
Sunday
|
||||
January
|
||||
February
|
||||
March
|
||||
April
|
||||
May
|
||||
June
|
||||
July
|
||||
August
|
||||
September
|
||||
October
|
||||
November
|
||||
December
|
||||
2021 2021 2021
|
||||
2 2 2
|
||||
April April April
|
||||
104 104 104
|
||||
14 14 14
|
||||
15 15 15
|
||||
Wednesday Wednesday Wednesday
|
||||
11 11
|
||||
22 22
|
||||
33 33
|
||||
Wednesday 23 22 33
|
||||
Thursday 2 22 33
|
||||
|
@ -1,56 +1,72 @@
|
||||
SELECT dateName('year', toDateTime('2021-04-14 11:22:33'));
|
||||
SELECT dateName('quarter', toDateTime('2021-04-14 11:22:33'));
|
||||
SELECT dateName('month', toDateTime('2021-04-14 11:22:33'));
|
||||
SELECT dateName('dayofyear', toDateTime('2021-04-14 11:22:33'));
|
||||
SELECT dateName('day', toDateTime('2021-04-14 11:22:33'));
|
||||
SELECT dateName('week', toDateTime('2021-04-14 11:22:33'));
|
||||
SELECT dateName('weekday', toDateTime('2021-04-14 11:22:33'));
|
||||
SELECT dateName('hour', toDateTime('2021-04-14 11:22:33'));
|
||||
SELECT dateName('minute', toDateTime('2021-04-14 11:22:33'));
|
||||
SELECT dateName('second', toDateTime('2021-04-14 11:22:33'));
|
||||
WITH
|
||||
toDate('2021-04-14') AS date_value,
|
||||
toDateTime('2021-04-14 11:22:33') AS date_time_value,
|
||||
toDateTime64('2021-04-14 11:22:33', 3) AS date_time_64_value
|
||||
SELECT dateName('year', date_value), dateName('year', date_time_value), dateName('year', date_time_64_value);
|
||||
|
||||
WITH
|
||||
toDate('2021-04-14') AS date_value,
|
||||
toDateTime('2021-04-14 11:22:33') AS date_time_value,
|
||||
toDateTime64('2021-04-14 11:22:33', 3) AS date_time_64_value
|
||||
SELECT dateName('quarter', date_value), dateName('quarter', date_time_value), dateName('quarter', date_time_64_value);
|
||||
|
||||
SELECT dateName('year', toDateTime64('2021-04-14 11:22:33', 3));
|
||||
SELECT dateName('quarter', toDateTime64('2021-04-14 11:22:33', 3));
|
||||
SELECT dateName('month', toDateTime64('2021-04-14 11:22:33', 3));
|
||||
SELECT dateName('dayofyear', toDateTime64('2021-04-14 11:22:33', 3));
|
||||
SELECT dateName('day', toDateTime64('2021-04-14 11:22:33', 3));
|
||||
SELECT dateName('week', toDateTime64('2021-04-14 11:22:33', 3));
|
||||
SELECT dateName('weekday', toDateTime64('2021-04-14 11:22:33', 3));
|
||||
SELECT dateName('hour', toDateTime64('2021-04-14 11:22:33', 3));
|
||||
SELECT dateName('minute', toDateTime64('2021-04-14 11:22:33', 3));
|
||||
SELECT dateName('second', toDateTime64('2021-04-14 11:22:33', 3));
|
||||
WITH
|
||||
toDate('2021-04-14') AS date_value,
|
||||
toDateTime('2021-04-14 11:22:33') AS date_time_value,
|
||||
toDateTime64('2021-04-14 11:22:33', 3) AS date_time_64_value
|
||||
SELECT dateName('month', date_value), dateName('month', date_time_value), dateName('month', date_time_64_value);
|
||||
|
||||
WITH
|
||||
toDate('2021-04-14') AS date_value,
|
||||
toDateTime('2021-04-14 11:22:33') AS date_time_value,
|
||||
toDateTime64('2021-04-14 11:22:33', 3) AS date_time_64_value
|
||||
SELECT dateName('dayofyear', date_value), dateName('dayofyear', date_time_value), dateName('dayofyear', date_time_64_value);
|
||||
|
||||
SELECT dateName('year', toDate('2021-04-14'));
|
||||
SELECT dateName('quarter', toDate('2021-04-14'));
|
||||
SELECT dateName('month', toDate('2021-04-14'));
|
||||
SELECT dateName('dayofyear', toDate('2021-04-14'));
|
||||
SELECT dateName('day', toDate('2021-04-14'));
|
||||
SELECT dateName('week', toDate('2021-04-14'));
|
||||
SELECT dateName('weekday', toDate('2021-04-14'));
|
||||
WITH
|
||||
toDate('2021-04-14') AS date_value,
|
||||
toDateTime('2021-04-14 11:22:33') AS date_time_value,
|
||||
toDateTime64('2021-04-14 11:22:33', 3) AS date_time_64_value
|
||||
SELECT dateName('day', date_value), dateName('day', date_time_value), dateName('day', date_time_64_value);
|
||||
|
||||
WITH
|
||||
toDate('2021-04-14') AS date_value,
|
||||
toDateTime('2021-04-14 11:22:33') AS date_time_value,
|
||||
toDateTime64('2021-04-14 11:22:33', 3) AS date_time_64_value
|
||||
SELECT dateName('week', date_value), dateName('week', date_time_value), dateName('week', date_time_64_value);
|
||||
|
||||
SELECT dateName('hour', toDateTime('2021-04-14 11:22:33'), 'Europe/Moscow'),
|
||||
dateName('hour', toDateTime('2021-04-14 11:22:33'), 'UTC');
|
||||
WITH
|
||||
toDate('2021-04-14') AS date_value,
|
||||
toDateTime('2021-04-14 11:22:33') AS date_time_value,
|
||||
toDateTime64('2021-04-14 11:22:33', 3) AS date_time_64_value
|
||||
SELECT dateName('weekday', date_value), dateName('weekday', date_time_value), dateName('weekday', date_time_64_value);
|
||||
|
||||
SELECT dateName('weekday', toDate('2021-04-12'));
|
||||
SELECT dateName('weekday', toDate('2021-04-13'));
|
||||
SELECT dateName('weekday', toDate('2021-04-14'));
|
||||
SELECT dateName('weekday', toDate('2021-04-15'));
|
||||
SELECT dateName('weekday', toDate('2021-04-16'));
|
||||
SELECT dateName('weekday', toDate('2021-04-17'));
|
||||
SELECT dateName('weekday', toDate('2021-04-18'));
|
||||
WITH
|
||||
toDateTime('2021-04-14 11:22:33') AS date_time_value,
|
||||
toDateTime64('2021-04-14 11:22:33', 3) AS date_time_64_value
|
||||
SELECT dateName('hour', date_time_value), dateName('hour', date_time_64_value);
|
||||
|
||||
SELECT dateName('month', toDate('2021-01-14'));
|
||||
SELECT dateName('month', toDate('2021-02-14'));
|
||||
SELECT dateName('month', toDate('2021-03-14'));
|
||||
SELECT dateName('month', toDate('2021-04-14'));
|
||||
SELECT dateName('month', toDate('2021-05-14'));
|
||||
SELECT dateName('month', toDate('2021-06-14'));
|
||||
SELECT dateName('month', toDate('2021-07-14'));
|
||||
SELECT dateName('month', toDate('2021-08-14'));
|
||||
SELECT dateName('month', toDate('2021-09-14'));
|
||||
SELECT dateName('month', toDate('2021-10-14'));
|
||||
SELECT dateName('month', toDate('2021-11-14'));
|
||||
SELECT dateName('month', toDate('2021-12-14'));
|
||||
WITH
|
||||
toDateTime('2021-04-14 11:22:33') AS date_time_value,
|
||||
toDateTime64('2021-04-14 11:22:33', 3) AS date_time_64_value
|
||||
SELECT dateName('minute', date_time_value), dateName('minute', date_time_64_value);
|
||||
|
||||
WITH
|
||||
toDateTime('2021-04-14 11:22:33') AS date_time_value,
|
||||
toDateTime64('2021-04-14 11:22:33', 3) AS date_time_64_value
|
||||
SELECT dateName('second', date_time_value), dateName('second', date_time_64_value);
|
||||
|
||||
WITH
|
||||
toDateTime('2021-04-14 23:22:33', 'UTC') as date
|
||||
SELECT
|
||||
dateName('weekday', date, 'UTC'),
|
||||
dateName('hour', date, 'UTC'),
|
||||
dateName('minute', date, 'UTC'),
|
||||
dateName('second', date, 'UTC');
|
||||
|
||||
WITH
|
||||
toDateTime('2021-04-14 23:22:33', 'UTC') as date
|
||||
SELECT
|
||||
dateName('weekday', date, 'Europe/Moscow'),
|
||||
dateName('hour', date, 'Europe/Moscow'),
|
||||
dateName('minute', date, 'Europe/Moscow'),
|
||||
dateName('second', date, 'Europe/Moscow');
|
||||
|
Loading…
Reference in New Issue
Block a user