2022-04-26 09:44:57 +00:00
# include <base/arithmeticOverflow.h>
2021-12-21 13:41:53 +00:00
# include <Common/DateLUTImpl.h>
2022-10-19 09:13:02 +00:00
# include <Columns/ColumnsDateTime.h>
2019-02-11 11:59:17 +00:00
# include <Columns/ColumnsNumber.h>
# include <DataTypes/DataTypeDate.h>
2021-07-15 11:41:52 +00:00
# include <DataTypes/DataTypeDate32.h>
2019-02-11 11:59:17 +00:00
# include <DataTypes/DataTypeDateTime.h>
2019-11-04 14:06:22 +00:00
# include <DataTypes/DataTypeDateTime64.h>
2019-02-11 11:59:17 +00:00
# include <DataTypes/DataTypeInterval.h>
# include <Functions/DateTimeTransforms.h>
# include <Functions/FunctionFactory.h>
2021-05-17 07:30:42 +00:00
# include <Functions/IFunction.h>
2019-02-11 11:59:17 +00:00
# include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH ;
extern const int ILLEGAL_COLUMN ;
extern const int ILLEGAL_TYPE_OF_ARGUMENT ;
extern const int ARGUMENT_OUT_OF_BOUND ;
}
namespace
{
2022-02-09 07:28:15 +00:00
2019-02-11 11:59:17 +00:00
class FunctionToStartOfInterval : public IFunction
{
public :
2021-04-10 23:33:54 +00:00
static FunctionPtr create ( ContextPtr ) { return std : : make_shared < FunctionToStartOfInterval > ( ) ; }
2019-02-11 11:59:17 +00:00
2023-09-29 16:52:48 +00:00
static constexpr auto name = " toStartOfInterval " ;
2019-02-11 11:59:17 +00:00
String getName ( ) const override { return name ; }
bool isVariadic ( ) const override { return true ; }
size_t getNumberOfArguments ( ) const override { return 0 ; }
2021-06-22 16:21:23 +00:00
bool isSuitableForShortCircuitArgumentsExecution ( const DataTypesWithConstInfo & /*arguments*/ ) const override { return false ; }
2021-04-29 14:48:26 +00:00
2023-09-29 16:52:48 +00:00
bool useDefaultImplementationForConstants ( ) const override { return true ; }
ColumnNumbers getArgumentsThatAreAlwaysConstant ( ) const override { return { 1 , 2 } ; }
bool hasInformationAboutMonotonicity ( ) const override { return true ; }
Monotonicity getMonotonicityForRange ( const IDataType & , const Field & , const Field & ) const override
{
return { . is_monotonic = true , . is_always_monotonic = true } ;
}
2019-02-11 11:59:17 +00:00
DataTypePtr getReturnTypeImpl ( const ColumnsWithTypeAndName & arguments ) const override
{
2019-02-12 14:18:00 +00:00
bool first_argument_is_date = false ;
auto check_first_argument = [ & ]
{
2021-05-28 12:39:36 +00:00
if ( ! isDate ( arguments [ 0 ] . type ) & & ! isDateTime ( arguments [ 0 ] . type ) & & ! isDateTime64 ( arguments [ 0 ] . type ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT , " Illegal type {} of argument of function {}. "
" Should be a date or a date with time " , arguments [ 0 ] . type - > getName ( ) , getName ( ) ) ;
2019-02-12 14:18:00 +00:00
first_argument_is_date = isDate ( arguments [ 0 ] . type ) ;
2019-02-11 11:59:17 +00:00
} ;
const DataTypeInterval * interval_type = nullptr ;
2019-02-12 14:18:00 +00:00
bool result_type_is_date = false ;
2022-02-15 23:43:08 +00:00
bool result_type_is_datetime = false ;
2023-09-29 16:52:48 +00:00
bool result_type_is_datetime_64 = false ;
2019-02-12 14:18:00 +00:00
auto check_interval_argument = [ & ]
{
2019-02-11 11:59:17 +00:00
interval_type = checkAndGetDataType < DataTypeInterval > ( arguments [ 1 ] . type . get ( ) ) ;
if ( ! interval_type )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT , " Illegal type {} of argument of function {}. "
" Should be an interval of time " , arguments [ 1 ] . type - > getName ( ) , getName ( ) ) ;
2023-09-29 16:52:48 +00:00
switch ( interval_type - > getKind ( ) )
{
case IntervalKind : : Nanosecond :
case IntervalKind : : Microsecond :
case IntervalKind : : Millisecond :
result_type_is_datetime_64 = true ;
break ;
case IntervalKind : : Second :
case IntervalKind : : Minute :
case IntervalKind : : Hour :
case IntervalKind : : Day :
result_type_is_datetime = true ;
break ;
case IntervalKind : : Week :
case IntervalKind : : Month :
case IntervalKind : : Quarter :
case IntervalKind : : Year :
result_type_is_date = true ;
break ;
}
2019-02-11 11:59:17 +00:00
} ;
2019-02-12 14:18:00 +00:00
auto check_timezone_argument = [ & ]
{
2023-09-29 16:52:48 +00:00
if ( ! isString ( arguments [ 2 ] . type ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT , " Illegal type {} of argument of function {}. "
" This argument is optional and must be a constant string with timezone name " ,
arguments [ 2 ] . type - > getName ( ) , getName ( ) ) ;
2019-02-12 14:18:00 +00:00
if ( first_argument_is_date & & result_type_is_date )
2021-09-06 14:24:03 +00:00
throw Exception ( ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ,
" The timezone argument of function {} with interval type {} is allowed only when the 1st argument "
2022-02-09 07:28:15 +00:00
" has the type DateTime or DateTime64 " ,
2021-09-06 14:24:03 +00:00
getName ( ) , interval_type - > getKind ( ) . toString ( ) ) ;
2019-02-11 11:59:17 +00:00
} ;
if ( arguments . size ( ) = = 2 )
{
2019-02-12 14:18:00 +00:00
check_first_argument ( ) ;
2019-02-11 11:59:17 +00:00
check_interval_argument ( ) ;
}
else if ( arguments . size ( ) = = 3 )
{
2019-02-12 14:18:00 +00:00
check_first_argument ( ) ;
2019-02-11 11:59:17 +00:00
check_interval_argument ( ) ;
check_timezone_argument ( ) ;
}
else
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ,
" Number of arguments for function {} doesn't match: passed {}, should be 2 or 3 " ,
getName ( ) , arguments . size ( ) ) ;
2019-02-11 11:59:17 +00:00
}
2019-02-12 14:18:00 +00:00
if ( result_type_is_date )
2021-07-01 15:13:43 +00:00
return std : : make_shared < DataTypeDate > ( ) ;
2022-02-15 23:43:08 +00:00
else if ( result_type_is_datetime )
2023-06-10 12:53:02 +00:00
return std : : make_shared < DataTypeDateTime > ( extractTimeZoneNameFromFunctionArguments ( arguments , 2 , 0 , false ) ) ;
2023-09-29 16:52:48 +00:00
else if ( result_type_is_datetime_64 )
2022-02-15 23:43:08 +00:00
{
auto scale = 0 ;
if ( interval_type - > getKind ( ) = = IntervalKind : : Nanosecond )
scale = 9 ;
else if ( interval_type - > getKind ( ) = = IntervalKind : : Microsecond )
scale = 6 ;
else if ( interval_type - > getKind ( ) = = IntervalKind : : Millisecond )
scale = 3 ;
2023-06-10 12:53:02 +00:00
return std : : make_shared < DataTypeDateTime64 > ( scale , extractTimeZoneNameFromFunctionArguments ( arguments , 2 , 0 , false ) ) ;
2022-02-15 23:43:08 +00:00
}
2023-09-29 16:52:48 +00:00
UNREACHABLE ( ) ;
2019-02-11 11:59:17 +00:00
}
2022-02-15 23:43:08 +00:00
ColumnPtr executeImpl ( const ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , size_t /* input_rows_count */ ) const override
2019-02-11 11:59:17 +00:00
{
2020-10-19 15:27:41 +00:00
const auto & time_column = arguments [ 0 ] ;
const auto & interval_column = arguments [ 1 ] ;
2020-04-17 13:26:44 +00:00
const auto & time_zone = extractTimeZoneFromFunctionArguments ( arguments , 2 , 0 ) ;
2023-09-29 16:52:48 +00:00
auto result_column = dispatchForTimeColumn ( time_column , interval_column , result_type , time_zone ) ;
2020-10-19 15:27:41 +00:00
return result_column ;
2019-02-11 11:59:17 +00:00
}
private :
2023-09-29 16:52:48 +00:00
ColumnPtr dispatchForTimeColumn (
2022-02-15 23:43:08 +00:00
const ColumnWithTypeAndName & time_column , const ColumnWithTypeAndName & interval_column , const DataTypePtr & result_type , const DateLUTImpl & time_zone ) const
2019-02-11 11:59:17 +00:00
{
2019-10-22 07:43:14 +00:00
const auto & from_datatype = * time_column . type . get ( ) ;
2022-02-15 23:43:08 +00:00
2023-09-29 16:52:48 +00:00
if ( isDateTime64 ( from_datatype ) )
2022-02-15 23:43:08 +00:00
{
2022-10-19 09:13:02 +00:00
const auto * time_column_vec = checkAndGetColumn < ColumnDateTime64 > ( time_column . column . get ( ) ) ;
2022-02-15 23:43:08 +00:00
auto scale = assert_cast < const DataTypeDateTime64 & > ( from_datatype ) . getScale ( ) ;
if ( time_column_vec )
2023-09-29 16:52:48 +00:00
return dispatchForIntervalColumn ( assert_cast < const DataTypeDateTime64 & > ( from_datatype ) , * time_column_vec , interval_column , result_type , time_zone , scale ) ;
2022-02-15 23:43:08 +00:00
}
2023-09-29 16:52:48 +00:00
if ( isDateTime ( from_datatype ) )
2019-02-11 11:59:17 +00:00
{
2022-10-19 09:13:02 +00:00
const auto * time_column_vec = checkAndGetColumn < ColumnDateTime > ( time_column . column . get ( ) ) ;
2019-02-11 11:59:17 +00:00
if ( time_column_vec )
2023-09-29 16:52:48 +00:00
return dispatchForIntervalColumn ( assert_cast < const DataTypeDateTime & > ( from_datatype ) , * time_column_vec , interval_column , result_type , time_zone ) ;
2019-02-11 11:59:17 +00:00
}
2023-09-29 16:52:48 +00:00
if ( isDate ( from_datatype ) )
2019-02-11 11:59:17 +00:00
{
2022-10-19 09:13:02 +00:00
const auto * time_column_vec = checkAndGetColumn < ColumnDate > ( time_column . column . get ( ) ) ;
2019-02-11 11:59:17 +00:00
if ( time_column_vec )
2023-09-29 16:52:48 +00:00
return dispatchForIntervalColumn ( assert_cast < const DataTypeDate & > ( from_datatype ) , * time_column_vec , interval_column , result_type , time_zone ) ;
2019-10-22 07:43:14 +00:00
}
2023-09-29 16:52:48 +00:00
if ( isDate32 ( from_datatype ) )
2021-07-15 11:41:52 +00:00
{
2022-10-19 09:13:02 +00:00
const auto * time_column_vec = checkAndGetColumn < ColumnDate32 > ( time_column . column . get ( ) ) ;
2021-07-15 11:41:52 +00:00
if ( time_column_vec )
2023-09-29 16:52:48 +00:00
return dispatchForIntervalColumn ( assert_cast < const DataTypeDate32 & > ( from_datatype ) , * time_column_vec , interval_column , result_type , time_zone ) ;
2019-02-11 11:59:17 +00:00
}
2023-09-29 16:52:48 +00:00
throw Exception ( ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT , " Illegal column for first argument of function {}. Must contain dates or dates with time " , getName ( ) ) ;
2019-02-11 11:59:17 +00:00
}
2023-09-29 16:52:48 +00:00
template < typename TimeColumnType , typename TimeDataType >
2019-02-11 11:59:17 +00:00
ColumnPtr dispatchForIntervalColumn (
2023-09-29 16:52:48 +00:00
const TimeDataType & time_data_type , const TimeColumnType & time_column , const ColumnWithTypeAndName & interval_column ,
2022-02-15 23:43:08 +00:00
const DataTypePtr & result_type , const DateLUTImpl & time_zone , const UInt16 scale = 1 ) const
2019-02-11 11:59:17 +00:00
{
const auto * interval_type = checkAndGetDataType < DataTypeInterval > ( interval_column . type . get ( ) ) ;
if ( ! interval_type )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : ILLEGAL_COLUMN , " Illegal column for second argument of function {}, must be an interval of time. " , getName ( ) ) ;
2023-09-29 16:52:48 +00:00
2019-02-11 11:59:17 +00:00
const auto * interval_column_const_int64 = checkAndGetColumnConst < ColumnInt64 > ( interval_column . column . get ( ) ) ;
if ( ! interval_column_const_int64 )
2023-09-29 16:52:48 +00:00
throw Exception ( ErrorCodes : : ILLEGAL_COLUMN , " Illegal column for second argument of function {}, must be a const interval of time. " , getName ( ) ) ;
2019-02-11 11:59:17 +00:00
Int64 num_units = interval_column_const_int64 - > getValue < Int64 > ( ) ;
if ( num_units < = 0 )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : ARGUMENT_OUT_OF_BOUND , " Value for second argument of function {} must be positive. " , getName ( ) ) ;
2019-02-11 11:59:17 +00:00
switch ( interval_type - > getKind ( ) )
{
2022-02-09 07:28:15 +00:00
case IntervalKind : : Nanosecond :
2023-09-29 16:52:48 +00:00
return execute < TimeDataType , DataTypeDateTime64 , IntervalKind : : Nanosecond > ( time_data_type , time_column , num_units , result_type , time_zone , scale ) ;
2022-02-09 07:28:15 +00:00
case IntervalKind : : Microsecond :
2023-09-29 16:52:48 +00:00
return execute < TimeDataType , DataTypeDateTime64 , IntervalKind : : Microsecond > ( time_data_type , time_column , num_units , result_type , time_zone , scale ) ;
2022-02-09 07:28:15 +00:00
case IntervalKind : : Millisecond :
2023-09-29 16:52:48 +00:00
return execute < TimeDataType , DataTypeDateTime64 , IntervalKind : : Millisecond > ( time_data_type , time_column , num_units , result_type , time_zone , scale ) ;
2019-11-16 13:54:52 +00:00
case IntervalKind : : Second :
2023-09-29 16:52:48 +00:00
return execute < TimeDataType , DataTypeDateTime , IntervalKind : : Second > ( time_data_type , time_column , num_units , result_type , time_zone , scale ) ;
2019-11-16 13:54:52 +00:00
case IntervalKind : : Minute :
2023-09-29 16:52:48 +00:00
return execute < TimeDataType , DataTypeDateTime , IntervalKind : : Minute > ( time_data_type , time_column , num_units , result_type , time_zone , scale ) ;
2019-11-16 13:54:52 +00:00
case IntervalKind : : Hour :
2023-09-29 16:52:48 +00:00
return execute < TimeDataType , DataTypeDateTime , IntervalKind : : Hour > ( time_data_type , time_column , num_units , result_type , time_zone , scale ) ;
2019-11-16 13:54:52 +00:00
case IntervalKind : : Day :
2023-09-29 16:52:48 +00:00
return execute < TimeDataType , DataTypeDateTime , IntervalKind : : Day > ( time_data_type , time_column , num_units , result_type , time_zone , scale ) ;
2019-11-16 13:54:52 +00:00
case IntervalKind : : Week :
2023-09-29 16:52:48 +00:00
return execute < TimeDataType , DataTypeDate , IntervalKind : : Week > ( time_data_type , time_column , num_units , result_type , time_zone , scale ) ;
2019-11-16 13:54:52 +00:00
case IntervalKind : : Month :
2023-09-29 16:52:48 +00:00
return execute < TimeDataType , DataTypeDate , IntervalKind : : Month > ( time_data_type , time_column , num_units , result_type , time_zone , scale ) ;
2019-11-16 13:54:52 +00:00
case IntervalKind : : Quarter :
2023-09-29 16:52:48 +00:00
return execute < TimeDataType , DataTypeDate , IntervalKind : : Quarter > ( time_data_type , time_column , num_units , result_type , time_zone , scale ) ;
2019-11-16 13:54:52 +00:00
case IntervalKind : : Year :
2023-09-29 16:52:48 +00:00
return execute < TimeDataType , DataTypeDate , IntervalKind : : Year > ( time_data_type , time_column , num_units , result_type , time_zone , scale ) ;
2019-02-11 11:59:17 +00:00
}
2022-10-07 19:20:14 +00:00
UNREACHABLE ( ) ;
2019-02-11 11:59:17 +00:00
}
2023-09-29 16:52:48 +00:00
template < typename TimeDataType , typename ToDataType , IntervalKind : : Kind unit , typename ColumnType >
2023-10-08 14:26:31 +00:00
ColumnPtr execute ( const TimeDataType & , const ColumnType & time_column_type , Int64 num_units , const DataTypePtr & result_type , const DateLUTImpl & time_zone , const UInt16 scale ) const
2019-02-11 11:59:17 +00:00
{
2022-02-15 23:43:08 +00:00
using ToColumnType = typename ToDataType : : ColumnType ;
2022-10-07 10:46:45 +00:00
using ToFieldType = typename ToDataType : : FieldType ;
2022-02-15 23:43:08 +00:00
const auto & time_data = time_column_type . getData ( ) ;
size_t size = time_data . size ( ) ;
auto result_col = result_type - > createColumn ( ) ;
2022-02-16 16:23:10 +00:00
auto * col_to = assert_cast < ToColumnType * > ( result_col . get ( ) ) ;
2022-02-15 23:43:08 +00:00
auto & result_data = col_to - > getData ( ) ;
2019-02-11 11:59:17 +00:00
result_data . resize ( size ) ;
2019-10-22 07:43:14 +00:00
2022-02-15 23:43:08 +00:00
Int64 scale_multiplier = DecimalUtils : : scaleMultiplier < DateTime64 > ( scale ) ;
for ( size_t i = 0 ; i ! = size ; + + i )
2023-09-29 16:52:48 +00:00
result_data [ i ] = static_cast < ToFieldType > ( ToStartOfInterval < unit > : : execute ( time_data [ i ] , num_units , time_zone , scale_multiplier ) ) ;
2022-02-15 23:43:08 +00:00
return result_col ;
2019-02-11 11:59:17 +00:00
}
} ;
2020-09-07 18:00:37 +00:00
}
2019-02-11 11:59:17 +00:00
2022-07-04 07:01:39 +00:00
REGISTER_FUNCTION ( ToStartOfInterval )
2019-02-11 11:59:17 +00:00
{
factory . registerFunction < FunctionToStartOfInterval > ( ) ;
}
}