Merge pull request #25372 from ClickHouse/add-date-name-function-merge

Merging #23085
This commit is contained in:
Maksim Kita 2021-06-17 10:16:32 +03:00 committed by GitHub
commit 37770f20fc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 492 additions and 16 deletions

View File

@ -27,17 +27,17 @@ SELECT
Returns the timezone of the server.
**Syntax**
**Syntax**
``` sql
timeZone()
```
Alias: `timezone`.
Alias: `timezone`.
**Returned value**
- Timezone.
- Timezone.
Type: [String](../../sql-reference/data-types/string.md).
@ -45,7 +45,7 @@ Type: [String](../../sql-reference/data-types/string.md).
Converts time or date and time to the specified time zone. The time zone is an attribute of the `Date` and `DateTime` data types. The internal value (number of seconds) of the table field or of the resultset's column does not change, the column's type changes and its string representation changes accordingly.
**Syntax**
**Syntax**
``` sql
toTimezone(value, timezone)
@ -53,14 +53,14 @@ toTimezone(value, timezone)
Alias: `toTimezone`.
**Arguments**
**Arguments**
- `value` — Time or date and time. [DateTime64](../../sql-reference/data-types/datetime64.md).
- `timezone` — Timezone for the returned value. [String](../../sql-reference/data-types/string.md).
**Returned value**
- Date and time.
- Date and time.
Type: [DateTime](../../sql-reference/data-types/datetime.md).
@ -102,21 +102,21 @@ int32samoa: 1546300800
Returns the timezone name of [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md) data types.
**Syntax**
**Syntax**
``` sql
timeZoneOf(value)
```
Alias: `timezoneOf`.
Alias: `timezoneOf`.
**Arguments**
- `value` — Date and time. [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md).
- `value` — Date and time. [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md).
**Returned value**
- Timezone name.
- Timezone name.
Type: [String](../../sql-reference/data-types/string.md).
@ -149,11 +149,11 @@ Alias: `timezoneOffset`.
**Arguments**
- `value` — Date and time. [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md).
- `value` — Date and time. [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md).
**Returned value**
- Offset from UTC in seconds.
- Offset from UTC in seconds.
Type: [Int32](../../sql-reference/data-types/int-uint.md).
@ -599,7 +599,7 @@ Aliases: `dateAdd`, `DATE_ADD`.
- `quarter`
- `year`
- `value` — Value of interval to add. [Int](../../sql-reference/data-types/int-uint.md).
- `value` — Value of interval to add. [Int](../../sql-reference/data-types/int-uint.md).
- `date` — The date or date with time to which `value` is added. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md).
**Returned value**
@ -704,7 +704,7 @@ Aliases: `dateSub`, `DATE_SUB`.
- `quarter`
- `year`
- `value` — Value of interval to subtract. [Int](../../sql-reference/data-types/int-uint.md).
- `value` — Value of interval to subtract. [Int](../../sql-reference/data-types/int-uint.md).
- `date` — The date or date with time from which `value` is subtracted. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md).
**Returned value**
@ -805,7 +805,7 @@ Aliases: `timeStampSub`, `TIMESTAMP_SUB`.
- `quarter`
- `year`
- `value` — Value of interval to subtract. [Int](../../sql-reference/data-types/int-uint.md).
- `value` — Value of interval to subtract. [Int](../../sql-reference/data-types/int-uint.md).
- `date` — Date or date with time. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md).
**Returned value**
@ -963,7 +963,7 @@ formatDateTime(Time, Format\[, Timezone\])
**Returned value(s)**
Returnes time and date values according to the determined format.
Returns time and date values according to the determined format.
**Replacement fields**
Using replacement fields, you can define a pattern for the resulting string. “Example” column shows formatting result for `2018-01-02 22:33:44`.
@ -1012,6 +1012,45 @@ Result:
└────────────────────────────────────────────┘
```
## dateName {#dataname}
Returns part of date with specified date part.
**Syntax**
``` sql
dateName(date_part, date)
```
**Arguments**
- `date_part` - Date part. Possible values .
- `date` — Date [Date](../../sql-reference/data-types/date.md) or DateTime [DateTime](../../sql-reference/data-types/datetime.md), [DateTime64](../../sql-reference/data-types/datetime64.md).
**Returned value**
- Specified date part of date.
Type: [String](../../sql-reference/data-types/string.md#string)
**Example**
Query:
```sql
WITH toDateTime('2021-04-14 11:22:33') AS date_value
SELECT dateName('year', date_value), dateName('month', date_value), dateName('day', date_value);
```
Result:
```text
┌─dateName('year', date_value)─┬─dateName('month', date_value)─┬─dateName('day', date_value)─┐
│ 2021 │ April │ 14 │
└──────────────────────────────┴───────────────────────────────┴─────────────────────────────
```
## FROM\_UNIXTIME {#fromunixfime}
Function converts Unix timestamp to a calendar date and a time of a day. When there is only a single argument of [Integer](../../sql-reference/data-types/int-uint.md) type, it acts in the same way as [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime) and return [DateTime](../../sql-reference/data-types/datetime.md) type.

349
src/Functions/dateName.cpp Normal file
View File

@ -0,0 +1,349 @@
#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/extractTimeZoneFromFunctionArguments.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
extern const int BAD_ARGUMENTS;
}
namespace
{
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
{
public:
static constexpr auto name = "dateName";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionDateNameImpl>(); }
String getName() const override { return name; }
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 2}; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() != 2 && arguments.size() != 3)
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"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());
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>();
}
ColumnPtr executeImpl(
const ColumnsWithTypeAndName & arguments,
const DataTypePtr & result_type,
[[maybe_unused]] size_t input_rows_count) const override
{
ColumnPtr res;
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.",
arguments[1].column->getName(),
getName());
return res;
}
template <typename DataType>
ColumnPtr executeType(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const
{
auto * times = checkAndGetColumn<typename DataType::ColumnType>(arguments[1].column.get());
if (!times)
return nullptr;
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());
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>)
time_zone_tmp = &extractTimeZoneFromFunctionArguments(arguments, 2, 1);
else
time_zone_tmp = &DateLUT::instance();
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 = times_data.getScale();
}
auto result_column = ColumnString::create();
auto & result_column_data = result_column->getChars();
auto & result_column_offsets = result_column->getOffsets();
/* longest possible word 'Wednesday' with zero terminator */
static constexpr size_t longest_word_length = 9 + 1;
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>)
{
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;
}
});
result_column_data.resize(buffer.position() - begin);
return result_column;
}
private:
template <typename Time>
struct YearWriter
{
static void write(WriteBuffer & buffer, Time source, const DateLUTImpl & timezone)
{
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);
}
}

