Added support for toQuarter, toRelativeQuarterNum functions. Enabled toRelativeHour/Minute/Second for Date arguments. Made function 'now' case insensitive. Added function 'dateDiff' that is needed for ODBC driver [#CLICKHOUSE-2].

This commit is contained in:
Alexey Milovidov 2017-12-22 04:54:29 +03:00
parent 74f191dfd3
commit 0972d7586a
8 changed files with 448 additions and 34 deletions

View File

@ -213,7 +213,7 @@ struct ConvertImpl<FromDataType, typename std::enable_if<!std::is_same<FromDataT
/// For argument of DateTime type, second argument with time zone could be specified.
if constexpr (std::is_same<FromDataType, DataTypeDateTime>::value)
time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1);
time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1, 0);
if (const auto col_from = checkAndGetColumn<ColumnVector<FromFieldType>>(col_with_type_and_name.column.get()))
{
@ -332,7 +332,7 @@ struct ConvertImpl<typename std::enable_if<!std::is_same<ToDataType, DataTypeStr
/// For conversion to DateTime type, second argument with time zone could be specified.
if (std::is_same<ToDataType, DataTypeDateTime>::value)
time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1);
time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1, 0);
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(block.getByPosition(arguments[0]).column.get()))
{
@ -506,7 +506,7 @@ struct ConvertImpl<DataTypeFixedString, ToDataType, Name>
/// For conversion to DateTime type, second argument with time zone could be specified.
if (std::is_same<ToDataType, DataTypeDateTime>::value)
time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1);
time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1, 0);
auto col_to = ColumnVector<ToFieldType>::create();
@ -666,7 +666,7 @@ public:
}
if (std::is_same<ToDataType, DataTypeDateTime>::value)
out_return_type = std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 1));
out_return_type = std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0));
else
out_return_type = std::make_shared<ToDataType>();
}

View File

