add datetype32 function support

This commit is contained in:
Neng Liu 2021-06-17 16:03:31 +08:00
parent 2bd7e6e454
commit b6eab5fd2f
23 changed files with 311 additions and 8 deletions

View File

@ -3,6 +3,7 @@
#include <AggregateFunctions/AggregateFunctionSequenceMatch.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <ext/range.h>
@ -62,6 +63,8 @@ AggregateFunctionPtr createAggregateFunctionSequenceBase(const std::string & nam
return std::make_shared<AggregateFunction<DataTypeDateTime::FieldType, Data<DataTypeDateTime::FieldType>>>(argument_types, params, pattern);
else if (which.isDate())
return std::make_shared<AggregateFunction<DataTypeDate::FieldType, Data<DataTypeDate::FieldType>>>(argument_types, params, pattern);
else if (which.isDate32())
return std::make_shared<AggregateFunction<DataTypeDate32::FieldType, Data<DataTypeDate32::FieldType>>>(argument_types, params, pattern);
throw Exception{"Illegal type " + time_arg->getName() + " of first argument of aggregate function "
+ name + ", must be DateTime",

View File

@ -4,6 +4,7 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeFixedString.h>
@ -50,6 +51,8 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
return res;
else if (which.isDate())
return std::make_shared<AggregateFunctionUniq<DataTypeDate::FieldType, Data>>(argument_types);
else if (which.isDate32())
return std::make_shared<AggregateFunctionUniq<DataTypeDate32::FieldType, Data>>(argument_types);
else if (which.isDateTime())
return std::make_shared<AggregateFunctionUniq<DataTypeDateTime::FieldType, Data>>(argument_types);
else if (which.isStringOrFixedString())
@ -96,6 +99,8 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
return res;
else if (which.isDate())
return std::make_shared<AggregateFunctionUniq<DataTypeDate::FieldType, Data<DataTypeDate::FieldType>>>(argument_types);
else if (which.isDate32())
return std::make_shared<AggregateFunctionUniq<DataTypeDate32::FieldType, Data<DataTypeDate32::FieldType>>>(argument_types);
else if (which.isDateTime())
return std::make_shared<AggregateFunctionUniq<DataTypeDateTime::FieldType, Data<DataTypeDateTime::FieldType>>>(argument_types);
else if (which.isStringOrFixedString())

View File

@ -6,6 +6,7 @@
#include <Common/FieldVisitors.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <functional>
@ -48,6 +49,8 @@ namespace
return res;
else if (which.isDate())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDate::FieldType>>(argument_types, params);
else if (which.isDate32())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDate32::FieldType>>(argument_types, params);
else if (which.isDateTime())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDateTime::FieldType>>(argument_types, params);
else if (which.isStringOrFixedString())

View File

@ -3,6 +3,7 @@
#include <AggregateFunctions/AggregateFunctionUniqUpTo.h>
#include <Common/FieldVisitors.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeFixedString.h>
@ -59,6 +60,8 @@ AggregateFunctionPtr createAggregateFunctionUniqUpTo(const std::string & name, c
return res;
else if (which.isDate())
return std::make_shared<AggregateFunctionUniqUpTo<DataTypeDate::FieldType>>(threshold, argument_types, params);
else if (which.isDate32())
return std::make_shared<AggregateFunctionUniqUpTo<DataTypeDate32::FieldType>>(threshold, argument_types, params);
else if (which.isDateTime())
return std::make_shared<AggregateFunctionUniqUpTo<DataTypeDateTime::FieldType>>(threshold, argument_types, params);
else if (which.isStringOrFixedString())

View File

@ -3,6 +3,7 @@
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <ext/range.h>
@ -45,6 +46,8 @@ AggregateFunctionPtr createAggregateFunctionWindowFunnel(const std::string & nam
return res;
else if (which.isDate())
return std::make_shared<AggregateFunctionWindowFunnel<DataTypeDate::FieldType, Data<DataTypeDate::FieldType>>>(arguments, params);
else if (which.isDate32())
return std::make_shared<AggregateFunctionWindowFunnel<DataTypeDate32::FieldType, Data<DataTypeDate32::FieldType>>>(arguments, params);
else if (which.isDateTime())
return std::make_shared<AggregateFunctionWindowFunnel<DataTypeDateTime::FieldType, Data<DataTypeDateTime::FieldType>>>(arguments, params);

View File

@ -62,6 +62,8 @@ void ExternalResultDescription::init(const Block & sample_block_)
types.emplace_back(ValueType::vtString, is_nullable);
else if (which.isDate())
types.emplace_back(ValueType::vtDate, is_nullable);
else if (which.isDate32())
types.emplace_back(ValueType::vtDate32, is_nullable);
else if (which.isDateTime())
types.emplace_back(ValueType::vtDateTime, is_nullable);
else if (which.isUUID())

View File

@ -24,6 +24,7 @@ struct ExternalResultDescription
vtFloat64,
vtString,
vtDate,
vtDate32,
vtDateTime,
vtUUID,
vtDateTime64,

View File

@ -77,6 +77,8 @@ MutableColumnUniquePtr DataTypeLowCardinality::createColumnUniqueImpl(const IDat
return creator(static_cast<ColumnFixedString *>(nullptr));
else if (which.isDate())
return creator(static_cast<ColumnVector<UInt16> *>(nullptr));
else if (which.isDate32())
return creator(static_cast<ColumnVector<Int32> *>(nullptr));
else if (which.isDateTime())
return creator(static_cast<ColumnVector<UInt32> *>(nullptr));
else if (which.isUUID())

View File

@ -322,9 +322,11 @@ struct WhichDataType
constexpr bool isEnum() const { return isEnum8() || isEnum16(); }
constexpr bool isDate() const { return idx == TypeIndex::Date; }
constexpr bool isDate32() const { return idx == TypeIndex::Date32; }
constexpr bool isDateTime() const { return idx == TypeIndex::DateTime; }
constexpr bool isDateTime64() const { return idx == TypeIndex::DateTime64; }
constexpr bool isDateOrDateTime() const { return isDate() || isDateTime() || isDateTime64(); }
constexpr bool isDateOrDateTime() const { return isDate() || isDate32() || isDateTime() || isDateTime64(); }
constexpr bool isDateOrDate32() const { return isDate() || isDate32(); }
constexpr bool isString() const { return idx == TypeIndex::String; }
constexpr bool isFixedString() const { return idx == TypeIndex::FixedString; }
@ -350,6 +352,10 @@ struct WhichDataType
template <typename T>
inline bool isDate(const T & data_type) { return WhichDataType(data_type).isDate(); }
template <typename T>
inline bool isDate32(const T & data_type) { return WhichDataType(data_type).isDate32(); }
template <typename T>
inline bool isDateOrDate32(const T & data_type) { return WhichDataType(data_type).isDateOrDate32(); }
template <typename T>
inline bool isDateOrDateTime(const T & data_type) { return WhichDataType(data_type).isDateOrDateTime(); }
template <typename T>
inline bool isDateTime(const T & data_type) { return WhichDataType(data_type).isDateTime(); }

View File

@ -35,6 +35,7 @@ static inline UInt32 dateIsNotSupported(const char * name)
struct ZeroTransform
{
static inline UInt16 execute(UInt16, UInt8, const DateLUTImpl &) { return 0; }
static inline UInt16 execute(Int32, UInt8, const DateLUTImpl &) { return 0; }
static inline UInt16 execute(UInt32, UInt8, const DateLUTImpl &) { return 0; }
static inline UInt16 execute(Int64, UInt8, const DateLUTImpl &) { return 0; }
};
@ -54,6 +55,11 @@ struct ToWeekImpl
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
return yw.second;
}
static inline UInt8 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(ExtendedDayNum(d), week_mode);
return yw.second;
}
static inline UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode);
@ -79,6 +85,11 @@ struct ToYearWeekImpl
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
return yw.first * 100 + yw.second;
}
static inline UInt32 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
YearWeek yw = time_zone.toYearWeek(ExtendedDayNum (d), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
return yw.first * 100 + yw.second;
}
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));
@ -102,6 +113,10 @@ struct ToStartOfWeekImpl
return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode);
// return time_zone.toFirstDayNumOfWeek(t, week_mode);
}
static inline UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d), week_mode);
}
static inline UInt16 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d), week_mode);

