Add document and test sql

This commit is contained in:
Andy Yang 2019-05-07 19:27:20 +08:00
parent 3d583dafea
commit 92b0c9a668
7 changed files with 163 additions and 55 deletions

View File

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

View File

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

View File

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

View File

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

View 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);

View File

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

View File

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