@ -18,7 +18,7 @@ static std::string extractTimeZoneNameFromColumn(const IColumn & column)
}
std::string extractTimeZoneNameFromFunctionArguments(const ColumnsWithTypeAndName & arguments, size_t time_zone_arg_num)
std::string extractTimeZoneNameFromFunctionArguments(const ColumnsWithTypeAndName & arguments, size_t time_zone_arg_num, size_t datetime_arg_num)
{
/// Explicit time zone may be passed in last argument.
if (arguments.size() == time_zone_arg_num + 1)
@ -31,14 +31,14 @@ std::string extractTimeZoneNameFromFunctionArguments(const ColumnsWithTypeAndNam
return {};
/// If time zone is attached to an argument of type DateTime.
if (const DataTypeDateTime * type = checkAndGetDataType<DataTypeDateTime>(arguments[0].type.get()))
if (const DataTypeDateTime * type = checkAndGetDataType<DataTypeDateTime>(arguments[datetime_arg_num].type.get()))
return type->getTimeZone().getTimeZone();
return {};
}
}
const DateLUTImpl & extractTimeZoneFromFunctionArguments(Block & block, const ColumnNumbers & arguments, size_t time_zone_arg_num)
const DateLUTImpl & extractTimeZoneFromFunctionArguments(Block & block, const ColumnNumbers & arguments, size_t time_zone_arg_num, size_t datetime_arg_num)
{
if (arguments.size() == time_zone_arg_num + 1)
return DateLUT::instance(extractTimeZoneNameFromColumn(*block.getByPosition(arguments[time_zone_arg_num]).column));
@ -48,7 +48,7 @@ const DateLUTImpl & extractTimeZoneFromFunctionArguments(Block & block, const Co
return DateLUT::instance();
/// If time zone is attached to an argument of type DateTime.
if (const DataTypeDateTime * type = checkAndGetDataType<DataTypeDateTime>(block.getByPosition(arguments[0]).type.get()))
if (const DataTypeDateTime * type = checkAndGetDataType<DataTypeDateTime>(block.getByPosition(arguments[datetime_arg_num]).type.get()))
return type->getTimeZone();
return DateLUT::instance();
@ -58,6 +58,7 @@ const DateLUTImpl & extractTimeZoneFromFunctionArguments(Block & block, const Co
void registerFunctionsDateTime(FunctionFactory & factory)
{
factory.registerFunction<FunctionToYear>();
factory.registerFunction<FunctionToQuarter>();
factory.registerFunction<FunctionToMonth>();
factory.registerFunction<FunctionToDayOfMonth>();
factory.registerFunction<FunctionToDayOfWeek>();
@ -73,6 +74,7 @@ void registerFunctionsDateTime(FunctionFactory & factory)
factory.registerFunction<FunctionToStartOfFiveMinute>();
factory.registerFunction<FunctionToStartOfHour>();
factory.registerFunction<FunctionToRelativeYearNum>();
factory.registerFunction<FunctionToRelativeQuarterNum>();
factory.registerFunction<FunctionToRelativeMonthNum>();
factory.registerFunction<FunctionToRelativeWeekNum>();
factory.registerFunction<FunctionToRelativeDayNum>();
@ -80,7 +82,7 @@ void registerFunctionsDateTime(FunctionFactory & factory)
factory.registerFunction<FunctionToRelativeMinuteNum>();
factory.registerFunction<FunctionToRelativeSecondNum>();
factory.registerFunction<FunctionToTime>();
factory.registerFunction<FunctionNow>();
factory.registerFunction(FunctionNow::name, FunctionNow::create, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionToday>();
factory.registerFunction<FunctionYesterday>();
factory.registerFunction<FunctionTimeSlot>();
@ -105,6 +107,8 @@ void registerFunctionsDateTime(FunctionFactory & factory)
factory.registerFunction<FunctionSubtractMonths>();
factory.registerFunction<FunctionSubtractYears>();
factory.registerFunction(FunctionDateDiff::name, FunctionDateDiff::create, FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionToTimeZone>();
}

View File

@ -21,6 +21,8 @@
#include <common/DateLUT.h>
#include <Poco/String.h>
#include <type_traits>
@ -62,8 +64,8 @@ namespace ErrorCodes
/// Determine working timezone either from optional argument with time zone name or from time zone in DateTime type of argument.
std::string extractTimeZoneNameFromFunctionArguments(const ColumnsWithTypeAndName & arguments, size_t time_zone_arg_num);
const DateLUTImpl & extractTimeZoneFromFunctionArguments(Block & block, const ColumnNumbers & arguments, size_t time_zone_arg_num);
std::string extractTimeZoneNameFromFunctionArguments(const ColumnsWithTypeAndName & arguments, size_t time_zone_arg_num, size_t datetime_arg_num);
const DateLUTImpl & extractTimeZoneFromFunctionArguments(Block & block, const ColumnNumbers & arguments, size_t time_zone_arg_num, size_t datetime_arg_num);
@ -269,6 +271,22 @@ struct ToYearImpl
using FactorTransform = ZeroTransform;
};
struct ToQuarterImpl
{
static constexpr auto name = "toQuarter";
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
{
return time_zone.toQuarter(t);
}
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toQuarter(DayNum_t(d));
}
using FactorTransform = ToStartOfYearImpl;
};
struct ToMonthImpl
{
static constexpr auto name = "toMonth";
@ -382,6 +400,22 @@ struct ToRelativeYearNumImpl
using FactorTransform = ZeroTransform;
};
struct ToRelativeQuarterNumImpl
{
static constexpr auto name = "toRelativeQuarterNum";
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
{
return time_zone.toRelativeQuarterNum(t);
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toRelativeQuarterNum(DayNum_t(d));
}
using FactorTransform = ZeroTransform;
};
struct ToRelativeMonthNumImpl
{
static constexpr auto name = "toRelativeMonthNum";
@ -439,9 +473,9 @@ struct ToRelativeHourNumImpl
{
return time_zone.toRelativeHourNum(t);
}
static inline UInt32 execute(UInt16, const DateLUTImpl &)
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
{
throw Exception("Illegal type Date of argument for function toRelativeHourNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return time_zone.toRelativeHourNum(DayNum_t(d));
}
using FactorTransform = ZeroTransform;
@ -455,9 +489,9 @@ struct ToRelativeMinuteNumImpl
{
return time_zone.toRelativeMinuteNum(t);
}
static inline UInt32 execute(UInt16, const DateLUTImpl &)
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
{
throw Exception("Illegal type Date of argument for function toRelativeMinuteNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return time_zone.toRelativeMinuteNum(DayNum_t(d));
}
using FactorTransform = ZeroTransform;
@ -471,9 +505,9 @@ struct ToRelativeSecondNumImpl
{
return t;
}
static inline UInt32 execute(UInt16, const DateLUTImpl &)
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
{
throw Exception("Illegal type Date of argument for function toRelativeSecondNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return time_zone.fromDayNum(DayNum_t(d));
}
using FactorTransform = ZeroTransform;
@ -549,7 +583,7 @@ struct DateTimeTransformImpl
{
using Op = Transformer<FromType, ToType, Transform>;
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 1);
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 1, 0);
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
if (const auto * sources = checkAndGetColumn<ColumnVector<FromType>>(source_col.get()))
@ -613,7 +647,7 @@ public:
/// For DateTime, if time zone is specified, attach it to type.
if (std::is_same<ToDataType, DataTypeDateTime>::value)
out_return_type = std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 1));
out_return_type = std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0));
else
out_return_type = std::make_shared<ToDataType>();
}
@ -844,7 +878,7 @@ struct DateTimeAddIntervalImpl
using ToType = decltype(Transform::execute(FromType(), 0, std::declval<DateLUTImpl>()));
using Op = Adder<FromType, ToType, Transform>;
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 2);
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 2, 0);
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
@ -931,14 +965,14 @@ public:
if (std::is_same<decltype(Transform::execute(DataTypeDate::FieldType(), 0, std::declval<DateLUTImpl>())), UInt16>::value)
out_return_type = std::make_shared<DataTypeDate>();
else
out_return_type = std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2));
out_return_type = std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
}
else
{
if (std::is_same<decltype(Transform::execute(DataTypeDateTime::FieldType(), 0, std::declval<DateLUTImpl>())), UInt16>::value)
out_return_type = std::make_shared<DataTypeDate>();
else
out_return_type = std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2));
out_return_type = std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
}
}
@ -960,6 +994,191 @@ public:
};
/** dateDiff('unit', t1, t2, [timezone])
* t1 and t2 can be Date or DateTime
*
* If timezone is specified, it applied to both arguments.
* If not, timezones from datatypes t1 and t2 are used.
* If that timezones are not the same, the result is unspecified.
*
* Timezone matters because days can have different length.
*/
class FunctionDateDiff : public IFunction
{
public:
static constexpr auto name = "dateDiff";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionDateDiff>(); };
String getName() const override
{
return name;
}
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 3 && arguments.size() != 4)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 3 or 4",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!arguments[0]->isString())
throw Exception("First argument for function " + getName() + " (unit) must be String",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!arguments[1]->isDateOrDateTime())
throw Exception("Second argument for function " + getName() + " must be Date or DateTime",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!arguments[2]->isDateOrDateTime())
throw Exception("Third argument for function " + getName() + " must be Date or DateTime",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (arguments.size() == 4 && !arguments[3]->isString())
throw Exception("Fourth argument for function " + getName() + " (timezone) must be String",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeInt64>();
}
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 3}; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
auto * unit_column = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!unit_column)
throw Exception("First argument for function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN);
String unit = Poco::toLower(unit_column->getValue<String>());
const IColumn & x = *block.getByPosition(arguments[1]).column;
const IColumn & y = *block.getByPosition(arguments[2]).column;
size_t rows = block.rows();
auto res = ColumnInt64::create(rows);
const DateLUTImpl & timezone_x = extractTimeZoneFromFunctionArguments(block, arguments, 3, 1);
const DateLUTImpl & timezone_y = extractTimeZoneFromFunctionArguments(block, arguments, 3, 2);
if (unit == "year" || unit == "yy" || unit == "yyyy")
dispatchForColumns<ToRelativeYearNumImpl>(x, y, timezone_x, timezone_y, res->getData());
else if (unit == "quarter" || unit == "qq" || unit == "q")
dispatchForColumns<ToRelativeQuarterNumImpl>(x, y, timezone_x, timezone_y, res->getData());
else if (unit == "month" || unit == "mm" || unit == "m")
dispatchForColumns<ToRelativeMonthNumImpl>(x, y, timezone_x, timezone_y, res->getData());
else if (unit == "week" || unit == "wk" || unit == "ww")
dispatchForColumns<ToRelativeWeekNumImpl>(x, y, timezone_x, timezone_y, res->getData());
else if (unit == "day" || unit == "dd" || unit == "d")
dispatchForColumns<ToRelativeDayNumImpl>(x, y, timezone_x, timezone_y, res->getData());
else if (unit == "hour" || unit == "hh")
dispatchForColumns<ToRelativeHourNumImpl>(x, y, timezone_x, timezone_y, res->getData());
else if (unit == "minute" || unit == "mi" || unit == "n")
dispatchForColumns<ToRelativeMinuteNumImpl>(x, y, timezone_x, timezone_y, res->getData());
else if (unit == "second" || unit == "ss" || unit == "s")
dispatchForColumns<ToRelativeSecondNumImpl>(x, y, timezone_x, timezone_y, res->getData());
else
throw Exception("Function " + getName() + " does not support '" + unit + "' unit", ErrorCodes::BAD_ARGUMENTS);
block.getByPosition(result).column = std::move(res);
}
private:
template <typename Transform>
void dispatchForColumns(
const IColumn & x, const IColumn & y,
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result)
{
if (auto * x_vec = checkAndGetColumn<ColumnUInt16>(&x))
dispatchForSecondColumn<Transform>(*x_vec, y, timezone_x, timezone_y, result);
else if (auto * x_vec = checkAndGetColumn<ColumnUInt32>(&x))
dispatchForSecondColumn<Transform>(*x_vec, y, timezone_x, timezone_y, result);
else if (auto * x_const = checkAndGetColumnConst<ColumnUInt16>(&x))
dispatchConstForSecondColumn<Transform>(x_const->getValue<UInt16>(), y, timezone_x, timezone_y, result);
else if (auto * x_const = checkAndGetColumnConst<ColumnUInt32>(&x))
dispatchConstForSecondColumn<Transform>(x_const->getValue<UInt32>(), y, timezone_x, timezone_y, result);
else
throw Exception("Illegal column for first argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN);
}
template <typename Transform, typename T1>
void dispatchForSecondColumn(
const ColumnVector<T1> & x, const IColumn & y,
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result)
{
if (auto * y_vec = checkAndGetColumn<ColumnUInt16>(&y))
vector_vector<Transform>(x, *y_vec, timezone_x, timezone_y, result);
else if (auto * y_vec = checkAndGetColumn<ColumnUInt32>(&y))
vector_vector<Transform>(x, *y_vec, timezone_x, timezone_y, result);
else if (auto * y_const = checkAndGetColumnConst<ColumnUInt16>(&y))
vector_constant<Transform>(x, y_const->getValue<UInt16>(), timezone_x, timezone_y, result);
else if (auto * y_const = checkAndGetColumnConst<ColumnUInt32>(&y))
vector_constant<Transform>(x, y_const->getValue<UInt32>(), timezone_x, timezone_y, result);
else
throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN);
}
template <typename Transform, typename T1>
void dispatchConstForSecondColumn(
T1 x, const IColumn & y,
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result)
{
if (auto * y_vec = checkAndGetColumn<ColumnUInt16>(&y))
constant_vector<Transform>(x, *y_vec, timezone_x, timezone_y, result);
else if (auto * y_vec = checkAndGetColumn<ColumnUInt32>(&y))
constant_vector<Transform>(x, *y_vec, timezone_x, timezone_y, result);
else
throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN);
}
template <typename Transform, typename T1, typename T2>
void vector_vector(
const ColumnVector<T1> & x, const ColumnVector<T2> & y,
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result)
{
const auto & x_data = x.getData();
const auto & y_data = y.getData();
for (size_t i = 0, size = x.size(); i < size; ++i)
result[i] = calculate<Transform>(x_data[i], y_data[i], timezone_x, timezone_y);
}
template <typename Transform, typename T1, typename T2>
void vector_constant(
const ColumnVector<T1> & x, T2 y,
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result)
{
const auto & x_data = x.getData();
for (size_t i = 0, size = x.size(); i < size; ++i)
result[i] = calculate<Transform>(x_data[i], y, timezone_x, timezone_y);
}
template <typename Transform, typename T1, typename T2>
void constant_vector(
T1 x, const ColumnVector<T2> & y,
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result)
{
const auto & y_data = y.getData();
for (size_t i = 0, size = y.size(); i < size; ++i)
result[i] = calculate<Transform>(x, y_data[i], timezone_x, timezone_y);
}
template <typename Transform, typename T1, typename T2>
Int64 calculate(T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y)
{
return Int64(Transform::execute(y, timezone_y))
- Int64(Transform::execute(x, timezone_x));
}
};
/// Get the current time. (It is a constant, it is evaluated once for the entire query.)
class FunctionNow : public IFunction
{
@ -1071,7 +1290,7 @@ public:
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() +
". Should be DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
String time_zone_name = extractTimeZoneNameFromFunctionArguments(arguments, 1);
String time_zone_name = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0);
out_return_type = std::make_shared<DataTypeDateTime>(time_zone_name);
}
@ -1277,6 +1496,7 @@ public:
using FunctionToYear = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToYearImpl>;
using FunctionToQuarter = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToQuarterImpl>;
using FunctionToMonth = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToMonthImpl>;
using FunctionToDayOfMonth = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToDayOfMonthImpl>;
using FunctionToDayOfWeek = FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToDayOfWeekImpl>;
@ -1294,10 +1514,10 @@ using FunctionToStartOfHour = FunctionDateOrDateTimeToSomething<DataTypeDateTime
using FunctionToTime = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToTimeImpl>;
using FunctionToRelativeYearNum = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToRelativeYearNumImpl>;
using FunctionToRelativeQuarterNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeQuarterNumImpl>;
using FunctionToRelativeMonthNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeMonthNumImpl>;
using FunctionToRelativeWeekNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeWeekNumImpl>;
using FunctionToRelativeDayNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeDayNumImpl>;
using FunctionToRelativeHourNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeHourNumImpl>;
using FunctionToRelativeMinuteNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeMinuteNumImpl>;
using FunctionToRelativeSecondNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeSecondNumImpl>;