View File

@ -64,6 +64,7 @@ void registerFunctionSubtractMonths(FunctionFactory &);
void registerFunctionSubtractQuarters(FunctionFactory &);
void registerFunctionSubtractYears(FunctionFactory &);
void registerFunctionDateDiff(FunctionFactory &);
void registerFunctionDateName(FunctionFactory &);
void registerFunctionToTimeZone(FunctionFactory &);
void registerFunctionFormatDateTime(FunctionFactory &);
void registerFunctionFromModifiedJulianDay(FunctionFactory &);
@ -134,6 +135,7 @@ void registerFunctionsDateTime(FunctionFactory & factory)
registerFunctionSubtractQuarters(factory);
registerFunctionSubtractYears(factory);
registerFunctionDateDiff(factory);
registerFunctionDateName(factory);
registerFunctionToTimeZone(factory);
registerFunctionFormatDateTime(factory);
registerFunctionFromModifiedJulianDay(factory);

View File

@ -222,6 +222,7 @@ SRCS(
currentDatabase.cpp
currentUser.cpp
dateDiff.cpp
dateName.cpp
date_trunc.cpp
decodeXMLComponent.cpp
decrypt.cpp

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

@ -0,0 +1,12 @@
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

@ -0,0 +1,72 @@
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);
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);
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);
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);
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);
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');