ClickHouse/src/Functions/CustomWeekTransforms.h

170 lines
5.6 KiB
C++
Raw Normal View History

2019-06-09 15:19:15 +00:00
#pragma once
2021-05-02 19:04:26 +00:00
2019-06-09 15:19:15 +00:00
#include <Columns/ColumnVector.h>
#include <Columns/ColumnsNumber.h>
2020-09-15 09:55:57 +00:00
#include <common/types.h>
#include <Core/DecimalFunctions.h>
2019-06-09 15:19:15 +00:00
#include <Functions/FunctionHelpers.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h>
2021-05-17 07:30:42 +00:00
#include <Functions/IFunction.h>
2019-06-09 15:19:15 +00:00
#include <Common/Exception.h>
#include <common/DateLUTImpl.h>
/// The default mode value to use for the WEEK() function
#define DEFAULT_WEEK_MODE 0
2021-05-02 19:04:26 +00:00
2019-06-09 15:19:15 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
/**
* CustomWeek Transformations.
*/
static inline UInt32 dateIsNotSupported(const char * name)
{
throw Exception("Illegal type Date of argument for function " + std::string(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
/// This factor transformation will say that the function is monotone everywhere.
struct ZeroTransform
{
static inline UInt16 execute(UInt16, UInt8, const DateLUTImpl &) { return 0; }
Extended range of DateTime64 to years 1925 - 2238 The Year 1925 is a starting point because most of the timezones switched to saner (mostly 15-minutes based) offsets somewhere during 1924 or before. And that significantly simplifies implementation. 2238 is to simplify arithmetics for sanitizing LUT index access; there are less than 0x1ffff days from 1925. * Extended DateLUTImpl internal LUT to 0x1ffff items, some of which represent negative (pre-1970) time values. As a collateral benefit, Date now correctly supports dates up to 2149 (instead of 2106). * Added a new strong typedef ExtendedDayNum, which represents dates pre-1970 and post 2149. * Functions that used to return DayNum now return ExtendedDayNum. * Refactored DateLUTImpl to untie DayNum from the dual role of being a value and an index (due to negative time). Index is now a different type LUTIndex with explicit conversion functions from DatNum, time_t, and ExtendedDayNum. * Updated DateLUTImpl to properly support values close to epoch start (1970-01-01 00:00), including negative ones. * Reduced resolution of DateLUTImpl::Values::time_at_offset_change to multiple of 15-minutes to allow storing 64-bits of time_t in DateLUTImpl::Value while keeping same size. * Minor performance updates to DateLUTImpl when building month LUT by skipping non-start-of-month days. * Fixed extractTimeZoneFromFunctionArguments to work correctly with DateTime64. * New unit-tests and stateless integration tests for both DateTime and DateTime64.
2020-04-17 13:26:44 +00:00
static inline UInt16 execute(UInt32, UInt8, const DateLUTImpl &) { return 0; }
static inline UInt16 execute(Int64, UInt8, const DateLUTImpl &) { return 0; }
2019-06-09 15:19:15 +00:00
};
struct ToWeekImpl
2019-06-09 15:19:15 +00:00
{
static constexpr auto name = "toWeek";
2019-06-09 15:19:15 +00:00
Extended range of DateTime64 to years 1925 - 2238 The Year 1925 is a starting point because most of the timezones switched to saner (mostly 15-minutes based) offsets somewhere during 1924 or before. And that significantly simplifies implementation. 2238 is to simplify arithmetics for sanitizing LUT index access; there are less than 0x1ffff days from 1925. * Extended DateLUTImpl internal LUT to 0x1ffff items, some of which represent negative (pre-1970) time values. As a collateral benefit, Date now correctly supports dates up to 2149 (instead of 2106). * Added a new strong typedef ExtendedDayNum, which represents dates pre-1970 and post 2149. * Functions that used to return DayNum now return ExtendedDayNum. * Refactored DateLUTImpl to untie DayNum from the dual role of being a value and an index (due to negative time). Index is now a different type LUTIndex with explicit conversion functions from DatNum, time_t, and ExtendedDayNum. * Updated DateLUTImpl to properly support values close to epoch start (1970-01-01 00:00), including negative ones. * Reduced resolution of DateLUTImpl::Values::time_at_offset_change to multiple of 15-minutes to allow storing 64-bits of time_t in DateLUTImpl::Value while keeping same size. * Minor performance updates to DateLUTImpl when building month LUT by skipping non-start-of-month days. * Fixed extractTimeZoneFromFunctionArguments to work correctly with DateTime64. * New unit-tests and stateless integration tests for both DateTime and DateTime64.
2020-04-17 13:26:44 +00:00
static inline UInt8 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
// TODO: ditch conversion to DayNum, since it doesn't support extended range.
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
return yw.second;
}
2019-06-09 15:19:15 +00:00
static inline UInt8 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
return yw.second;
2019-06-09 15:19:15 +00:00
}
static inline UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode);
return yw.second;
2019-06-09 15:19:15 +00:00
}
using FactorTransform = ZeroTransform;
};
struct ToYearWeekImpl
2019-06-09 15:19:15 +00:00
{
static constexpr auto name = "toYearWeek";
2019-06-09 15:19:15 +00:00
Extended range of DateTime64 to years 1925 - 2238 The Year 1925 is a starting point because most of the timezones switched to saner (mostly 15-minutes based) offsets somewhere during 1924 or before. And that significantly simplifies implementation. 2238 is to simplify arithmetics for sanitizing LUT index access; there are less than 0x1ffff days from 1925. * Extended DateLUTImpl internal LUT to 0x1ffff items, some of which represent negative (pre-1970) time values. As a collateral benefit, Date now correctly supports dates up to 2149 (instead of 2106). * Added a new strong typedef ExtendedDayNum, which represents dates pre-1970 and post 2149. * Functions that used to return DayNum now return ExtendedDayNum. * Refactored DateLUTImpl to untie DayNum from the dual role of being a value and an index (due to negative time). Index is now a different type LUTIndex with explicit conversion functions from DatNum, time_t, and ExtendedDayNum. * Updated DateLUTImpl to properly support values close to epoch start (1970-01-01 00:00), including negative ones. * Reduced resolution of DateLUTImpl::Values::time_at_offset_change to multiple of 15-minutes to allow storing 64-bits of time_t in DateLUTImpl::Value while keeping same size. * Minor performance updates to DateLUTImpl when building month LUT by skipping non-start-of-month days. * Fixed extractTimeZoneFromFunctionArguments to work correctly with DateTime64. * New unit-tests and stateless integration tests for both DateTime and DateTime64.
2020-04-17 13:26:44 +00:00
static inline UInt32 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
// TODO: ditch toDayNum()
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
return yw.first * 100 + yw.second;
}
2019-06-09 15:19:15 +00:00
static inline UInt32 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
return yw.first * 100 + yw.second;
2019-06-09 15:19:15 +00:00
}
static inline UInt32 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
return yw.first * 100 + yw.second;
}
using FactorTransform = ZeroTransform;
};
struct ToStartOfWeekImpl
{
static constexpr auto name = "toStartOfWeek";
Extended range of DateTime64 to years 1925 - 2238 The Year 1925 is a starting point because most of the timezones switched to saner (mostly 15-minutes based) offsets somewhere during 1924 or before. And that significantly simplifies implementation. 2238 is to simplify arithmetics for sanitizing LUT index access; there are less than 0x1ffff days from 1925. * Extended DateLUTImpl internal LUT to 0x1ffff items, some of which represent negative (pre-1970) time values. As a collateral benefit, Date now correctly supports dates up to 2149 (instead of 2106). * Added a new strong typedef ExtendedDayNum, which represents dates pre-1970 and post 2149. * Functions that used to return DayNum now return ExtendedDayNum. * Refactored DateLUTImpl to untie DayNum from the dual role of being a value and an index (due to negative time). Index is now a different type LUTIndex with explicit conversion functions from DatNum, time_t, and ExtendedDayNum. * Updated DateLUTImpl to properly support values close to epoch start (1970-01-01 00:00), including negative ones. * Reduced resolution of DateLUTImpl::Values::time_at_offset_change to multiple of 15-minutes to allow storing 64-bits of time_t in DateLUTImpl::Value while keeping same size. * Minor performance updates to DateLUTImpl when building month LUT by skipping non-start-of-month days. * Fixed extractTimeZoneFromFunctionArguments to work correctly with DateTime64. * New unit-tests and stateless integration tests for both DateTime and DateTime64.
2020-04-17 13:26:44 +00:00
static inline UInt16 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode);
// return time_zone.toFirstDayNumOfWeek(t, week_mode);
}
static inline UInt16 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode);
Extended range of DateTime64 to years 1925 - 2238 The Year 1925 is a starting point because most of the timezones switched to saner (mostly 15-minutes based) offsets somewhere during 1924 or before. And that significantly simplifies implementation. 2238 is to simplify arithmetics for sanitizing LUT index access; there are less than 0x1ffff days from 1925. * Extended DateLUTImpl internal LUT to 0x1ffff items, some of which represent negative (pre-1970) time values. As a collateral benefit, Date now correctly supports dates up to 2149 (instead of 2106). * Added a new strong typedef ExtendedDayNum, which represents dates pre-1970 and post 2149. * Functions that used to return DayNum now return ExtendedDayNum. * Refactored DateLUTImpl to untie DayNum from the dual role of being a value and an index (due to negative time). Index is now a different type LUTIndex with explicit conversion functions from DatNum, time_t, and ExtendedDayNum. * Updated DateLUTImpl to properly support values close to epoch start (1970-01-01 00:00), including negative ones. * Reduced resolution of DateLUTImpl::Values::time_at_offset_change to multiple of 15-minutes to allow storing 64-bits of time_t in DateLUTImpl::Value while keeping same size. * Minor performance updates to DateLUTImpl when building month LUT by skipping non-start-of-month days. * Fixed extractTimeZoneFromFunctionArguments to work correctly with DateTime64. * New unit-tests and stateless integration tests for both DateTime and DateTime64.
2020-04-17 13:26:44 +00:00
// return time_zone.toFirstDayNumOfWeek(t, week_mode);
}
static inline UInt16 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
Extended range of DateTime64 to years 1925 - 2238 The Year 1925 is a starting point because most of the timezones switched to saner (mostly 15-minutes based) offsets somewhere during 1924 or before. And that significantly simplifies implementation. 2238 is to simplify arithmetics for sanitizing LUT index access; there are less than 0x1ffff days from 1925. * Extended DateLUTImpl internal LUT to 0x1ffff items, some of which represent negative (pre-1970) time values. As a collateral benefit, Date now correctly supports dates up to 2149 (instead of 2106). * Added a new strong typedef ExtendedDayNum, which represents dates pre-1970 and post 2149. * Functions that used to return DayNum now return ExtendedDayNum. * Refactored DateLUTImpl to untie DayNum from the dual role of being a value and an index (due to negative time). Index is now a different type LUTIndex with explicit conversion functions from DatNum, time_t, and ExtendedDayNum. * Updated DateLUTImpl to properly support values close to epoch start (1970-01-01 00:00), including negative ones. * Reduced resolution of DateLUTImpl::Values::time_at_offset_change to multiple of 15-minutes to allow storing 64-bits of time_t in DateLUTImpl::Value while keeping same size. * Minor performance updates to DateLUTImpl when building month LUT by skipping non-start-of-month days. * Fixed extractTimeZoneFromFunctionArguments to work correctly with DateTime64. * New unit-tests and stateless integration tests for both DateTime and DateTime64.
2020-04-17 13:26:44 +00:00
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d), week_mode);
2019-06-09 15:19:15 +00:00
}
using FactorTransform = ZeroTransform;
};
template <typename FromType, typename ToType, typename Transform>
struct Transformer
{
2020-10-17 16:48:53 +00:00
explicit Transformer(Transform transform_)
: transform(std::move(transform_))
{}
template <typename FromVectorType, typename ToVectorType>
void
vector(const FromVectorType & vec_from, ToVectorType & vec_to, UInt8 week_mode, const DateLUTImpl & time_zone) const
2019-06-09 15:19:15 +00:00
{
size_t size = vec_from.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = transform.execute(vec_from[i], week_mode, time_zone);
2019-06-09 15:19:15 +00:00
}
private:
const Transform transform;
2019-06-09 15:19:15 +00:00
};
template <typename FromDataType, typename ToDataType>
2019-06-09 15:19:15 +00:00
struct CustomWeekTransformImpl
{
template <typename Transform>
static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/, Transform transform = {})
2019-06-09 15:19:15 +00:00
{
const auto op = Transformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform>{std::move(transform)};
2019-06-09 15:19:15 +00:00
UInt8 week_mode = DEFAULT_WEEK_MODE;
if (arguments.size() > 1)
{
2020-10-17 16:48:53 +00:00
if (const auto * week_mode_column = checkAndGetColumnConst<ColumnUInt8>(arguments[1].column.get()))
2019-06-09 15:19:15 +00:00
week_mode = week_mode_column->getValue<UInt8>();
}
2020-10-17 16:48:53 +00:00
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0);
const ColumnPtr source_col = arguments[0].column;
if (const auto * sources = checkAndGetColumn<typename FromDataType::ColumnType>(source_col.get()))
2019-06-09 15:19:15 +00:00
{
auto col_to = ToDataType::ColumnType::create();
op.vector(sources->getData(), col_to->getData(), week_mode, time_zone);
2020-10-17 16:48:53 +00:00
return col_to;
2019-06-09 15:19:15 +00:00
}
else
{
throw Exception(
2020-10-17 16:48:53 +00:00
"Illegal column " + arguments[0].column->getName() + " of first argument of function "
2019-06-09 15:19:15 +00:00
+ Transform::name,
ErrorCodes::ILLEGAL_COLUMN);
}
}
};
}