View File

@ -0,0 +1,24 @@
2017-01-01 2017-01-01 00:00:00 1 1 2017-01-01 2017-01-01 0 0
2017-02-01 2017-02-01 00:00:00 1 1 2017-01-01 2017-01-01 0 0
2017-03-01 2017-03-01 00:00:00 1 1 2017-01-01 2017-01-01 0 0
2017-04-01 2017-04-01 00:00:00 2 2 2017-04-01 2017-04-01 1 1
2017-05-01 2017-05-01 00:00:00 2 2 2017-04-01 2017-04-01 1 1
2017-06-01 2017-06-01 00:00:00 2 2 2017-04-01 2017-04-01 1 1
2017-07-01 2017-07-01 00:00:00 3 3 2017-07-01 2017-07-01 2 2
2017-08-01 2017-08-01 00:00:00 3 3 2017-07-01 2017-07-01 2 2
2017-09-01 2017-09-01 00:00:00 3 3 2017-07-01 2017-07-01 2 2
2017-10-01 2017-10-01 00:00:00 4 4 2017-10-01 2017-10-01 3 3
2017-11-01 2017-11-01 00:00:00 4 4 2017-10-01 2017-10-01 3 3
2017-12-01 2017-12-01 00:00:00 4 4 2017-10-01 2017-10-01 3 3
2018-01-01 2018-01-01 00:00:00 1 1 2018-01-01 2018-01-01 4 4
2018-02-01 2018-02-01 00:00:00 1 1 2018-01-01 2018-01-01 4 4
2018-03-01 2018-03-01 00:00:00 1 1 2018-01-01 2018-01-01 4 4
2018-04-01 2018-04-01 00:00:00 2 2 2018-04-01 2018-04-01 5 5
2018-05-01 2018-05-01 00:00:00 2 2 2018-04-01 2018-04-01 5 5
2018-06-01 2018-06-01 00:00:00 2 2 2018-04-01 2018-04-01 5 5
2018-07-01 2018-07-01 00:00:00 3 3 2018-07-01 2018-07-01 6 6
2018-08-01 2018-08-01 00:00:00 3 3 2018-07-01 2018-07-01 6 6
2018-09-01 2018-09-01 00:00:00 3 3 2018-07-01 2018-07-01 6 6
2018-10-01 2018-10-01 00:00:00 4 4 2018-10-01 2018-10-01 7 7
2018-11-01 2018-11-01 00:00:00 4 4 2018-10-01 2018-10-01 7 7
2018-12-01 2018-12-01 00:00:00 4 4 2018-10-01 2018-10-01 7 7

