Updated function dateName before merge

This commit is contained in:
Maksim Kita 2021-06-16 21:19:05 +03:00
parent 5191fecb09
commit ac2f9dd15a
4 changed files with 294 additions and 297 deletions

View File

@ -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,7 +105,8 @@ public:
{
ColumnPtr res;
if (!((res = executeType<DataTypeDate>(arguments, result_type)) || (res = executeType<DataTypeDateTime>(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,
@ -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,85 +140,84 @@ 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)
{
for (size_t i = 0; i < times_data.size(); ++i)
{
if constexpr (std::is_same_v<DataType, DataTypeDateTime64>)
{
// 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);
const auto components = DecimalUtils::split(times_data[i], scale);
writer.write(buffer, static_cast<Int64>(components.whole), time_zone);
}
else
{
datepart_writer.writeDatePart(pos, datepart, vec[i], time_zone);
writer.write(buffer, times_data[i], time_zone);
}
dst_offsets[i] = pos - begin;
++pos;
/// Null terminator
++buffer.position();
result_column_offsets[i] = buffer.position() - begin;
}
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);
writeText(ToYearImpl::execute(source, timezone), buffer);
}
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)
template <typename Time>
struct QuarterWriter
{
writeNumber(target, ToYearImpl::execute(source, timezone));
}
static inline void writeQuarter(char *& target, Time source, const DateLUTImpl & timezone)
static inline void write(WriteBuffer & buffer, Time source, const DateLUTImpl & timezone)
{
writeNumber(target, ToQuarterImpl::execute(source, timezone));
writeText(ToQuarterImpl::execute(source, timezone), buffer);
}
};
static inline void writeMonth(char *& target, Time source, const DateLUTImpl & timezone)
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 monthnames[]
= {"January",
static constexpr std::string_view month_names[] =
{
"January",
"February",
"March",
"April",
@ -221,114 +228,122 @@ private:
"September",
"October",
"November",
"December"};
writeString(target, monthnames[month - 1]);
}
"December"
};
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(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);
}
}

View File

@ -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); }

View File

@ -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

View File

@ -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');