View File

@ -45,6 +45,7 @@ struct ZeroTransform
{
static inline UInt16 execute(Int64, const DateLUTImpl &) { return 0; }
static inline UInt16 execute(UInt32, const DateLUTImpl &) { return 0; }
static inline UInt16 execute(Int32, const DateLUTImpl &) { return 0; }
static inline UInt16 execute(UInt16, const DateLUTImpl &) { return 0; }
};
@ -60,6 +61,10 @@ struct ToDateImpl
{
return UInt16(time_zone.toDayNum(t));
}
static inline UInt16 execute(Int32 t, const DateLUTImpl & time_zone)
{
return dateIsNotSupported(name);
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl &)
{
return d;
@ -81,6 +86,10 @@ struct ToStartOfDayImpl
{
return time_zone.toDate(t);
}
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toDate(ExtendedDayNum(d));
}
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toDate(ExtendedDayNum(d));
@ -103,6 +112,10 @@ struct ToMondayImpl
//return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t));
return time_zone.toFirstDayNumOfWeek(t);
}
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d));
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d));
@ -123,6 +136,10 @@ struct ToStartOfMonthImpl
{
return time_zone.toFirstDayNumOfMonth(time_zone.toDayNum(t));
}
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfMonth(ExtendedDayNum(d));
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfMonth(ExtendedDayNum(d));
@ -143,6 +160,10 @@ struct ToStartOfQuarterImpl
{
return time_zone.toFirstDayNumOfQuarter(time_zone.toDayNum(t));
}
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfQuarter(ExtendedDayNum(d));
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfQuarter(ExtendedDayNum(d));
@ -163,6 +184,10 @@ struct ToStartOfYearImpl
{
return time_zone.toFirstDayNumOfYear(time_zone.toDayNum(t));
}
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfYear(ExtendedDayNum(d));
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfYear(ExtendedDayNum(d));
@ -185,7 +210,10 @@ struct ToTimeImpl
{
return time_zone.toTime(t) + 86400;
}
static inline UInt32 execute(Int32, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
static inline UInt32 execute(UInt16, const DateLUTImpl &)
{
return dateIsNotSupported(name);
@ -206,6 +234,10 @@ struct ToStartOfMinuteImpl
{
return time_zone.toStartOfMinute(t);
}
static inline UInt32 execute(Int32, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
static inline UInt32 execute(UInt16, const DateLUTImpl &)
{
return dateIsNotSupported(name);
@ -241,6 +273,10 @@ struct ToStartOfSecondImpl
{
throw Exception("Illegal type DateTime of argument for function " + std::string(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
static inline UInt32 execute(Int32, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
static inline UInt32 execute(UInt16, const DateLUTImpl &)
{
return dateIsNotSupported(name);
@ -261,6 +297,10 @@ struct ToStartOfFiveMinuteImpl
{
return time_zone.toStartOfFiveMinute(t);
}
static inline UInt32 execute(Int32, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
static inline UInt32 execute(UInt16, const DateLUTImpl &)
{
return dateIsNotSupported(name);
@ -281,6 +321,10 @@ struct ToStartOfTenMinutesImpl
{
return time_zone.toStartOfTenMinutes(t);
}
static inline UInt32 execute(Int32, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
static inline UInt32 execute(UInt16, const DateLUTImpl &)
{
return dateIsNotSupported(name);
@ -301,6 +345,10 @@ struct ToStartOfFifteenMinutesImpl
{
return time_zone.toStartOfFifteenMinutes(t);
}
static inline UInt32 execute(Int32, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
static inline UInt32 execute(UInt16, const DateLUTImpl &)
{
return dateIsNotSupported(name);
@ -325,6 +373,11 @@ struct TimeSlotImpl
return t / 1800 * 1800;
}
static inline UInt32 execute(Int32, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
static inline UInt32 execute(UInt16, const DateLUTImpl &)
{
return dateIsNotSupported(name);
@ -347,6 +400,11 @@ struct ToStartOfHourImpl
return time_zone.toStartOfHour(t);
}
static inline UInt32 execute(Int32, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
static inline UInt32 execute(UInt16, const DateLUTImpl &)
{
return dateIsNotSupported(name);
@ -367,6 +425,10 @@ struct ToYearImpl
{
return time_zone.toYear(t);
}
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toYear(ExtendedDayNum(d));
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toYear(ExtendedDayNum(d));
@ -387,6 +449,10 @@ struct ToQuarterImpl
{
return time_zone.toQuarter(t);
}
static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toQuarter(ExtendedDayNum(d));
}
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toQuarter(ExtendedDayNum(d));
@ -407,6 +473,10 @@ struct ToMonthImpl
{
return time_zone.toMonth(t);
}
static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toMonth(ExtendedDayNum(d));
}
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toMonth(ExtendedDayNum(d));
@ -427,6 +497,10 @@ struct ToDayOfMonthImpl
{
return time_zone.toDayOfMonth(t);
}
static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toDayOfMonth(ExtendedDayNum(d));
}
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toDayOfMonth(ExtendedDayNum(d));
@ -447,6 +521,10 @@ struct ToDayOfWeekImpl
{
return time_zone.toDayOfWeek(t);
}
static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toDayOfWeek(ExtendedDayNum(d));
}
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toDayOfWeek(ExtendedDayNum(d));
@ -467,6 +545,10 @@ struct ToDayOfYearImpl
{
return time_zone.toDayOfYear(t);
}
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toDayOfYear(ExtendedDayNum(d));
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toDayOfYear(ExtendedDayNum(d));
@ -487,7 +569,10 @@ struct ToHourImpl
{
return time_zone.toHour(t);
}
static inline UInt8 execute(Int32, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
static inline UInt8 execute(UInt16, const DateLUTImpl &)
{
return dateIsNotSupported(name);
@ -510,6 +595,11 @@ struct TimezoneOffsetImpl
return time_zone.timezoneOffset(t);
}
static inline time_t execute(Int32, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
static inline time_t execute(UInt16, const DateLUTImpl &)
{
return dateIsNotSupported(name);
@ -530,6 +620,10 @@ struct ToMinuteImpl
{
return time_zone.toMinute(t);
}
static inline UInt8 execute(Int32, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
static inline UInt8 execute(UInt16, const DateLUTImpl &)
{
return dateIsNotSupported(name);
@ -550,6 +644,10 @@ struct ToSecondImpl
{
return time_zone.toSecond(t);
}
static inline UInt8 execute(Int32, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
static inline UInt8 execute(UInt16, const DateLUTImpl &)
{
return dateIsNotSupported(name);
@ -570,6 +668,10 @@ struct ToISOYearImpl
{
return time_zone.toISOYear(time_zone.toDayNum(t));
}
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toISOYear(ExtendedDayNum(d));
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toISOYear(ExtendedDayNum(d));
@ -590,6 +692,10 @@ struct ToStartOfISOYearImpl
{
return time_zone.toFirstDayNumOfISOYear(time_zone.toDayNum(t));
}
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfISOYear(ExtendedDayNum(d));
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toFirstDayNumOfISOYear(ExtendedDayNum(d));
@ -610,6 +716,10 @@ struct ToISOWeekImpl
{
return time_zone.toISOWeek(time_zone.toDayNum(t));
}
static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toISOWeek(ExtendedDayNum(d));
}
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toISOWeek(ExtendedDayNum(d));
@ -630,6 +740,10 @@ struct ToRelativeYearNumImpl
{
return time_zone.toYear(static_cast<time_t>(t));
}
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toYear(ExtendedDayNum(d));
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toYear(ExtendedDayNum(d));
@ -650,6 +764,10 @@ struct ToRelativeQuarterNumImpl
{
return time_zone.toRelativeQuarterNum(static_cast<time_t>(t));
}
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toRelativeQuarterNum(ExtendedDayNum(d));
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toRelativeQuarterNum(ExtendedDayNum(d));
@ -670,6 +788,10 @@ struct ToRelativeMonthNumImpl
{
return time_zone.toRelativeMonthNum(static_cast<time_t>(t));
}
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toRelativeMonthNum(ExtendedDayNum(d));
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toRelativeMonthNum(ExtendedDayNum(d));
@ -690,6 +812,10 @@ struct ToRelativeWeekNumImpl
{
return time_zone.toRelativeWeekNum(static_cast<time_t>(t));
}
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toRelativeWeekNum(ExtendedDayNum(d));
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toRelativeWeekNum(ExtendedDayNum(d));
@ -710,6 +836,10 @@ struct ToRelativeDayNumImpl
{
return time_zone.toDayNum(static_cast<time_t>(t));
}
static inline UInt16 execute(Int32 d, const DateLUTImpl &)
{
return static_cast<ExtendedDayNum>(d);
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl &)
{
return static_cast<DayNum>(d);
@ -731,6 +861,10 @@ struct ToRelativeHourNumImpl
{
return time_zone.toRelativeHourNum(static_cast<time_t>(t));
}
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toRelativeHourNum(ExtendedDayNum(d));
}
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toRelativeHourNum(ExtendedDayNum(d));
@ -751,6 +885,10 @@ struct ToRelativeMinuteNumImpl
{
return time_zone.toRelativeMinuteNum(static_cast<time_t>(t));
}
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toRelativeMinuteNum(ExtendedDayNum(d));
}
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toRelativeMinuteNum(ExtendedDayNum(d));
@ -771,6 +909,10 @@ struct ToRelativeSecondNumImpl
{
return t;
}
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.fromDayNum(ExtendedDayNum(d));
}
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.fromDayNum(ExtendedDayNum(d));
@ -791,6 +933,10 @@ struct ToYYYYMMImpl
{
return time_zone.toNumYYYYMM(t);
}
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toNumYYYYMM(static_cast<ExtendedDayNum>(d));
}
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toNumYYYYMM(static_cast<DayNum>(d));
@ -811,6 +957,10 @@ struct ToYYYYMMDDImpl
{
return time_zone.toNumYYYYMMDD(t);
}
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toNumYYYYMMDD(static_cast<ExtendedDayNum>(d));
}
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toNumYYYYMMDD(static_cast<DayNum>(d));
@ -831,6 +981,10 @@ struct ToYYYYMMDDhhmmssImpl
{
return time_zone.toNumYYYYMMDDhhmmss(t);
}
static inline UInt64 execute(Int32 d, const DateLUTImpl & time_zone)
{
return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(static_cast<ExtendedDayNum>(d)));
}
static inline UInt64 execute(UInt16 d, const DateLUTImpl & time_zone)
{
return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(static_cast<DayNum>(d)));

View File

@ -1,5 +1,6 @@
#pragma once
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <Functions/CustomWeekTransforms.h>
@ -104,6 +105,9 @@ public:
if (which.isDate())
return CustomWeekTransformImpl<DataTypeDate, ToDataType>::execute(
arguments, result_type, input_rows_count, Transform{});
else if (which.isDate32())
return CustomWeekTransformImpl<DataTypeDate32, ToDataType>::execute(
arguments, result_type, input_rows_count, Transform{});
else if (which.isDateTime())
return CustomWeekTransformImpl<DataTypeDateTime, ToDataType>::execute(
arguments, result_type, input_rows_count, Transform{});

View File

@ -2,6 +2,7 @@
#include <common/DateLUTImpl.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
@ -117,6 +118,11 @@ struct AddDaysImpl
{
return d + delta;
}
static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &)
{
return d + delta;
}
};
struct AddWeeksImpl
@ -138,6 +144,11 @@ struct AddWeeksImpl
{
return d + delta * 7;
}
static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &)
{
return d + delta * 7;
}
};
struct AddMonthsImpl
@ -159,6 +170,11 @@ struct AddMonthsImpl
{
return time_zone.addMonths(ExtendedDayNum(d), delta);
}
static inline Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.addMonths(ExtendedDayNum(d), delta);
}
};
struct AddQuartersImpl
@ -180,6 +196,11 @@ struct AddQuartersImpl
{
return time_zone.addQuarters(ExtendedDayNum(d), delta);
}
static inline Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.addQuarters(ExtendedDayNum(d), delta);
}
};
struct AddYearsImpl
@ -201,6 +222,11 @@ struct AddYearsImpl
{
return time_zone.addYears(ExtendedDayNum(d), delta);
}
static inline Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.addYears(ExtendedDayNum(d), delta);
}
};
template <typename Transform>
@ -342,7 +368,7 @@ template <typename FieldType> struct ResultDataTypeMap {};
template <> struct ResultDataTypeMap<UInt16> { using ResultDataType = DataTypeDate; };
template <> struct ResultDataTypeMap<Int16> { using ResultDataType = DataTypeDate; };
template <> struct ResultDataTypeMap<UInt32> { using ResultDataType = DataTypeDateTime; };
template <> struct ResultDataTypeMap<Int32> { using ResultDataType = DataTypeDateTime; };
template <> struct ResultDataTypeMap<Int32> { using ResultDataType = DataTypeDate32; };
template <> struct ResultDataTypeMap<DateTime64> { using ResultDataType = DataTypeDateTime64; };
template <> struct ResultDataTypeMap<Int64> { using ResultDataType = DataTypeDateTime64; };
}
@ -470,6 +496,11 @@ public:
return DateTimeAddIntervalImpl<DataTypeDate, TransformResultDataType<DataTypeDate>, Transform>::execute(
Transform{}, arguments, result_type);
}
else if (which.isDate32())
{
return DateTimeAddIntervalImpl<DataTypeDate32, TransformResultDataType<DataTypeDate32>, Transform>::execute(
Transform{}, arguments, result_type);
}
else if (which.isDateTime())
{
return DateTimeAddIntervalImpl<DataTypeDateTime, TransformResultDataType<DataTypeDateTime>, Transform>::execute(

View File

@ -57,7 +57,7 @@ public:
"must be of type Date or DateTime. The 2nd 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>)
if (isDateOrDate32(arguments[0].type) && (std::is_same_v<ToDataType, DataTypeDate> || std::is_same_v<ToDataType, DataTypeDate32>))
throw Exception(
"The timezone argument of function " + getName() + " is allowed only when the 1st argument has the type DateTime",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -103,6 +103,8 @@ public:
if (which.isDate())
return DateTimeTransformImpl<DataTypeDate, ToDataType, Transform>::execute(arguments, result_type, input_rows_count);
else if (which.isDate32())
return DateTimeTransformImpl<DataTypeDate32, ToDataType, Transform>::execute(arguments, result_type, input_rows_count);
else if (which.isDateTime())
return DateTimeTransformImpl<DataTypeDateTime, ToDataType, Transform>::execute(arguments, result_type, input_rows_count);
else if (which.isDateTime64())
@ -146,6 +148,12 @@ public:
== Transform::FactorTransform::execute(UInt16(right.get<UInt64>()), date_lut)
? is_monotonic : is_not_monotonic;
}
else if (checkAndGetDataType<DataTypeDate32>(&type))
{
return Transform::FactorTransform::execute(Int32(left.get<UInt64>()), date_lut)
== Transform::FactorTransform::execute(Int32(right.get<UInt64>()), date_lut)
? is_monotonic : is_not_monotonic;
}
else
{
return Transform::FactorTransform::execute(UInt32(left.get<UInt64>()), date_lut)

View File

@ -1081,7 +1081,7 @@ public:
const DataTypeTuple * right_tuple = checkAndGetDataType<DataTypeTuple>(arguments[1].get());
bool both_represented_by_number = arguments[0]->isValueRepresentedByNumber() && arguments[1]->isValueRepresentedByNumber();
bool has_date = left.isDate() || right.isDate();
bool has_date = left.isDateOrDate32() || right.isDateOrDate32();
if (!((both_represented_by_number && !has_date) /// Do not allow to compare date and number.
|| (left.isStringOrFixedString() || right.isStringOrFixedString()) /// Everything can be compared with string by conversion.

View File

@ -682,6 +682,8 @@ public:
return executeType<Int64>(arguments);
else if (which.isDate())
return executeType<UInt16>(arguments);
else if (which.isDate32())
return executeType<Int32>(arguments);
else if (which.isDateTime())
return executeType<UInt32>(arguments);
else if (which.isDecimal32())
@ -985,6 +987,7 @@ private:
else if (which.isEnum8()) executeIntType<Int8, first>(icolumn, vec_to);
else if (which.isEnum16()) executeIntType<Int16, first>(icolumn, vec_to);
else if (which.isDate()) executeIntType<UInt16, first>(icolumn, vec_to);
else if (which.isDate32()) executeIntType<Int32, first>(icolumn, vec_to);
else if (which.isDateTime()) executeIntType<UInt32, first>(icolumn, vec_to);
/// TODO: executeIntType() for Decimal32/64 leads to incompatible result
else if (which.isDecimal32()) executeBigIntType<Decimal32, first>(icolumn, vec_to);

View File

@ -6,6 +6,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeNullable.h>
@ -405,6 +406,9 @@ ColumnPtr FunctionArrayIntersect::executeImpl(const ColumnsWithTypeAndName & arg
using DateMap = ClearableHashMapWithStackMemory<DataTypeDate::FieldType,
size_t, DefaultHash<DataTypeDate::FieldType>, INITIAL_SIZE_DEGREE>;
using Date32Map = ClearableHashMapWithStackMemory<DataTypeDate32::FieldType,
size_t, DefaultHash<DataTypeDate32::FieldType>, INITIAL_SIZE_DEGREE>;
using DateTimeMap = ClearableHashMapWithStackMemory<
DataTypeDateTime::FieldType, size_t,
DefaultHash<DataTypeDateTime::FieldType>, INITIAL_SIZE_DEGREE>;
@ -419,6 +423,8 @@ ColumnPtr FunctionArrayIntersect::executeImpl(const ColumnsWithTypeAndName & arg
if (which.isDate())
result_column = execute<DateMap, ColumnVector<DataTypeDate::FieldType>, true>(arrays, std::move(column));
else if (which.isDate32())
result_column = execute<Date32Map, ColumnVector<DataTypeDate32::FieldType>, true>(arrays, std::move(column));
else if (which.isDateTime())
result_column = execute<DateTimeMap, ColumnVector<DataTypeDateTime::FieldType>, true>(arrays, std::move(column));
else if (which.isString())

View File

@ -5,6 +5,7 @@
#include <Columns/ColumnNullable.h>
#include <Common/assert_cast.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypesNumber.h>
@ -115,6 +116,8 @@ private:
f(Float64());
else if (which.isDate())
f(DataTypeDate::FieldType());
else if (which.isDate32())
f(DataTypeDate::FieldType());
else if (which.isDateTime())
f(DataTypeDateTime::FieldType());
else

View File

@ -1,6 +1,7 @@
#include <common/DateLUTImpl.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeInterval.h>
@ -39,6 +40,11 @@ namespace
return time_zone.toStartOfYearInterval(ExtendedDayNum(d), years);
}
static UInt16 execute(Int32 d, UInt64 years, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfYearInterval(ExtendedDayNum(d), years);
}
static UInt16 execute(UInt32 t, UInt64 years, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years);
@ -60,6 +66,11 @@ namespace
return time_zone.toStartOfQuarterInterval(ExtendedDayNum(d), quarters);
}
static UInt16 execute(Int32 d, UInt64 quarters, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfQuarterInterval(ExtendedDayNum(d), quarters);
}
static UInt16 execute(UInt32 t, UInt64 quarters, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters);
@ -81,6 +92,11 @@ namespace
return time_zone.toStartOfMonthInterval(ExtendedDayNum(d), months);
}
static UInt16 execute(Int32 d, UInt64 months, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfMonthInterval(ExtendedDayNum(d), months);
}
static UInt16 execute(UInt32 t, UInt64 months, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months);
@ -102,6 +118,11 @@ namespace
return time_zone.toStartOfWeekInterval(ExtendedDayNum(d), weeks);
}
static UInt16 execute(Int32 d, UInt64 weeks, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfWeekInterval(ExtendedDayNum(d), weeks);
}
static UInt16 execute(UInt32 t, UInt64 weeks, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks);
@ -123,6 +144,11 @@ namespace
return time_zone.toStartOfDayInterval(ExtendedDayNum(d), days);
}
static UInt32 execute(Int32 d, UInt64 days, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfDayInterval(ExtendedDayNum(d), days);
}
static UInt32 execute(UInt32 t, UInt64 days, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days);
@ -140,6 +166,7 @@ namespace
static constexpr auto name = function_name;
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
static UInt32 execute(Int32, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
static UInt32 execute(UInt32 t, UInt64 hours, const DateLUTImpl & time_zone) { return time_zone.toStartOfHourInterval(t, hours); }
static UInt32 execute(Int64 t, UInt64 hours, const DateLUTImpl & time_zone) { return time_zone.toStartOfHourInterval(t, hours); }
};
@ -151,6 +178,8 @@ namespace
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
static UInt32 execute(Int32, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
static UInt32 execute(UInt32 t, UInt64 minutes, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfMinuteInterval(t, minutes);
@ -169,6 +198,8 @@ namespace
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
static UInt32 execute(Int32, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
static UInt32 execute(UInt32 t, UInt64 seconds, const DateLUTImpl & time_zone)
{
return time_zone.toStartOfSecondInterval(t, seconds);
@ -254,7 +285,10 @@ public:
}
if (result_type_is_date)
return std::make_shared<DataTypeDate>();
if (isDate(arguments[0].type))
return std::make_shared<DataTypeDate>();
else
return std::make_shared<DataTypeDate32>();
else
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
}
@ -299,6 +333,12 @@ private:
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDate&>(from_datatype), *time_column_vec, interval_column, time_zone);
}
if (which_type.isDate32())
{
const auto * time_column_vec = checkAndGetColumn<ColumnInt32>(time_column.column.get());
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDate32&>(from_datatype), *time_column_vec, interval_column, time_zone);
}
if (which_type.isDateTime64())
{
const auto * time_column_vec = checkAndGetColumn<DataTypeDateTime64::ColumnType>(time_column.column.get());

View File

@ -90,6 +90,7 @@ struct BloomFilterHash
else if (which.isEnum8()) return build_hash_column(getNumberTypeHash<Int64, Int8>(field));
else if (which.isEnum16()) return build_hash_column(getNumberTypeHash<Int64, Int16>(field));
else if (which.isDate()) return build_hash_column(getNumberTypeHash<UInt64, UInt16>(field));
else if (which.isDate32()) return build_hash_column(getNumberTypeHash<UInt64, Int32>(field));
else if (which.isDateTime()) return build_hash_column(getNumberTypeHash<UInt64, UInt32>(field));
else if (which.isFloat32()) return build_hash_column(getNumberTypeHash<Float64, Float64>(field));
else if (which.isFloat64()) return build_hash_column(getNumberTypeHash<Float64, Float64>(field));
@ -151,6 +152,7 @@ struct BloomFilterHash
else if (which.isEnum8()) getNumberTypeHash<Int8, is_first>(column, vec, pos);
else if (which.isEnum16()) getNumberTypeHash<Int16, is_first>(column, vec, pos);
else if (which.isDate()) getNumberTypeHash<UInt16, is_first>(column, vec, pos);
else if (which.isDate32()) getNumberTypeHash<Int32, is_first>(column, vec, pos);
else if (which.isDateTime()) getNumberTypeHash<UInt32, is_first>(column, vec, pos);
else if (which.isFloat32()) getNumberTypeHash<Float32, is_first>(column, vec, pos);
else if (which.isFloat64()) getNumberTypeHash<Float64, is_first>(column, vec, pos);

View File

@ -159,10 +159,18 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
{
return static_cast<UInt16>(static_cast<const DataTypeDateTime &>(*from_type_hint).getTimeZone().toDayNum(src.get<UInt64>()).toUnderType());
}
else if (which_type.isDate32() && which_from_type.isDateTime())
{
return static_cast<Int32>(static_cast<const DataTypeDateTime &>(*from_type_hint).getTimeZone().toDayNum(src.get<UInt64>()).toUnderType());
}
else if (which_type.isDateTime() && which_from_type.isDate())
{
return static_cast<const DataTypeDateTime &>(type).getTimeZone().fromDayNum(DayNum(src.get<UInt64>()));
}
else if (which_type.isDateTime() && which_from_type.isDate32())
{
return static_cast<const DataTypeDateTime &>(type).getTimeZone().fromDayNum(DayNum(src.get<Int32>()));
}
else if (type.isValueRepresentedByNumber() && src.getType() != Field::Types::String)
{
if (which_type.isUInt8()) return convertNumericType<UInt8>(src, type);

View File

@ -43,7 +43,7 @@ FillingTransform::FillingTransform(
DataTypePtr to_type;
/// TODO Wrong results for big integers.
if (isInteger(type) || which.isDate() || which.isDateTime())
if (isInteger(type) || which.isDate() || which.isDate32() || which.isDateTime())
{
max_type = Field::Types::Int64;
to_type = std::make_shared<DataTypeInt64>();

View File

@ -1545,6 +1545,7 @@ void checkVersionColumnTypesConversion(const IDataType * old_type, const IDataTy
if ((which_old_type.isInt() && !which_new_type.isInt())
|| (which_old_type.isUInt() && !which_new_type.isUInt())
|| (which_old_type.isDate() && !which_new_type.isDate())
|| (which_old_type.isDate32() && !which_new_type.isDate32())
|| (which_old_type.isDateTime() && !which_new_type.isDateTime())
|| (which_old_type.isFloat() && !which_new_type.isFloat()))
{