2011-10-16 04:17:41 +00:00
|
|
|
|
#pragma once
|
|
|
|
|
|
|
|
|
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
|
|
|
|
#include <DB/DataTypes/DataTypeDate.h>
|
|
|
|
|
#include <DB/DataTypes/DataTypeDateTime.h>
|
2012-09-03 04:45:46 +00:00
|
|
|
|
#include <DB/DataTypes/DataTypeArray.h>
|
|
|
|
|
|
2011-10-16 04:17:41 +00:00
|
|
|
|
#include <DB/Columns/ColumnConst.h>
|
2012-09-03 04:45:46 +00:00
|
|
|
|
#include <DB/Columns/ColumnArray.h>
|
|
|
|
|
|
2011-10-16 04:17:41 +00:00
|
|
|
|
#include <DB/Functions/IFunction.h>
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
|
|
|
|
/** Функции работы с датой и временем.
|
|
|
|
|
*
|
|
|
|
|
* toYear, toMonth, toDayOfMonth, toDayOfWeek, toHour, toMinute, toSecond,
|
2012-08-31 20:38:05 +00:00
|
|
|
|
* toMonday, toStartOfMonth, toStartOfYear, toStartOfMinute, toStartOfHour
|
2011-10-16 04:17:41 +00:00
|
|
|
|
* toTime,
|
2012-08-31 20:38:05 +00:00
|
|
|
|
* now
|
|
|
|
|
* TODO: makeDate, makeDateTime
|
2014-06-26 00:58:14 +00:00
|
|
|
|
*
|
2011-10-16 04:17:41 +00:00
|
|
|
|
* (toDate - расположена в файле FunctionsConversion.h)
|
|
|
|
|
*
|
|
|
|
|
* Возвращаемые типы:
|
|
|
|
|
* toYear -> UInt16
|
|
|
|
|
* toMonth, toDayOfMonth, toDayOfWeek, toHour, toMinute, toSecond -> UInt8
|
2012-08-31 20:38:05 +00:00
|
|
|
|
* toMonday, toStartOfMonth, toStartOfYear -> Date
|
|
|
|
|
* toStartOfMinute, toStartOfHour, toTime, now -> DateTime
|
2012-09-03 04:45:46 +00:00
|
|
|
|
*
|
|
|
|
|
* А также:
|
|
|
|
|
*
|
|
|
|
|
* timeSlot(EventTime)
|
|
|
|
|
* - округляет время до получаса.
|
2014-06-26 00:58:14 +00:00
|
|
|
|
*
|
2012-09-03 04:45:46 +00:00
|
|
|
|
* timeSlots(StartTime, Duration)
|
|
|
|
|
* - для интервала времени, начинающегося в StartTime и продолжающегося Duration секунд,
|
|
|
|
|
* возвращает массив моментов времени, состоящий из округлений вниз до получаса точек из этого интервала.
|
|
|
|
|
* Например, timeSlots(toDateTime('2012-01-01 12:20:00'), 600) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')].
|
|
|
|
|
* Это нужно для поиска хитов, входящих в соответствующий визит.
|
2011-10-16 04:17:41 +00:00
|
|
|
|
*/
|
|
|
|
|
|
2012-09-03 04:45:46 +00:00
|
|
|
|
|
|
|
|
|
#define TIME_SLOT_SIZE 1800
|
|
|
|
|
|
|
|
|
|
|
2011-10-16 04:17:41 +00:00
|
|
|
|
struct ToYearImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toYear(t); }
|
|
|
|
|
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toYear(DayNum_t(d)); }
|
2011-10-16 04:17:41 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
struct ToMonthImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toMonth(t); }
|
|
|
|
|
static inline UInt8 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toMonth(DayNum_t(d)); }
|
2011-10-16 04:17:41 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
struct ToDayOfMonthImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toDayOfMonth(t); }
|
|
|
|
|
static inline UInt8 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toDayOfMonth(DayNum_t(d)); }
|
2011-10-16 04:17:41 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
struct ToDayOfWeekImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toDayOfWeek(t); }
|
|
|
|
|
static inline UInt8 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toDayOfWeek(DayNum_t(d)); }
|
2011-10-16 04:17:41 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
struct ToHourImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toHourInaccurate(t); }
|
|
|
|
|
static inline UInt8 execute(UInt16 d, DateLUT & date_lut)
|
2011-10-16 04:17:41 +00:00
|
|
|
|
{
|
|
|
|
|
throw Exception("Illegal type Date of argument for function toHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
struct ToMinuteImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toMinuteInaccurate(t); }
|
|
|
|
|
static inline UInt8 execute(UInt16 d, DateLUT & date_lut)
|
2011-10-16 04:17:41 +00:00
|
|
|
|
{
|
|
|
|
|
throw Exception("Illegal type Date of argument for function toMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
struct ToSecondImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toSecondInaccurate(t); }
|
|
|
|
|
static inline UInt8 execute(UInt16 d, DateLUT & date_lut)
|
2011-10-16 04:17:41 +00:00
|
|
|
|
{
|
|
|
|
|
throw Exception("Illegal type Date of argument for function toSecond", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
struct ToMondayImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toFirstDayNumOfWeek(date_lut.toDayNum(t)); }
|
|
|
|
|
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toFirstDayNumOfWeek(DayNum_t(d)); }
|
2011-10-16 04:17:41 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
struct ToStartOfMonthImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toFirstDayNumOfMonth(date_lut.toDayNum(t)); }
|
|
|
|
|
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toFirstDayNumOfMonth(DayNum_t(d)); }
|
2011-10-16 04:17:41 +00:00
|
|
|
|
};
|
|
|
|
|
|
2013-12-20 14:12:20 +00:00
|
|
|
|
struct ToStartOfQuarterImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toFirstDayNumOfQuarter(date_lut.toDayNum(t)); }
|
|
|
|
|
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toFirstDayNumOfQuarter(DayNum_t(d)); }
|
2013-12-20 14:12:20 +00:00
|
|
|
|
};
|
|
|
|
|
|
2012-08-31 20:38:05 +00:00
|
|
|
|
struct ToStartOfYearImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toFirstDayNumOfYear(date_lut.toDayNum(t)); }
|
|
|
|
|
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toFirstDayNumOfYear(DayNum_t(d)); }
|
2012-08-31 20:38:05 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2011-10-16 04:17:41 +00:00
|
|
|
|
struct ToTimeImpl
|
|
|
|
|
{
|
2011-10-16 07:11:36 +00:00
|
|
|
|
/// При переводе во время, дату будем приравнивать к 1970-01-02.
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toTimeInaccurate(t) + 86400; }
|
|
|
|
|
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
2011-10-16 04:17:41 +00:00
|
|
|
|
{
|
|
|
|
|
throw Exception("Illegal type Date of argument for function toTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
2012-08-31 20:38:05 +00:00
|
|
|
|
struct ToStartOfMinuteImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toStartOfMinuteInaccurate(t); }
|
|
|
|
|
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
2012-08-31 20:38:05 +00:00
|
|
|
|
{
|
|
|
|
|
throw Exception("Illegal type Date of argument for function toStartOfMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
struct ToStartOfHourImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toStartOfHourInaccurate(t); }
|
|
|
|
|
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
2012-08-31 20:38:05 +00:00
|
|
|
|
{
|
|
|
|
|
throw Exception("Illegal type Date of argument for function toStartOfHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
2014-01-22 12:27:55 +00:00
|
|
|
|
struct ToRelativeYearNumImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toYear(t); }
|
|
|
|
|
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toYear(DayNum_t(d)); }
|
2014-01-22 12:27:55 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
struct ToRelativeMonthNumImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeMonthNum(t); }
|
|
|
|
|
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toRelativeMonthNum(DayNum_t(d)); }
|
2014-01-22 12:27:55 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
struct ToRelativeWeekNumImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeWeekNum(t); }
|
|
|
|
|
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toRelativeWeekNum(DayNum_t(d)); }
|
2014-01-22 12:27:55 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
struct ToRelativeDayNumImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt16 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toDayNum(t); }
|
|
|
|
|
static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return static_cast<DayNum_t>(d); }
|
2014-01-22 12:27:55 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
struct ToRelativeHourNumImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeHourNum(t); }
|
|
|
|
|
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
2014-01-22 12:27:55 +00:00
|
|
|
|
{
|
2014-01-27 13:49:06 +00:00
|
|
|
|
throw Exception("Illegal type Date of argument for function toRelativeHourNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2014-01-22 12:27:55 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
struct ToRelativeMinuteNumImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeMinuteNum(t); }
|
|
|
|
|
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
2014-01-22 12:27:55 +00:00
|
|
|
|
{
|
2014-01-27 13:49:06 +00:00
|
|
|
|
throw Exception("Illegal type Date of argument for function toRelativeMinuteNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2014-01-22 12:27:55 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
struct ToRelativeSecondNumImpl
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return t; }
|
|
|
|
|
static inline UInt32 execute(UInt16 d, DateLUT & date_lut)
|
2014-01-22 12:27:55 +00:00
|
|
|
|
{
|
2014-01-27 13:49:06 +00:00
|
|
|
|
throw Exception("Illegal type Date of argument for function toRelativeSecondNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2014-01-22 12:27:55 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
2011-10-16 04:17:41 +00:00
|
|
|
|
|
|
|
|
|
template <typename FromType, typename ToType, typename Transform, typename Name>
|
|
|
|
|
struct DateTimeTransformImpl
|
|
|
|
|
{
|
|
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
2014-07-08 23:52:53 +00:00
|
|
|
|
DateLUT & date_lut = DateLUT::instance();
|
2011-10-16 04:17:41 +00:00
|
|
|
|
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (const ColumnVector<FromType> * col_from = typeid_cast<const ColumnVector<FromType> *>(&*block.getByPosition(arguments[0]).column))
|
2011-10-16 04:17:41 +00:00
|
|
|
|
{
|
|
|
|
|
ColumnVector<ToType> * col_to = new ColumnVector<ToType>;
|
|
|
|
|
block.getByPosition(result).column = col_to;
|
|
|
|
|
|
|
|
|
|
const typename ColumnVector<FromType>::Container_t & vec_from = col_from->getData();
|
|
|
|
|
typename ColumnVector<ToType>::Container_t & vec_to = col_to->getData();
|
|
|
|
|
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], date_lut);
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
else if (const ColumnConst<FromType> * col_from = typeid_cast<const ColumnConst<FromType> *>(&*block.getByPosition(arguments[0]).column))
|
2011-10-16 04:17:41 +00:00
|
|
|
|
{
|
|
|
|
|
block.getByPosition(result).column = new ColumnConst<ToType>(col_from->size(), Transform::execute(col_from->getData(), date_lut));
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
2014-11-12 17:23:26 +00:00
|
|
|
|
+ " of first argument of function " + Name::name,
|
2011-10-16 04:17:41 +00:00
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
template <typename ToDataType, typename Transform, typename Name>
|
|
|
|
|
class FunctionDateOrDateTimeToSomething : public IFunction
|
|
|
|
|
{
|
|
|
|
|
public:
|
2014-11-12 17:23:26 +00:00
|
|
|
|
static constexpr auto name = Name::name;
|
|
|
|
|
static IFunction * create(const Context & context) { return new FunctionDateOrDateTimeToSomething; };
|
|
|
|
|
|
2011-10-16 04:17:41 +00:00
|
|
|
|
/// Получить имя функции.
|
|
|
|
|
String getName() const
|
|
|
|
|
{
|
2014-11-12 17:23:26 +00:00
|
|
|
|
return name;
|
2011-10-16 04:17:41 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
|
|
|
|
DataTypePtr getReturnType(const DataTypes & arguments) const
|
|
|
|
|
{
|
|
|
|
|
if (arguments.size() != 1)
|
|
|
|
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
2013-06-21 20:34:19 +00:00
|
|
|
|
+ toString(arguments.size()) + ", should be 1.",
|
2011-10-16 04:17:41 +00:00
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
|
|
return new ToDataType;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Выполнить функцию над блоком.
|
|
|
|
|
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
|
|
|
|
IDataType * from_type = &*block.getByPosition(arguments[0]).type;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
|
|
|
|
if (typeid_cast<const DataTypeDate *>(from_type))
|
2011-10-16 04:17:41 +00:00
|
|
|
|
DateTimeTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform, Name>::execute(block, arguments, result);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
else if (typeid_cast<const DataTypeDateTime * >(from_type))
|
2011-10-16 04:17:41 +00:00
|
|
|
|
DateTimeTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform, Name>::execute(block, arguments, result);
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2012-08-31 20:38:05 +00:00
|
|
|
|
/// Получить текущее время. (Оно - константа, вычисляется один раз за весь запрос.)
|
|
|
|
|
class FunctionNow : public IFunction
|
|
|
|
|
{
|
|
|
|
|
public:
|
2014-11-12 17:23:26 +00:00
|
|
|
|
static constexpr auto name = "now";
|
|
|
|
|
static IFunction * create(const Context & context) { return new FunctionNow; };
|
|
|
|
|
|
2012-08-31 20:38:05 +00:00
|
|
|
|
/// Получить имя функции.
|
|
|
|
|
String getName() const
|
|
|
|
|
{
|
2014-11-12 17:23:26 +00:00
|
|
|
|
return name;
|
2012-08-31 20:38:05 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
|
|
|
|
DataTypePtr getReturnType(const DataTypes & arguments) const
|
|
|
|
|
{
|
|
|
|
|
if (arguments.size() != 0)
|
|
|
|
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
2013-06-21 20:34:19 +00:00
|
|
|
|
+ toString(arguments.size()) + ", should be 0.",
|
2012-08-31 20:38:05 +00:00
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
|
|
return new DataTypeDateTime;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Выполнить функцию над блоком.
|
|
|
|
|
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
|
|
|
|
block.getByPosition(result).column = new ColumnConstUInt32(
|
2013-06-08 20:19:29 +00:00
|
|
|
|
block.rowsInFirstColumn(),
|
2012-08-31 20:38:05 +00:00
|
|
|
|
time(0));
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2014-09-04 11:48:07 +00:00
|
|
|
|
class FunctionToday : public IFunction
|
|
|
|
|
{
|
|
|
|
|
public:
|
2014-11-12 17:23:26 +00:00
|
|
|
|
static constexpr auto name = "today";
|
|
|
|
|
static IFunction * create(const Context & context) { return new FunctionToday; };
|
|
|
|
|
|
2014-09-04 11:48:07 +00:00
|
|
|
|
/// Получить имя функции.
|
|
|
|
|
String getName() const
|
|
|
|
|
{
|
2014-11-12 17:23:26 +00:00
|
|
|
|
return name;
|
2014-09-04 11:48:07 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
|
|
|
|
DataTypePtr getReturnType(const DataTypes & arguments) const
|
|
|
|
|
{
|
|
|
|
|
if (arguments.size() != 0)
|
|
|
|
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
|
|
|
|
+ toString(arguments.size()) + ", should be 0.",
|
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
|
|
return new DataTypeDate;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Выполнить функцию над блоком.
|
|
|
|
|
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
|
|
|
|
block.getByPosition(result).column = new ColumnConstUInt16(
|
|
|
|
|
block.rowsInFirstColumn(),
|
|
|
|
|
DateLUT::instance().toDayNum(time(0)));
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
class FunctionYesterday : public IFunction
|
|
|
|
|
{
|
|
|
|
|
public:
|
2014-11-12 17:23:26 +00:00
|
|
|
|
static constexpr auto name = "yesterday";
|
|
|
|
|
static IFunction * create(const Context & context) { return new FunctionYesterday; };
|
|
|
|
|
|
2014-09-04 11:48:07 +00:00
|
|
|
|
/// Получить имя функции.
|
|
|
|
|
String getName() const
|
|
|
|
|
{
|
2014-11-12 17:23:26 +00:00
|
|
|
|
return name;
|
2014-09-04 11:48:07 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
|
|
|
|
DataTypePtr getReturnType(const DataTypes & arguments) const
|
|
|
|
|
{
|
|
|
|
|
if (arguments.size() != 0)
|
|
|
|
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
|
|
|
|
+ toString(arguments.size()) + ", should be 0.",
|
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
|
|
return new DataTypeDate;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Выполнить функцию над блоком.
|
|
|
|
|
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
|
|
|
|
block.getByPosition(result).column = new ColumnConstUInt16(
|
|
|
|
|
block.rowsInFirstColumn(),
|
|
|
|
|
DateLUT::instance().toDayNum(time(0)) - 1);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2012-09-03 04:45:46 +00:00
|
|
|
|
class FunctionTimeSlot : public IFunction
|
|
|
|
|
{
|
|
|
|
|
public:
|
2014-11-12 17:23:26 +00:00
|
|
|
|
static constexpr auto name = "timeSlot";
|
|
|
|
|
static IFunction * create(const Context & context) { return new FunctionTimeSlot; };
|
|
|
|
|
|
2012-09-03 04:45:46 +00:00
|
|
|
|
/// Получить имя функции.
|
|
|
|
|
String getName() const
|
|
|
|
|
{
|
2014-11-12 17:23:26 +00:00
|
|
|
|
return name;
|
2012-09-03 04:45:46 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
|
|
|
|
DataTypePtr getReturnType(const DataTypes & arguments) const
|
|
|
|
|
{
|
|
|
|
|
if (arguments.size() != 1)
|
|
|
|
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
2013-06-21 20:34:19 +00:00
|
|
|
|
+ toString(arguments.size()) + ", should be 1.",
|
2012-09-03 04:45:46 +00:00
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (!typeid_cast<const DataTypeDateTime *>(&*arguments[0]))
|
2012-09-03 04:45:46 +00:00
|
|
|
|
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be DateTime.",
|
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
|
|
|
|
return new DataTypeDateTime;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Выполнить функцию над блоком.
|
|
|
|
|
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (const ColumnUInt32 * times = typeid_cast<const ColumnUInt32 *>(&*block.getByPosition(arguments[0]).column))
|
2012-09-03 04:45:46 +00:00
|
|
|
|
{
|
|
|
|
|
ColumnUInt32 * res = new ColumnUInt32;
|
|
|
|
|
ColumnUInt32::Container_t & res_vec = res->getData();
|
|
|
|
|
const ColumnUInt32::Container_t & vec = times->getData();
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2012-09-03 04:45:46 +00:00
|
|
|
|
size_t size = vec.size();
|
|
|
|
|
res_vec.resize(size);
|
|
|
|
|
|
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
|
res_vec[i] = vec[i] / TIME_SLOT_SIZE * TIME_SLOT_SIZE;
|
|
|
|
|
|
|
|
|
|
block.getByPosition(result).column = res;
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
else if (const ColumnConstUInt32 * const_times = typeid_cast<const ColumnConstUInt32 *>(&*block.getByPosition(arguments[0]).column))
|
2012-09-03 04:45:46 +00:00
|
|
|
|
{
|
2013-06-08 20:19:29 +00:00
|
|
|
|
block.getByPosition(result).column = new ColumnConstUInt32(block.rowsInFirstColumn(), const_times->getData() / TIME_SLOT_SIZE * TIME_SLOT_SIZE);
|
2012-09-03 04:45:46 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
|
|
|
|
+ " of argument of function " + getName(),
|
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
template <typename DurationType>
|
|
|
|
|
struct TimeSlotsImpl
|
|
|
|
|
{
|
|
|
|
|
static void vector_vector(
|
2013-12-08 02:29:40 +00:00
|
|
|
|
const PODArray<UInt32> & starts, const PODArray<DurationType> & durations,
|
|
|
|
|
PODArray<UInt32> & result_values, ColumnArray::Offsets_t & result_offsets)
|
2012-09-03 04:45:46 +00:00
|
|
|
|
{
|
|
|
|
|
size_t size = starts.size();
|
|
|
|
|
|
|
|
|
|
result_offsets.resize(size);
|
|
|
|
|
result_values.reserve(size);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2012-09-03 04:45:46 +00:00
|
|
|
|
ColumnArray::Offset_t current_offset = 0;
|
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
|
{
|
|
|
|
|
for (UInt32 value = starts[i] / TIME_SLOT_SIZE; value <= (starts[i] + durations[i]) / TIME_SLOT_SIZE; ++value)
|
|
|
|
|
{
|
|
|
|
|
result_values.push_back(value * TIME_SLOT_SIZE);
|
|
|
|
|
++current_offset;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
result_offsets[i] = current_offset;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
static void vector_constant(
|
2013-12-08 02:29:40 +00:00
|
|
|
|
const PODArray<UInt32> & starts, DurationType duration,
|
|
|
|
|
PODArray<UInt32> & result_values, ColumnArray::Offsets_t & result_offsets)
|
2012-09-03 04:45:46 +00:00
|
|
|
|
{
|
|
|
|
|
size_t size = starts.size();
|
|
|
|
|
|
|
|
|
|
result_offsets.resize(size);
|
|
|
|
|
result_values.reserve(size);
|
|
|
|
|
|
|
|
|
|
ColumnArray::Offset_t current_offset = 0;
|
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
|
{
|
|
|
|
|
for (UInt32 value = starts[i] / TIME_SLOT_SIZE; value <= (starts[i] + duration) / TIME_SLOT_SIZE; ++value)
|
|
|
|
|
{
|
|
|
|
|
result_values.push_back(value * TIME_SLOT_SIZE);
|
|
|
|
|
++current_offset;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
result_offsets[i] = current_offset;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
static void constant_vector(
|
2013-12-08 02:29:40 +00:00
|
|
|
|
UInt32 start, const PODArray<DurationType> & durations,
|
|
|
|
|
PODArray<UInt32> & result_values, ColumnArray::Offsets_t & result_offsets)
|
2012-09-03 04:45:46 +00:00
|
|
|
|
{
|
|
|
|
|
size_t size = durations.size();
|
|
|
|
|
|
|
|
|
|
result_offsets.resize(size);
|
|
|
|
|
result_values.reserve(size);
|
|
|
|
|
|
|
|
|
|
ColumnArray::Offset_t current_offset = 0;
|
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
|
{
|
|
|
|
|
for (UInt32 value = start / TIME_SLOT_SIZE; value <= (start + durations[i]) / TIME_SLOT_SIZE; ++value)
|
|
|
|
|
{
|
|
|
|
|
result_values.push_back(value * TIME_SLOT_SIZE);
|
|
|
|
|
++current_offset;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
result_offsets[i] = current_offset;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
static void constant_constant(
|
|
|
|
|
UInt32 start, DurationType duration,
|
|
|
|
|
Array & result)
|
|
|
|
|
{
|
|
|
|
|
for (UInt32 value = start / TIME_SLOT_SIZE; value <= (start + duration) / TIME_SLOT_SIZE; ++value)
|
|
|
|
|
result.push_back(static_cast<UInt64>(value * TIME_SLOT_SIZE));
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
class FunctionTimeSlots : public IFunction
|
|
|
|
|
{
|
|
|
|
|
public:
|
2014-11-12 17:23:26 +00:00
|
|
|
|
static constexpr auto name = "timeSlots";
|
|
|
|
|
static IFunction * create(const Context & context) { return new FunctionTimeSlots; };
|
|
|
|
|
|
2012-09-03 04:45:46 +00:00
|
|
|
|
/// Получить имя функции.
|
|
|
|
|
String getName() const
|
|
|
|
|
{
|
2014-11-12 17:23:26 +00:00
|
|
|
|
return name;
|
2012-09-03 04:45:46 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
|
|
|
|
DataTypePtr getReturnType(const DataTypes & arguments) const
|
|
|
|
|
{
|
|
|
|
|
if (arguments.size() != 2)
|
|
|
|
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
2013-06-21 20:34:19 +00:00
|
|
|
|
+ toString(arguments.size()) + ", should be 2.",
|
2012-09-03 04:45:46 +00:00
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (!typeid_cast<const DataTypeDateTime *>(&*arguments[0]))
|
2012-09-03 04:45:46 +00:00
|
|
|
|
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be DateTime.",
|
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (!typeid_cast<const DataTypeUInt32 *>(&*arguments[1]))
|
2012-09-03 06:32:38 +00:00
|
|
|
|
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ". Must be UInt32.",
|
2012-09-03 04:45:46 +00:00
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
|
|
|
|
return new DataTypeArray(new DataTypeDateTime);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Выполнить функцию над блоком.
|
|
|
|
|
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
const ColumnUInt32 * starts = typeid_cast<const ColumnUInt32 *>(&*block.getByPosition(arguments[0]).column);
|
|
|
|
|
const ColumnConstUInt32 * const_starts = typeid_cast<const ColumnConstUInt32 *>(&*block.getByPosition(arguments[0]).column);
|
2012-09-03 04:45:46 +00:00
|
|
|
|
|
2014-06-26 00:58:14 +00:00
|
|
|
|
const ColumnUInt32 * durations = typeid_cast<const ColumnUInt32 *>(&*block.getByPosition(arguments[1]).column);
|
|
|
|
|
const ColumnConstUInt32 * const_durations = typeid_cast<const ColumnConstUInt32 *>(&*block.getByPosition(arguments[1]).column);
|
2012-09-03 04:45:46 +00:00
|
|
|
|
|
|
|
|
|
ColumnArray * res = new ColumnArray(new ColumnUInt32);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
ColumnUInt32::Container_t & res_values = typeid_cast<ColumnUInt32 &>(res->getData()).getData();
|
2012-09-03 04:45:46 +00:00
|
|
|
|
|
|
|
|
|
if (starts && durations)
|
|
|
|
|
{
|
2012-09-03 06:32:38 +00:00
|
|
|
|
TimeSlotsImpl<UInt32>::vector_vector(starts->getData(), durations->getData(), res_values, res->getOffsets());
|
2012-09-03 04:45:46 +00:00
|
|
|
|
block.getByPosition(result).column = res;
|
|
|
|
|
}
|
|
|
|
|
else if (starts && const_durations)
|
|
|
|
|
{
|
2012-09-03 06:32:38 +00:00
|
|
|
|
TimeSlotsImpl<UInt32>::vector_constant(starts->getData(), const_durations->getData(), res_values, res->getOffsets());
|
2012-09-03 04:45:46 +00:00
|
|
|
|
block.getByPosition(result).column = res;
|
|
|
|
|
}
|
|
|
|
|
else if (const_starts && durations)
|
|
|
|
|
{
|
2012-09-03 06:32:38 +00:00
|
|
|
|
TimeSlotsImpl<UInt32>::constant_vector(const_starts->getData(), durations->getData(), res_values, res->getOffsets());
|
2012-09-03 04:45:46 +00:00
|
|
|
|
block.getByPosition(result).column = res;
|
|
|
|
|
}
|
|
|
|
|
else if (const_starts && const_durations)
|
|
|
|
|
{
|
|
|
|
|
Array const_res;
|
2012-09-03 06:32:38 +00:00
|
|
|
|
TimeSlotsImpl<UInt32>::constant_constant(const_starts->getData(), const_durations->getData(), const_res);
|
2013-06-08 20:19:29 +00:00
|
|
|
|
block.getByPosition(result).column = new ColumnConstArray(block.rowsInFirstColumn(), const_res, new DataTypeArray(new DataTypeDateTime));
|
2012-09-03 04:45:46 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal columns " + block.getByPosition(arguments[0]).column->getName()
|
|
|
|
|
+ ", " + block.getByPosition(arguments[1]).column->getName()
|
|
|
|
|
+ " of arguments of function " + getName(),
|
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2014-11-12 17:23:26 +00:00
|
|
|
|
struct NameToYear { static constexpr auto name = "toYear"; };
|
|
|
|
|
struct NameToMonth { static constexpr auto name = "toMonth"; };
|
|
|
|
|
struct NameToDayOfMonth { static constexpr auto name = "toDayOfMonth"; };
|
|
|
|
|
struct NameToDayOfWeek { static constexpr auto name = "toDayOfWeek"; };
|
|
|
|
|
struct NameToHour { static constexpr auto name = "toHour"; };
|
|
|
|
|
struct NameToMinute { static constexpr auto name = "toMinute"; };
|
|
|
|
|
struct NameToSecond { static constexpr auto name = "toSecond"; };
|
|
|
|
|
struct NameToMonday { static constexpr auto name = "toMonday"; };
|
|
|
|
|
struct NameToStartOfMonth { static constexpr auto name = "toStartOfMonth"; };
|
|
|
|
|
struct NameToStartOfQuarter { static constexpr auto name = "toStartOfQuarter"; };
|
|
|
|
|
struct NameToStartOfYear { static constexpr auto name = "toStartOfYear"; };
|
|
|
|
|
struct NameToStartOfMinute { static constexpr auto name = "toStartOfMinute"; };
|
|
|
|
|
struct NameToStartOfHour { static constexpr auto name = "toStartOfHour"; };
|
|
|
|
|
struct NameToTime { static constexpr auto name = "toTime"; };
|
|
|
|
|
struct NameToRelativeYearNum { static constexpr auto name = "toRelativeYearNum"; };
|
|
|
|
|
struct NameToRelativeMonthNum { static constexpr auto name = "toRelativeMonthNum"; };
|
|
|
|
|
struct NameToRelativeWeekNum { static constexpr auto name = "toRelativeWeekNum"; };
|
|
|
|
|
struct NameToRelativeDayNum { static constexpr auto name = "toRelativeDayNum"; };
|
|
|
|
|
struct NameToRelativeHourNum { static constexpr auto name = "toRelativeHourNum"; };
|
|
|
|
|
struct NameToRelativeMinuteNum { static constexpr auto name = "toRelativeMinuteNum"; };
|
|
|
|
|
struct NameToRelativeSecondNum { static constexpr auto name = "toRelativeSecondNum"; };
|
2014-01-22 12:27:55 +00:00
|
|
|
|
|
2011-10-16 04:17:41 +00:00
|
|
|
|
|
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToYearImpl, NameToYear> FunctionToYear;
|
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToMonthImpl, NameToMonth> FunctionToMonth;
|
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToDayOfMonthImpl, NameToDayOfMonth> FunctionToDayOfMonth;
|
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToDayOfWeekImpl, NameToDayOfWeek> FunctionToDayOfWeek;
|
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToHourImpl, NameToHour> FunctionToHour;
|
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToMinuteImpl, NameToMinute> FunctionToMinute;
|
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeUInt8, ToSecondImpl, NameToSecond> FunctionToSecond;
|
2012-08-31 20:38:05 +00:00
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeDate, ToMondayImpl, NameToMonday> FunctionToMonday;
|
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfMonthImpl, NameToStartOfMonth> FunctionToStartOfMonth;
|
2013-12-20 14:12:20 +00:00
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfQuarterImpl, NameToStartOfQuarter> FunctionToStartOfQuarter;
|
2012-08-31 20:38:05 +00:00
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeDate, ToStartOfYearImpl, NameToStartOfYear> FunctionToStartOfYear;
|
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfMinuteImpl, NameToStartOfMinute> FunctionToStartOfMinute;
|
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfHourImpl, NameToStartOfHour> FunctionToStartOfHour;
|
2011-10-16 04:17:41 +00:00
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToTimeImpl, NameToTime> FunctionToTime;
|
|
|
|
|
|
2014-01-22 12:27:55 +00:00
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToRelativeYearNumImpl, NameToRelativeYearNum> FunctionToRelativeYearNum;
|
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeMonthNumImpl, NameToRelativeMonthNum> FunctionToRelativeMonthNum;
|
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeWeekNumImpl, NameToRelativeWeekNum> FunctionToRelativeWeekNum;
|
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeDayNumImpl, NameToRelativeDayNum> FunctionToRelativeDayNum;
|
|
|
|
|
|
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeHourNumImpl, NameToRelativeHourNum> FunctionToRelativeHourNum;
|
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeMinuteNumImpl, NameToRelativeMinuteNum> FunctionToRelativeMinuteNum;
|
|
|
|
|
typedef FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeSecondNumImpl, NameToRelativeSecondNum> FunctionToRelativeSecondNum;
|
|
|
|
|
|
|
|
|
|
|
2011-10-16 04:17:41 +00:00
|
|
|
|
}
|