2018-09-26 00:31:40 +00:00
# include <common/DateLUTImpl.h>
# include <DataTypes/DataTypeDate.h>
# include <DataTypes/DataTypeDateTime.h>
2019-11-04 14:06:22 +00:00
# include <DataTypes/DataTypeDateTime64.h>
2018-09-26 00:31:40 +00:00
# include <Columns/ColumnVector.h>
2019-12-09 13:12:54 +00:00
# include <Functions/IFunctionImpl.h>
2018-09-26 00:31:40 +00:00
# include <Functions/FunctionHelpers.h>
# include <Functions/extractTimeZoneFromFunctionArguments.h>
# include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH ;
extern const int ILLEGAL_TYPE_OF_ARGUMENT ;
extern const int ILLEGAL_COLUMN ;
}
2019-10-10 13:49:40 +00:00
/// AddOnDateTime64DefaultImpl provides default implementation of add-X functionality for DateTime64.
///
/// Default implementation is not to change fractional part, but only modify whole part as if it was DateTime.
/// That means large whole values (for scale less than 9) might not fit into UInt32-range,
/// and hence default implementation will produce incorrect results.
2019-10-09 13:18:38 +00:00
template < typename T >
2019-10-10 13:49:40 +00:00
struct AddOnDateTime64DefaultImpl
2019-10-09 13:18:38 +00:00
{
2019-10-10 13:49:40 +00:00
/*explicit*/ AddOnDateTime64DefaultImpl ( UInt32 scale_ = 0 )
2019-12-06 21:05:31 +00:00
: scale_multiplier ( DecimalUtils : : scaleMultiplier < DateTime64 : : NativeType > ( scale_ ) )
2019-10-09 13:18:38 +00:00
{ }
2019-10-10 13:49:40 +00:00
// Default implementation for add/sub on DateTime64: do math on whole part (the same way as for DateTime), leave fractional as it is.
2019-10-09 13:18:38 +00:00
inline DateTime64 execute ( const DateTime64 & t , Int64 delta , const DateLUTImpl & time_zone ) const
{
2019-12-06 21:05:31 +00:00
const auto components = DecimalUtils : : splitWithScaleMultiplier ( t , scale_multiplier ) ;
2019-10-09 13:18:38 +00:00
2019-11-03 05:38:45 +00:00
const auto whole = static_cast < const T * > ( this ) - > execute ( static_cast < UInt32 > ( components . whole ) , delta , time_zone ) ;
2019-12-06 21:05:31 +00:00
return DecimalUtils : : decimalFromComponentsWithMultiplier < DateTime64 > ( static_cast < DateTime64 : : NativeType > ( whole ) , components . fractional , scale_multiplier ) ;
2019-10-09 13:18:38 +00:00
}
UInt32 scale_multiplier = 1 ;
} ;
2018-09-26 00:31:40 +00:00
2019-10-10 13:49:40 +00:00
/// Type of first argument of 'execute' function overload defines what INPUT DataType it is used for.
/// Return type defines what is the OUTPUT (return) type of the CH function.
/// Corresponding types:
/// - UInt16 => DataTypeDate
/// - UInt32 => DataTypeDateTime
/// - DateTime64 => DataTypeDateTime64
/// Please note that INPUT and OUTPUT types may differ, e.g.:
/// - 'AddSecondsImpl::execute(UInt32, ...) -> UInt32' is available to the ClickHouse users as 'addSeconds(DateTime, ...) -> DateTime'
/// - 'AddSecondsImpl::execute(UInt16, ...) -> UInt32' is available to the ClickHouse users as 'addSeconds(Date, ...) -> DateTime'
struct AddSecondsImpl : public AddOnDateTime64DefaultImpl < AddSecondsImpl >
2018-09-26 00:31:40 +00:00
{
2019-10-10 13:49:40 +00:00
using Base = AddOnDateTime64DefaultImpl < AddSecondsImpl > ;
using Base : : Base ;
using Base : : execute ;
2018-09-26 00:31:40 +00:00
static constexpr auto name = " addSeconds " ;
static inline UInt32 execute ( UInt32 t , Int64 delta , const DateLUTImpl & )
{
return t + delta ;
}
static inline UInt32 execute ( UInt16 d , Int64 delta , const DateLUTImpl & time_zone )
{
return time_zone . fromDayNum ( DayNum ( d ) ) + delta ;
}
} ;
2019-10-10 13:49:40 +00:00
struct AddMinutesImpl : public AddOnDateTime64DefaultImpl < AddMinutesImpl >
2018-09-26 00:31:40 +00:00
{
2019-10-10 13:49:40 +00:00
using Base = AddOnDateTime64DefaultImpl < AddMinutesImpl > ;
using Base : : Base ;
using Base : : execute ;
2018-09-26 00:31:40 +00:00
static constexpr auto name = " addMinutes " ;
static inline UInt32 execute ( UInt32 t , Int64 delta , const DateLUTImpl & )
{
return t + delta * 60 ;
}
static inline UInt32 execute ( UInt16 d , Int64 delta , const DateLUTImpl & time_zone )
{
return time_zone . fromDayNum ( DayNum ( d ) ) + delta * 60 ;
}
} ;
2019-10-10 13:49:40 +00:00
struct AddHoursImpl : public AddOnDateTime64DefaultImpl < AddHoursImpl >
2018-09-26 00:31:40 +00:00
{
2019-10-10 13:49:40 +00:00
using Base = AddOnDateTime64DefaultImpl < AddHoursImpl > ;
using Base : : Base ;
using Base : : execute ;
2018-09-26 00:31:40 +00:00
static constexpr auto name = " addHours " ;
static inline UInt32 execute ( UInt32 t , Int64 delta , const DateLUTImpl & )
{
return t + delta * 3600 ;
}
static inline UInt32 execute ( UInt16 d , Int64 delta , const DateLUTImpl & time_zone )
{
return time_zone . fromDayNum ( DayNum ( d ) ) + delta * 3600 ;
}
} ;
2019-10-10 13:49:40 +00:00
struct AddDaysImpl : public AddOnDateTime64DefaultImpl < AddDaysImpl >
2018-09-26 00:31:40 +00:00
{
2019-10-10 13:49:40 +00:00
using Base = AddOnDateTime64DefaultImpl < AddDaysImpl > ;
using Base : : Base ;
using Base : : execute ;
2018-09-26 00:31:40 +00:00
static constexpr auto name = " addDays " ;
2019-10-09 13:18:38 +00:00
// static inline UInt32 execute(UInt64 t, Int64 delta, const DateLUTImpl & time_zone)
// {
// // TODO (nemkov): LUT does not support out-of range date values for now.
// return time_zone.addDays(t, delta);
// }
2019-09-26 15:12:40 +00:00
2018-09-26 00:31:40 +00:00
static inline UInt32 execute ( UInt32 t , Int64 delta , const DateLUTImpl & time_zone )
{
return time_zone . addDays ( t , delta ) ;
}
static inline UInt16 execute ( UInt16 d , Int64 delta , const DateLUTImpl & )
{
return d + delta ;
}
} ;
2019-10-10 13:49:40 +00:00
struct AddWeeksImpl : public AddOnDateTime64DefaultImpl < AddWeeksImpl >
2018-09-26 00:31:40 +00:00
{
2019-10-10 13:49:40 +00:00
using Base = AddOnDateTime64DefaultImpl < AddWeeksImpl > ;
using Base : : Base ;
using Base : : execute ;
2018-09-26 00:31:40 +00:00
static constexpr auto name = " addWeeks " ;
static inline UInt32 execute ( UInt32 t , Int64 delta , const DateLUTImpl & time_zone )
{
return time_zone . addWeeks ( t , delta ) ;
}
static inline UInt16 execute ( UInt16 d , Int64 delta , const DateLUTImpl & )
{
return d + delta * 7 ;
}
} ;
2019-10-10 13:49:40 +00:00
struct AddMonthsImpl : public AddOnDateTime64DefaultImpl < AddMonthsImpl >
2018-09-26 00:31:40 +00:00
{
2019-10-10 13:49:40 +00:00
using Base = AddOnDateTime64DefaultImpl < AddMonthsImpl > ;
using Base : : Base ;
using Base : : execute ;
2018-09-26 00:31:40 +00:00
static constexpr auto name = " addMonths " ;
static inline UInt32 execute ( UInt32 t , Int64 delta , const DateLUTImpl & time_zone )
{
return time_zone . addMonths ( t , delta ) ;
}
static inline UInt16 execute ( UInt16 d , Int64 delta , const DateLUTImpl & time_zone )
{
return time_zone . addMonths ( DayNum ( d ) , delta ) ;
}
} ;
2019-10-10 13:49:40 +00:00
struct AddQuartersImpl : public AddOnDateTime64DefaultImpl < AddQuartersImpl >
2018-12-18 13:16:48 +00:00
{
2019-10-10 13:49:40 +00:00
using Base = AddOnDateTime64DefaultImpl < AddQuartersImpl > ;
using Base : : Base ;
using Base : : execute ;
2018-12-18 13:16:48 +00:00
static constexpr auto name = " addQuarters " ;
static inline UInt32 execute ( UInt32 t , Int64 delta , const DateLUTImpl & time_zone )
{
return time_zone . addQuarters ( t , delta ) ;
}
static inline UInt16 execute ( UInt16 d , Int64 delta , const DateLUTImpl & time_zone )
{
return time_zone . addQuarters ( DayNum ( d ) , delta ) ;
}
} ;
2019-10-10 13:49:40 +00:00
struct AddYearsImpl : public AddOnDateTime64DefaultImpl < AddYearsImpl >
2018-09-26 00:31:40 +00:00
{
2019-10-10 13:49:40 +00:00
using Base = AddOnDateTime64DefaultImpl < AddYearsImpl > ;
using Base : : Base ;
using Base : : execute ;
2018-09-26 00:31:40 +00:00
static constexpr auto name = " addYears " ;
static inline UInt32 execute ( UInt32 t , Int64 delta , const DateLUTImpl & time_zone )
{
return time_zone . addYears ( t , delta ) ;
}
static inline UInt16 execute ( UInt16 d , Int64 delta , const DateLUTImpl & time_zone )
{
return time_zone . addYears ( DayNum ( d ) , delta ) ;
}
} ;
template < typename Transform >
2019-10-10 13:49:40 +00:00
struct SubtractIntervalImpl : public Transform
2018-09-26 00:31:40 +00:00
{
2019-10-10 13:49:40 +00:00
using Transform : : Transform ;
2018-09-26 00:31:40 +00:00
2019-10-09 13:18:38 +00:00
template < typename T >
2019-10-30 13:59:38 +00:00
inline auto execute ( T t , Int64 delta , const DateLUTImpl & time_zone ) const
2018-09-26 00:31:40 +00:00
{
2019-10-09 13:18:38 +00:00
return Transform : : execute ( t , - delta , time_zone ) ;
2018-09-26 00:31:40 +00:00
}
} ;
struct SubtractSecondsImpl : SubtractIntervalImpl < AddSecondsImpl > { static constexpr auto name = " subtractSeconds " ; } ;
struct SubtractMinutesImpl : SubtractIntervalImpl < AddMinutesImpl > { static constexpr auto name = " subtractMinutes " ; } ;
struct SubtractHoursImpl : SubtractIntervalImpl < AddHoursImpl > { static constexpr auto name = " subtractHours " ; } ;
struct SubtractDaysImpl : SubtractIntervalImpl < AddDaysImpl > { static constexpr auto name = " subtractDays " ; } ;
struct SubtractWeeksImpl : SubtractIntervalImpl < AddWeeksImpl > { static constexpr auto name = " subtractWeeks " ; } ;
struct SubtractMonthsImpl : SubtractIntervalImpl < AddMonthsImpl > { static constexpr auto name = " subtractMonths " ; } ;
2018-12-18 13:16:48 +00:00
struct SubtractQuartersImpl : SubtractIntervalImpl < AddQuartersImpl > { static constexpr auto name = " subtractQuarters " ; } ;
2018-09-26 00:31:40 +00:00
struct SubtractYearsImpl : SubtractIntervalImpl < AddYearsImpl > { static constexpr auto name = " subtractYears " ; } ;
2019-10-09 13:18:38 +00:00
template < typename Transform >
2018-09-26 00:31:40 +00:00
struct Adder
{
2019-10-09 13:18:38 +00:00
const Transform transform ;
explicit Adder ( Transform transform_ )
: transform ( std : : move ( transform_ ) )
{ }
template < typename FromVectorType , typename ToVectorType >
2019-10-10 13:49:40 +00:00
void vector_vector ( const FromVectorType & vec_from , ToVectorType & vec_to , const IColumn & delta , const DateLUTImpl & time_zone ) const
2018-09-26 00:31:40 +00:00
{
size_t size = vec_from . size ( ) ;
vec_to . resize ( size ) ;
for ( size_t i = 0 ; i < size ; + + i )
2019-10-09 13:18:38 +00:00
vec_to [ i ] = transform . execute ( vec_from [ i ] , delta . getInt ( i ) , time_zone ) ;
2018-09-26 00:31:40 +00:00
}
2019-10-09 13:18:38 +00:00
template < typename FromVectorType , typename ToVectorType >
2019-10-10 13:49:40 +00:00
void vector_constant ( const FromVectorType & vec_from , ToVectorType & vec_to , Int64 delta , const DateLUTImpl & time_zone ) const
2018-09-26 00:31:40 +00:00
{
size_t size = vec_from . size ( ) ;
vec_to . resize ( size ) ;
for ( size_t i = 0 ; i < size ; + + i )
2019-10-09 13:18:38 +00:00
vec_to [ i ] = transform . execute ( vec_from [ i ] , delta , time_zone ) ;
2018-09-26 00:31:40 +00:00
}
2019-10-09 13:18:38 +00:00
template < typename FromType , typename ToVectorType >
2019-10-10 13:49:40 +00:00
void constant_vector ( const FromType & from , ToVectorType & vec_to , const IColumn & delta , const DateLUTImpl & time_zone ) const
2018-09-26 00:31:40 +00:00
{
size_t size = delta . size ( ) ;
vec_to . resize ( size ) ;
for ( size_t i = 0 ; i < size ; + + i )
2019-10-09 13:18:38 +00:00
vec_to [ i ] = transform . execute ( from , delta . getInt ( i ) , time_zone ) ;
2018-09-26 00:31:40 +00:00
}
} ;
2019-10-09 13:18:38 +00:00
template < typename FromDataType , typename ToDataType , typename Transform >
2018-09-26 00:31:40 +00:00
struct DateTimeAddIntervalImpl
{
2019-10-09 13:18:38 +00:00
static void execute ( Transform transform , Block & block , const ColumnNumbers & arguments , size_t result )
2018-09-26 00:31:40 +00:00
{
2019-10-09 13:18:38 +00:00
using FromValueType = typename FromDataType : : FieldType ;
using FromColumnType = typename FromDataType : : ColumnType ;
using ToColumnType = typename ToDataType : : ColumnType ;
auto op = Adder < Transform > { std : : move ( transform ) } ;
2018-09-26 00:31:40 +00:00
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments ( block , arguments , 2 , 0 ) ;
const ColumnPtr source_col = block . getByPosition ( arguments [ 0 ] ) . column ;
2019-10-09 13:18:38 +00:00
auto result_col = block . getByPosition ( result ) . type - > createColumn ( ) ;
auto col_to = assert_cast < ToColumnType * > ( result_col . get ( ) ) ;
2018-09-26 00:31:40 +00:00
2019-10-09 13:18:38 +00:00
if ( const auto * sources = checkAndGetColumn < FromColumnType > ( source_col . get ( ) ) )
{
2018-09-26 00:31:40 +00:00
const IColumn & delta_column = * block . getByPosition ( arguments [ 1 ] ) . column ;
if ( const auto * delta_const_column = typeid_cast < const ColumnConst * > ( & delta_column ) )
2019-10-09 13:18:38 +00:00
op . vector_constant ( sources - > getData ( ) , col_to - > getData ( ) , delta_const_column - > getField ( ) . get < Int64 > ( ) , time_zone ) ;
2018-09-26 00:31:40 +00:00
else
2019-10-09 13:18:38 +00:00
op . vector_vector ( sources - > getData ( ) , col_to - > getData ( ) , delta_column , time_zone ) ;
2018-09-26 00:31:40 +00:00
}
2019-10-09 13:18:38 +00:00
else if ( const auto * sources_const = checkAndGetColumnConst < FromColumnType > ( source_col . get ( ) ) )
2018-09-26 00:31:40 +00:00
{
2019-10-09 13:18:38 +00:00
op . constant_vector ( sources_const - > template getValue < FromValueType > ( ) , col_to - > getData ( ) , * block . getByPosition ( arguments [ 1 ] ) . column , time_zone ) ;
2018-09-26 00:31:40 +00:00
}
else
{
throw Exception ( " Illegal column " + block . getByPosition ( arguments [ 0 ] ) . column - > getName ( )
+ " of first argument of function " + Transform : : name ,
ErrorCodes : : ILLEGAL_COLUMN ) ;
}
2019-10-09 13:18:38 +00:00
block . getByPosition ( result ) . column = std : : move ( result_col ) ;
2018-09-26 00:31:40 +00:00
}
} ;
2019-10-10 13:49:40 +00:00
namespace date_and_time_type_details
{
// Compile-time mapping of value (DataType::FieldType) types to corresponding DataType
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 < DateTime64 > { using ResultDataType = DataTypeDateTime64 ; } ;
}
2018-09-26 00:31:40 +00:00
template < typename Transform >
class FunctionDateOrDateTimeAddInterval : public IFunction
{
public :
static constexpr auto name = Transform : : name ;
static FunctionPtr create ( const Context & ) { return std : : make_shared < FunctionDateOrDateTimeAddInterval > ( ) ; }
String getName ( ) const override
{
return name ;
}
bool isVariadic ( ) const override { return true ; }
size_t getNumberOfArguments ( ) const override { return 0 ; }
DataTypePtr getReturnTypeImpl ( const ColumnsWithTypeAndName & arguments ) const override
{
if ( arguments . size ( ) ! = 2 & & arguments . size ( ) ! = 3 )
throw Exception ( " Number of arguments for function " + getName ( ) + " doesn't match: passed "
+ toString ( arguments . size ( ) ) + " , should be 2 or 3 " ,
ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ) ;
2019-05-24 12:11:03 +00:00
if ( ! isNativeNumber ( arguments [ 1 ] . type ) )
2018-09-26 00:31:40 +00:00
throw Exception ( " Second argument for function " + getName ( ) + " (delta) must be number " ,
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
if ( arguments . size ( ) = = 2 )
{
if ( ! isDateOrDateTime ( arguments [ 0 ] . type ) )
throw Exception { " Illegal type " + arguments [ 0 ] . type - > getName ( ) + " of argument of function " + getName ( ) +
" . Should be a date or a date with time " , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT } ;
}
else
{
if ( ! WhichDataType ( arguments [ 0 ] . type ) . isDateTime ( )
| | ! WhichDataType ( arguments [ 2 ] . type ) . isString ( ) )
throw Exception (
" Function " + getName ( ) + " supports 2 or 3 arguments. The 1st argument "
" must be of type Date or DateTime. The 2nd argument must be number. "
" The 3rd argument (optional) must be "
" a constant string with timezone name. The timezone argument is allowed "
" only when the 1st argument has the type DateTime " ,
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
}
2019-10-09 13:18:38 +00:00
switch ( arguments [ 0 ] . type - > getTypeId ( ) )
2018-09-26 00:31:40 +00:00
{
2019-10-09 13:18:38 +00:00
case TypeIndex : : Date :
2019-10-10 13:49:40 +00:00
return resolveReturnType < DataTypeDate > ( arguments ) ;
2019-10-09 13:18:38 +00:00
case TypeIndex : : DateTime :
2019-10-10 13:49:40 +00:00
return resolveReturnType < DataTypeDateTime > ( arguments ) ;
2019-10-09 13:18:38 +00:00
case TypeIndex : : DateTime64 :
2019-10-10 13:49:40 +00:00
return resolveReturnType < DataTypeDateTime64 > ( arguments ) ;
2019-10-09 13:18:38 +00:00
default :
{
throw Exception ( " Invalid type of 1st argument of function " + getName ( ) + " : "
+ arguments [ 0 ] . type - > getName ( ) + " , expected: Date, DateTime or DateTime64. " ,
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
}
2018-09-26 00:31:40 +00:00
}
2019-10-09 13:18:38 +00:00
}
2019-10-10 13:53:53 +00:00
// Helper templates to deduce return type based on argument type, since some overloads may promote or denote types, e.g. addSeconds(Date, 1) => DateTime
2019-10-09 13:18:38 +00:00
template < typename FieldType >
using TransformExecuteReturnType = decltype ( std : : declval < Transform > ( ) . execute ( FieldType ( ) , 0 , std : : declval < DateLUTImpl > ( ) ) ) ;
2019-10-10 13:53:53 +00:00
// Deduces RETURN DataType from INTPUT DataType, based on return type of Transform{}.execute(INPUT_TYPE, UInt64, DateLUTImpl).
2019-10-09 13:18:38 +00:00
// e.g. for Transform-type that has execute()-overload with 'UInt16' input and 'UInt32' return,
// argument type is expected to be 'Date', and result type is deduced to be 'DateTime'.
2019-10-10 13:53:53 +00:00
template < typename FromDataType >
using TransformResultDataType = typename date_and_time_type_details : : ResultDataTypeMap < TransformExecuteReturnType < typename FromDataType : : FieldType > > : : ResultDataType ;
2019-10-09 13:18:38 +00:00
2019-10-10 13:49:40 +00:00
template < typename FromDataType >
2019-10-09 13:18:38 +00:00
DataTypePtr resolveReturnType ( const ColumnsWithTypeAndName & arguments ) const
{
2019-10-10 13:53:53 +00:00
using ResultDataType = TransformResultDataType < FromDataType > ;
2019-10-09 13:18:38 +00:00
if constexpr ( std : : is_same_v < ResultDataType , DataTypeDate > )
return std : : make_shared < DataTypeDate > ( ) ;
else if constexpr ( std : : is_same_v < ResultDataType , DataTypeDateTime > )
2018-09-26 00:31:40 +00:00
{
2019-10-09 13:18:38 +00:00
return std : : make_shared < DataTypeDateTime > ( extractTimeZoneNameFromFunctionArguments ( arguments , 2 , 0 ) ) ;
}
else if constexpr ( std : : is_same_v < ResultDataType , DataTypeDateTime64 > )
{
2019-10-10 13:49:40 +00:00
// TODO (vnemkov): what if there is an overload of Transform::execute() that returns DateTime64 from DateTime or Date ?
// Shall we use the default scale or one from optional argument ?
2019-10-09 13:18:38 +00:00
const auto & datetime64_type = assert_cast < const DataTypeDateTime64 & > ( * arguments [ 0 ] . type ) ;
return std : : make_shared < DataTypeDateTime64 > ( datetime64_type . getScale ( ) , extractTimeZoneNameFromFunctionArguments ( arguments , 2 , 0 ) ) ;
2018-09-26 00:31:40 +00:00
}
else
{
2019-11-10 13:13:26 +00:00
static_assert ( " Failed to resolve return type. " ) ;
2018-09-26 00:31:40 +00:00
}
2019-11-11 04:15:24 +00:00
//to make PVS and GCC happy.
return nullptr ;
2018-09-26 00:31:40 +00:00
}
bool useDefaultImplementationForConstants ( ) const override { return true ; }
ColumnNumbers getArgumentsThatAreAlwaysConstant ( ) const override { return { 2 } ; }
void executeImpl ( Block & block , const ColumnNumbers & arguments , size_t result , size_t /*input_rows_count*/ ) override
{
const IDataType * from_type = block . getByPosition ( arguments [ 0 ] ) . type . get ( ) ;
WhichDataType which ( from_type ) ;
if ( which . isDate ( ) )
2019-10-09 13:18:38 +00:00
{
2019-10-10 13:53:53 +00:00
DateTimeAddIntervalImpl < DataTypeDate , TransformResultDataType < DataTypeDate > , Transform > : : execute ( Transform { } , block , arguments , result ) ;
2019-10-09 13:18:38 +00:00
}
2018-09-26 00:31:40 +00:00
else if ( which . isDateTime ( ) )
2019-10-09 13:18:38 +00:00
{
2019-10-10 13:53:53 +00:00
DateTimeAddIntervalImpl < DataTypeDateTime , TransformResultDataType < DataTypeDateTime > , Transform > : : execute ( Transform { } , block , arguments , result ) ;
2019-10-09 13:18:38 +00:00
}
else if ( const auto * datetime64_type = assert_cast < const DataTypeDateTime64 * > ( from_type ) )
{
2019-10-10 13:53:53 +00:00
DateTimeAddIntervalImpl < DataTypeDateTime64 , TransformResultDataType < DataTypeDateTime64 > , Transform > : : execute ( Transform { datetime64_type - > getScale ( ) } , block , arguments , result ) ;
2019-10-09 13:18:38 +00:00
}
2018-09-26 00:31:40 +00:00
else
throw Exception ( " Illegal type " + block . getByPosition ( arguments [ 0 ] ) . type - > getName ( ) + " of argument of function " + getName ( ) ,
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
}
} ;
}