View File

@ -0,0 +1 @@
SELECT (toDate('2017-01-01') AS base) + INTERVAL number MONTH AS d, toDateTime(d) AS t, toQuarter(d) AS qd, toQuarter(t) AS qt, toStartOfQuarter(d) AS sqd, toStartOfQuarter(t) AS sqt, toRelativeQuarterNum(d) - toRelativeQuarterNum(base) AS qdiff_d, toRelativeQuarterNum(t) - toRelativeQuarterNum(base) as qdiff_t FROM system.numbers LIMIT 24;

View File

@ -0,0 +1,71 @@
Various intervals
-1
0
1
-7
-3
1
-23
-11
1
-104
-52
1
-730
-364
1
-17520
-8736
24
-1051200
-524160
1440
-63072000
-31449600
86400
Date and DateTime arguments
-63072000
-31449600
86400
Constant and non-constant arguments
-1051200
-524160
1440
Case insensitive
-10
Dependance of timezones
0
1
1
25
1500
90000
0
1
1
24
1440
86400
0
1
1
25
1500
90000
0
1
1
24
1440
86400
Additional test
1
1
1
1
1
1
1
1
1
1

View File

@ -0,0 +1,76 @@
SELECT 'Various intervals';
SELECT dateDiff('year', toDate('2017-12-31'), toDate('2016-01-01'));
SELECT dateDiff('year', toDate('2017-12-31'), toDate('2017-01-01'));
SELECT dateDiff('year', toDate('2017-12-31'), toDate('2018-01-01'));
SELECT dateDiff('quarter', toDate('2017-12-31'), toDate('2016-01-01'));
SELECT dateDiff('quarter', toDate('2017-12-31'), toDate('2017-01-01'));
SELECT dateDiff('quarter', toDate('2017-12-31'), toDate('2018-01-01'));
SELECT dateDiff('month', toDate('2017-12-31'), toDate('2016-01-01'));
SELECT dateDiff('month', toDate('2017-12-31'), toDate('2017-01-01'));
SELECT dateDiff('month', toDate('2017-12-31'), toDate('2018-01-01'));
SELECT dateDiff('week', toDate('2017-12-31'), toDate('2016-01-01'));
SELECT dateDiff('week', toDate('2017-12-31'), toDate('2017-01-01'));
SELECT dateDiff('week', toDate('2017-12-31'), toDate('2018-01-01'));
SELECT dateDiff('day', toDate('2017-12-31'), toDate('2016-01-01'));
SELECT dateDiff('day', toDate('2017-12-31'), toDate('2017-01-01'));
SELECT dateDiff('day', toDate('2017-12-31'), toDate('2018-01-01'));
SELECT dateDiff('hour', toDate('2017-12-31'), toDate('2016-01-01'));
SELECT dateDiff('hour', toDate('2017-12-31'), toDate('2017-01-01'));
SELECT dateDiff('hour', toDate('2017-12-31'), toDate('2018-01-01'));
SELECT dateDiff('minute', toDate('2017-12-31'), toDate('2016-01-01'));
SELECT dateDiff('minute', toDate('2017-12-31'), toDate('2017-01-01'));
SELECT dateDiff('minute', toDate('2017-12-31'), toDate('2018-01-01'));
SELECT dateDiff('second', toDate('2017-12-31'), toDate('2016-01-01'));
SELECT dateDiff('second', toDate('2017-12-31'), toDate('2017-01-01'));
SELECT dateDiff('second', toDate('2017-12-31'), toDate('2018-01-01'));
SELECT 'Date and DateTime arguments';
SELECT dateDiff('second', toDate('2017-12-31'), toDateTime('2016-01-01 00:00:00'));
SELECT dateDiff('second', toDateTime('2017-12-31 00:00:00'), toDate('2017-01-01'));
SELECT dateDiff('second', toDateTime('2017-12-31 00:00:00'), toDateTime('2018-01-01 00:00:00'));
SELECT 'Constant and non-constant arguments';
SELECT dateDiff('minute', materialize(toDate('2017-12-31')), toDate('2016-01-01'));
SELECT dateDiff('minute', toDate('2017-12-31'), materialize(toDate('2017-01-01')));
SELECT dateDiff('minute', materialize(toDate('2017-12-31')), materialize(toDate('2018-01-01')));
SELECT 'Case insensitive';
SELECT DATEDIFF('year', today(), today() - INTERVAL 10 YEAR);
SELECT 'Dependance of timezones';
SELECT dateDiff('month', toDate('2014-10-26'), toDate('2014-10-27'), 'Europe/Moscow');
SELECT dateDiff('week', toDate('2014-10-26'), toDate('2014-10-27'), 'Europe/Moscow');
SELECT dateDiff('day', toDate('2014-10-26'), toDate('2014-10-27'), 'Europe/Moscow');
SELECT dateDiff('hour', toDate('2014-10-26'), toDate('2014-10-27'), 'Europe/Moscow');
SELECT dateDiff('minute', toDate('2014-10-26'), toDate('2014-10-27'), 'Europe/Moscow');
SELECT dateDiff('second', toDate('2014-10-26'), toDate('2014-10-27'), 'Europe/Moscow');
SELECT dateDiff('month', toDate('2014-10-26'), toDate('2014-10-27'), 'UTC');
SELECT dateDiff('week', toDate('2014-10-26'), toDate('2014-10-27'), 'UTC');
SELECT dateDiff('day', toDate('2014-10-26'), toDate('2014-10-27'), 'UTC');
SELECT dateDiff('hour', toDate('2014-10-26'), toDate('2014-10-27'), 'UTC');
SELECT dateDiff('minute', toDate('2014-10-26'), toDate('2014-10-27'), 'UTC');
SELECT dateDiff('second', toDate('2014-10-26'), toDate('2014-10-27'), 'UTC');
SELECT dateDiff('month', toDateTime('2014-10-26 00:00:00', 'Europe/Moscow'), toDateTime('2014-10-27 00:00:00', 'Europe/Moscow'));
SELECT dateDiff('week', toDateTime('2014-10-26 00:00:00', 'Europe/Moscow'), toDateTime('2014-10-27 00:00:00', 'Europe/Moscow'));
SELECT dateDiff('day', toDateTime('2014-10-26 00:00:00', 'Europe/Moscow'), toDateTime('2014-10-27 00:00:00', 'Europe/Moscow'));
SELECT dateDiff('hour', toDateTime('2014-10-26 00:00:00', 'Europe/Moscow'), toDateTime('2014-10-27 00:00:00', 'Europe/Moscow'));
SELECT dateDiff('minute', toDateTime('2014-10-26 00:00:00', 'Europe/Moscow'), toDateTime('2014-10-27 00:00:00', 'Europe/Moscow'));
SELECT dateDiff('second', toDateTime('2014-10-26 00:00:00', 'Europe/Moscow'), toDateTime('2014-10-27 00:00:00', 'Europe/Moscow'));
SELECT dateDiff('month', toDateTime('2014-10-26 00:00:00', 'UTC'), toDateTime('2014-10-27 00:00:00', 'UTC'));
SELECT dateDiff('week', toDateTime('2014-10-26 00:00:00', 'UTC'), toDateTime('2014-10-27 00:00:00', 'UTC'));
SELECT dateDiff('day', toDateTime('2014-10-26 00:00:00', 'UTC'), toDateTime('2014-10-27 00:00:00', 'UTC'));
SELECT dateDiff('hour', toDateTime('2014-10-26 00:00:00', 'UTC'), toDateTime('2014-10-27 00:00:00', 'UTC'));
SELECT dateDiff('minute', toDateTime('2014-10-26 00:00:00', 'UTC'), toDateTime('2014-10-27 00:00:00', 'UTC'));
SELECT dateDiff('second', toDateTime('2014-10-26 00:00:00', 'UTC'), toDateTime('2014-10-27 00:00:00', 'UTC'));
SELECT 'Additional test';
SELECT number = dateDiff('month', now() - INTERVAL number MONTH, now()) FROM system.numbers LIMIT 10;

