mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Add document and test sql
This commit is contained in:
parent
3d583dafea
commit
92b0c9a668
@ -1,12 +1,12 @@
|
||||
#pragma once
|
||||
#include <Core/Types.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <common/DateLUTImpl.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <regex>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
#include <regex>
|
||||
#include <Common/Exception.h>
|
||||
#include <common/DateLUTImpl.h>
|
||||
|
||||
/// Custom date defaults to January 1 ( 01-01 )
|
||||
#define DEFAULT_CUSTOM_MONTH 1
|
||||
@ -14,7 +14,6 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
@ -88,8 +87,12 @@ struct ToCustomWeekImpl
|
||||
template <typename FromType, typename ToType, typename Transform>
|
||||
struct Transformer
|
||||
{
|
||||
static void vector(const PaddedPODArray<FromType> & vec_from, PaddedPODArray<ToType> & vec_to,
|
||||
UInt8 custom_month, UInt8 custom_day, const DateLUTImpl & time_zone)
|
||||
static void vector(
|
||||
const PaddedPODArray<FromType> & vec_from,
|
||||
PaddedPODArray<ToType> & vec_to,
|
||||
UInt8 custom_month,
|
||||
UInt8 custom_day,
|
||||
const DateLUTImpl & time_zone)
|
||||
{
|
||||
size_t size = vec_from.size();
|
||||
vec_to.resize(size);
|
||||
@ -123,9 +126,10 @@ struct CustomDateTransformImpl
|
||||
custom_day = std::stoi(custom_date.substr(3, 2));
|
||||
}
|
||||
else
|
||||
throw Exception(String("The second argument for function ") + Transform::name +
|
||||
" must be a constant string with custom date(MM-DD)",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
throw Exception(
|
||||
String("The second argument for function ") + Transform::name
|
||||
+ " must be a constant string with custom date(MM-DD)",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
|
||||
@ -139,8 +143,9 @@ struct CustomDateTransformImpl
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Transform::name,
|
||||
throw Exception(
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function "
|
||||
+ Transform::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
|
@ -1,14 +1,13 @@
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <Functions/CustomDateTransforms.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
#include <Functions/CustomDateTransforms.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
@ -24,10 +23,7 @@ public:
|
||||
static constexpr auto name = Transform::name;
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionCustomDateToSomething>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
@ -51,7 +47,8 @@ public:
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
if (!isString(arguments[1].type))
|
||||
throw Exception(
|
||||
"Function " + getName() + " supports 1 or 2 or 3 arguments. The 1st argument "
|
||||
"Function " + getName()
|
||||
+ " supports 1 or 2 or 3 arguments. The 1st argument "
|
||||
"must be of type Date or DateTime. The 2nd argument (optional) must be "
|
||||
"a constant string with custom date(MM-DD). The 3nd argument (optional) must be "
|
||||
"a constant string with timezone name",
|
||||
@ -61,27 +58,29 @@ public:
|
||||
{
|
||||
if (!isDateOrDateTime(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
+ ". Should be a date or a date with time",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
if (!isString(arguments[1].type))
|
||||
throw Exception(
|
||||
"Function " + getName() + " supports 1 or 2 or 3 arguments. The 1st argument "
|
||||
"must be of type Date or DateTime. The 2nd argument (optional) must be "
|
||||
"a constant string with custom date(MM-DD). The 3nd argument (optional) must be "
|
||||
"a constant string with timezone name",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
"Function " + getName()
|
||||
+ " supports 1 or 2 or 3 arguments. The 1st argument "
|
||||
"must be of type Date or DateTime. The 2nd argument (optional) must be "
|
||||
"a constant string with custom date(MM-DD). The 3nd argument (optional) must be "
|
||||
"a constant string with timezone name",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
if (!isString(arguments[2].type))
|
||||
throw Exception(
|
||||
"Function " + getName() + " supports 1 or 2 or 3 arguments. The 1st argument "
|
||||
"must be of type Date or DateTime. The 2nd argument (optional) must be "
|
||||
"a constant string with custom date(MM-DD). The 3nd argument (optional) must be "
|
||||
"a constant string with timezone name",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
"Function " + getName()
|
||||
+ " supports 1 or 2 or 3 arguments. The 1st argument "
|
||||
"must be of type Date or DateTime. The 2nd argument (optional) must be "
|
||||
"a constant string with custom date(MM-DD). The 3nd argument (optional) must be "
|
||||
"a constant string with timezone name",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
if (isDate(arguments[0].type) && std::is_same_v<ToDataType, DataTypeDate>)
|
||||
throw Exception(
|
||||
"The timezone argument of function " + getName() + " is allowed only when the 1st argument has the type DateTime",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
"The timezone argument of function " + getName() + " is allowed only when the 1st argument has the type DateTime",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
@ -105,23 +104,23 @@ public:
|
||||
WhichDataType which(from_type);
|
||||
|
||||
if (which.isDate())
|
||||
CustomDateTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result, input_rows_count);
|
||||
CustomDateTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform>::execute(
|
||||
block, arguments, result, input_rows_count);
|
||||
else if (which.isDateTime())
|
||||
CustomDateTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result, input_rows_count);
|
||||
CustomDateTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform>::execute(
|
||||
block, arguments, result, input_rows_count);
|
||||
else
|
||||
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
||||
throw Exception(
|
||||
"Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
|
||||
bool hasInformationAboutMonotonicity() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
bool hasInformationAboutMonotonicity() const override { return true; }
|
||||
|
||||
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override
|
||||
{
|
||||
IFunction::Monotonicity is_monotonic { true };
|
||||
IFunction::Monotonicity is_monotonic{true};
|
||||
IFunction::Monotonicity is_not_monotonic;
|
||||
|
||||
if (std::is_same_v<typename Transform::FactorTransform, ZeroTransform>)
|
||||
@ -141,17 +140,18 @@ public:
|
||||
if (checkAndGetDataType<DataTypeDate>(&type))
|
||||
{
|
||||
return Transform::FactorTransform::execute(UInt16(left.get<UInt64>()), DEFAULT_CUSTOM_MONTH, DEFAULT_CUSTOM_DAY, date_lut)
|
||||
== Transform::FactorTransform::execute(UInt16(right.get<UInt64>()), DEFAULT_CUSTOM_MONTH, DEFAULT_CUSTOM_DAY, date_lut)
|
||||
? is_monotonic : is_not_monotonic;
|
||||
== Transform::FactorTransform::execute(UInt16(right.get<UInt64>()), DEFAULT_CUSTOM_MONTH, DEFAULT_CUSTOM_DAY, date_lut)
|
||||
? is_monotonic
|
||||
: is_not_monotonic;
|
||||
}
|
||||
else
|
||||
{
|
||||
return Transform::FactorTransform::execute(UInt32(left.get<UInt64>()), DEFAULT_CUSTOM_MONTH, DEFAULT_CUSTOM_DAY, date_lut)
|
||||
== Transform::FactorTransform::execute(UInt32(right.get<UInt64>()), DEFAULT_CUSTOM_MONTH, DEFAULT_CUSTOM_DAY, date_lut)
|
||||
? is_monotonic : is_not_monotonic;
|
||||
== Transform::FactorTransform::execute(UInt32(right.get<UInt64>()), DEFAULT_CUSTOM_MONTH, DEFAULT_CUSTOM_DAY, date_lut)
|
||||
? is_monotonic
|
||||
: is_not_monotonic;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -1,13 +1,12 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/CustomDateTransforms.h>
|
||||
#include <Functions/FunctionCustomDateToSomething.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using FunctionToCustomWeek = FunctionCustomDateToSomething<DataTypeUInt8, ToCustomWeekImpl>;
|
||||
|
||||
void registerFunctionToCustomWeek(FunctionFactory & factory)
|
||||
@ -16,5 +15,3 @@ void registerFunctionToCustomWeek(FunctionFactory & factory)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
@ -0,0 +1,45 @@
|
||||
2016-12-22 52 2016 2015-12-28
|
||||
2016-12-23 52 2016 2015-12-28
|
||||
2016-12-24 52 2016 2015-12-28
|
||||
2016-12-25 52 2016 2015-12-28
|
||||
2016-12-26 1 2017 2016-12-26
|
||||
2016-12-27 1 2017 2016-12-26
|
||||
2016-12-28 1 2017 2016-12-26
|
||||
2016-12-29 1 2017 2016-12-26
|
||||
2016-12-30 1 2017 2016-12-26
|
||||
2016-12-31 1 2017 2016-12-26
|
||||
2017-01-01 1 2017 2016-12-26
|
||||
2017-01-02 2 2017 2016-12-26
|
||||
2017-01-03 2 2017 2016-12-26
|
||||
2017-01-04 2 2017 2016-12-26
|
||||
2017-01-05 2 2017 2016-12-26
|
||||
2017-01-20 00:00:00 52 2016 2016-01-25
|
||||
2017-01-21 00:00:00 52 2016 2016-01-25
|
||||
2017-01-22 00:00:00 52 2016 2016-01-25
|
||||
2017-01-23 00:00:00 1 2017 2017-01-23
|
||||
2017-01-24 00:00:00 1 2017 2017-01-23
|
||||
2017-01-25 00:00:00 1 2017 2017-01-23
|
||||
2017-01-26 00:00:00 1 2017 2017-01-23
|
||||
2017-01-27 00:00:00 1 2017 2017-01-23
|
||||
2017-01-28 00:00:00 1 2017 2017-01-23
|
||||
2017-01-29 00:00:00 1 2017 2017-01-23
|
||||
2017-01-30 00:00:00 2 2017 2017-01-23
|
||||
2017-01-31 00:00:00 2 2017 2017-01-23
|
||||
2017-02-01 00:00:00 2 2017 2017-01-23
|
||||
2017-02-02 00:00:00 2 2017 2017-01-23
|
||||
2017-02-03 00:00:00 2 2017 2017-01-23
|
||||
2017-01-20 00:00:00 52 2016 2016-01-25
|
||||
2017-01-21 00:00:00 52 2016 2016-01-25
|
||||
2017-01-22 00:00:00 52 2016 2016-01-25
|
||||
2017-01-23 00:00:00 1 2017 2017-01-23
|
||||
2017-01-24 00:00:00 1 2017 2017-01-23
|
||||
2017-01-25 00:00:00 1 2017 2017-01-23
|
||||
2017-01-26 00:00:00 1 2017 2017-01-23
|
||||
2017-01-27 00:00:00 1 2017 2017-01-23
|
||||
2017-01-28 00:00:00 1 2017 2017-01-23
|
||||
2017-01-29 00:00:00 1 2017 2017-01-23
|
||||
2017-01-30 00:00:00 2 2017 2017-01-23
|
||||
2017-01-31 00:00:00 2 2017 2017-01-23
|
||||
2017-02-01 00:00:00 2 2017 2017-01-23
|
||||
2017-02-02 00:00:00 2 2017 2017-01-23
|
||||
2017-02-03 00:00:00 2 2017 2017-01-23
|
6
dbms/tests/queries/0_stateless/00941_to_custom_week.sql
Normal file
6
dbms/tests/queries/0_stateless/00941_to_custom_week.sql
Normal file
@ -0,0 +1,6 @@
|
||||
-- By default, the week contains January 1 is the first week:
|
||||
SELECT toDate('2016-12-22') + number AS d, toCustomWeek(d) as week, toCustomYear(d) AS year, toStartOfCustomYear(d) AS startday FROM numbers(15);
|
||||
-- The week contains January 28 is the first week:
|
||||
SELECT toDateTime(toDate('2017-01-20') + number, 'Europe/Moscow' ) AS d, toCustomWeek(d,'01-28') as week, toCustomYear(d,'01-28') AS year, toStartOfCustomYear(d,'01-28') AS startday FROM numbers(15);
|
||||
-- and with timezone
|
||||
SELECT toDateTime(toDate('2017-01-20') + number, 'Europe/Moscow' ) AS d, toCustomWeek(d,'01-28','Europe/Moscow') as week, toCustomYear(d,'01-28','Europe/Moscow') AS year, toStartOfCustomYear(d,'01-28','Europe/Moscow') AS startday FROM numbers(15);
|
@ -76,6 +76,14 @@ Returns the date.
|
||||
Rounds down a date or date with time to the first day of ISO year.
|
||||
Returns the date.
|
||||
|
||||
## toStartOfCustomYear
|
||||
|
||||
Rounds down a date or date with time to the first day of Custom year.
|
||||
Returns the date.
|
||||
```
|
||||
toStartOfCustomYear(DateTime1, [, CustomDate][, Timezone])
|
||||
```
|
||||
|
||||
## toStartOfQuarter
|
||||
|
||||
Rounds down a date or date with time to the first day of the quarter.
|
||||
@ -169,6 +177,53 @@ Converts a date or date with time to a UInt16 number containing the ISO Year num
|
||||
|
||||
Converts a date or date with time to a UInt8 number containing the ISO Week number.
|
||||
|
||||
## toCustomYear
|
||||
|
||||
Converts a date or date with time to a UInt16 number containing the Custom Year number.
|
||||
```
|
||||
toCustomYear(DateTime, [, CustomDate][, Timezone])
|
||||
```
|
||||
|
||||
## toCustomWeek
|
||||
|
||||
Converts a date or date with time to a UInt8 number containing the Custom Week number.
|
||||
The week number 1 is the first week in year that contains some day, which default to January 1, it can be modify by parameter of function.
|
||||
Week begins at monday.
|
||||
```
|
||||
toCustomWeek(DateTime1, [, CustomDate][, Timezone])
|
||||
```
|
||||
**Parameters**
|
||||
|
||||
- `DateTime1` – Date or DateTime.
|
||||
- `CustomDate` – Optional parameter, default is January 1, format: MM-DD.
|
||||
- `Timezone` – Optional parameter, it behaves like any other conversion function.
|
||||
|
||||
**Example**
|
||||
By default, the week contains January 1 is the first week:
|
||||
|
||||
``` sql
|
||||
SELECT toDate('2016-12-27') AS date, toCustomYear(date) AS year, toCustomWeek(date) AS week, toStartOfCustomYear(date) AS startday;
|
||||
```
|
||||
|
||||
```
|
||||
┌───────date─┬─year─┬─week─┬───startday─┐
|
||||
│ 2016-12-27 │ 2017 │ 1 │ 2016-12-26 │
|
||||
└───────────┴─────┴──────┴─────────┘
|
||||
```
|
||||
|
||||
The week contains January 28 is the first week:
|
||||
|
||||
``` sql
|
||||
SELECT toDate('2016-12-27') AS date, toCustomYear(date, '01-28') AS year, toCustomWeek(date,'01-28') AS week, toStartOfCustomYear(date,'01-28') AS startday;
|
||||
```
|
||||
|
||||
```
|
||||
┌───────date─┬─year─┬─week─┬───startday─┐
|
||||
│ 2016-12-27 │ 2016 │ 49 │ 2016-01-25 │
|
||||
└───────────┴─────┴──────┴─────────┘
|
||||
```
|
||||
|
||||
|
||||
## now
|
||||
|
||||
Accepts zero arguments and returns the current time at one of the moments of request execution.
|
||||
|
@ -394,7 +394,7 @@ public:
|
||||
}
|
||||
|
||||
/// Custom year begins with a monday of the week that is contained January 1,
|
||||
/// which day can be modified through config of first_week_of_day.
|
||||
/// which day can be modified through custom_month and custom_day of parameters.
|
||||
/// Example: Custom year 2019 begins at 2018-12-31. And Custom year 2017 begins at 2016-12-26.
|
||||
inline DayNum toFirstDayNumOfCustomYear(DayNum d, UInt8 custom_month, UInt8 custom_day) const
|
||||
{
|
||||
@ -414,7 +414,7 @@ public:
|
||||
|
||||
/// Custom week number. Week begins at monday.
|
||||
/// The week number 1 is the first week in year that contains January 1,
|
||||
/// which day can be modified through config of first_week_of_day.
|
||||
/// which day can be modified through custom_month and custom_day of parameters.
|
||||
inline unsigned toCustomWeek(DayNum d, UInt8 custom_month, UInt8 custom_day) const
|
||||
{
|
||||
return 1 + (toFirstDayNumOfWeek(d) - toFirstDayNumOfCustomYear(d, custom_month, custom_day)) / 7;
|
||||
|
Loading…
Reference in New Issue
Block a user