View File

@ -104,6 +104,7 @@ public:
inline time_t toDate(time_t t) const { return find(t).date; }
inline unsigned toMonth(time_t t) const { return find(t).month; }
inline unsigned toQuarter(time_t t) const { return (find(t).month - 1) / 3 + 1; }
inline unsigned toYear(time_t t) const { return find(t).year; }
inline unsigned toDayOfWeek(time_t t) const { return find(t).day_of_week; }
inline unsigned toDayOfMonth(time_t t) const { return find(t).day_of_month; }
@ -122,8 +123,7 @@ public:
inline DayNum_t toFirstDayNumOfWeek(time_t t) const
{
size_t index = findIndex(t);
return DayNum_t(index - (lut[index].day_of_week - 1));
return toFirstDayNumOfWeek(toDayNum(t));
}
/// Round down to start of month.
@ -140,8 +140,7 @@ public:
inline DayNum_t toFirstDayNumOfMonth(time_t t) const
{
size_t index = findIndex(t);
return DayNum_t(index - (lut[index].day_of_month - 1));
return toFirstDayNumOfMonth(toDayNum(t));
}
/// Round down to start of quarter.
@ -181,9 +180,9 @@ public:
return years_lut[lut[d].year - DATE_LUT_MIN_YEAR];
}
inline time_t toFirstDayNumOfYear(time_t t) const
inline DayNum_t toFirstDayNumOfYear(time_t t) const
{
return lut[years_lut[lut[findIndex(t)].year - DATE_LUT_MIN_YEAR]].date;
return toFirstDayNumOfYear(toDayNum(t));
}
inline time_t toFirstDayOfNextMonth(time_t t) const
@ -302,6 +301,7 @@ public:
inline time_t toDate(DayNum_t d) const { return lut[d].date; }
inline unsigned toMonth(DayNum_t d) const { return lut[d].month; }
inline unsigned toQuarter(DayNum_t d) const { return (lut[d].month - 1) / 3 + 1; }
inline unsigned toYear(DayNum_t d) const { return lut[d].year; }
inline unsigned toDayOfWeek(DayNum_t d) const { return lut[d].day_of_week; }
inline unsigned toDayOfMonth(DayNum_t d) const { return lut[d].day_of_month; }
@ -317,8 +317,7 @@ public:
inline unsigned toRelativeWeekNum(time_t t) const
{
size_t index = findIndex(t);
return (index + 8 - lut[index].day_of_week) / 7;
return toRelativeWeekNum(toDayNum(t));
}
/// Number of month from some fixed moment in the past (year * 12 + month)
@ -329,8 +328,17 @@ public:
inline unsigned toRelativeMonthNum(time_t t) const
{
size_t index = findIndex(t);
return lut[index].year * 12 + lut[index].month;
return toRelativeMonthNum(toDayNum(t));
}
inline unsigned toRelativeQuarterNum(DayNum_t d) const
{
return lut[d].year * 4 + (lut[d].month - 1) / 3;
}
inline unsigned toRelativeQuarterNum(time_t t) const
{
return toRelativeQuarterNum(toDayNum(t));
}
/// We count all hour-length intervals, unrelated to offset changes.
@ -344,11 +352,21 @@ public:
return (t + 86400 - offset_at_start_of_epoch) / 3600;
}
inline time_t toRelativeHourNum(DayNum_t d) const
{
return toRelativeHourNum(lut[d].date);
}
inline time_t toRelativeMinuteNum(time_t t) const
{
return t / 60;
}
inline time_t toRelativeMinuteNum(DayNum_t d) const
{
return toRelativeMinuteNum(lut[d].date);
}
/// Create DayNum_t from year, month, day of month.
inline DayNum_t makeDayNum(UInt16 year, UInt8 month, UInt8 day_of_month) const
{