2011-10-15 23:40:56 +00:00
# pragma once
2016-01-13 00:32:59 +00:00
# include <type_traits>
2017-04-01 09:19:00 +00:00
# include <IO/WriteBufferFromVector.h>
# include <IO/ReadBufferFromMemory.h>
2017-07-21 06:35:58 +00:00
# include <IO/Operators.h>
2018-02-11 23:57:07 +00:00
# include <IO/parseDateTimeBestEffort.h>
2017-04-01 09:19:00 +00:00
# include <DataTypes/DataTypeFactory.h>
# include <DataTypes/DataTypesNumber.h>
2018-08-21 18:25:38 +00:00
# include <DataTypes/DataTypesDecimal.h>
2017-04-01 09:19:00 +00:00
# include <DataTypes/DataTypeString.h>
# include <DataTypes/DataTypeFixedString.h>
# include <DataTypes/DataTypeDate.h>
2021-07-15 11:41:52 +00:00
# include <DataTypes/DataTypeDate32.h>
2017-04-01 09:19:00 +00:00
# include <DataTypes/DataTypeDateTime.h>
2019-10-24 10:44:38 +00:00
# include <DataTypes/DataTypeDateTime64.h>
2017-04-01 09:19:00 +00:00
# include <DataTypes/DataTypeEnum.h>
# include <DataTypes/DataTypeArray.h>
# include <DataTypes/DataTypeTuple.h>
2020-10-10 06:49:03 +00:00
# include <DataTypes/DataTypeMap.h>
2017-04-01 09:19:00 +00:00
# include <DataTypes/DataTypeNullable.h>
2017-12-09 20:56:53 +00:00
# include <DataTypes/DataTypeNothing.h>
2017-07-04 10:42:53 +00:00
# include <DataTypes/DataTypeUUID.h>
2017-10-30 02:18:06 +00:00
# include <DataTypes/DataTypeInterval.h>
2019-09-02 09:16:49 +00:00
# include <DataTypes/DataTypeAggregateFunction.h>
2021-03-09 14:46:52 +00:00
# include <DataTypes/Serializations/SerializationDecimal.h>
2018-06-10 19:22:49 +00:00
# include <Formats/FormatSettings.h>
2017-04-01 09:19:00 +00:00
# include <Columns/ColumnString.h>
# include <Columns/ColumnFixedString.h>
# include <Columns/ColumnConst.h>
# include <Columns/ColumnArray.h>
# include <Columns/ColumnNullable.h>
# include <Columns/ColumnTuple.h>
2020-10-10 06:49:03 +00:00
# include <Columns/ColumnMap.h>
2018-02-22 21:20:02 +00:00
# include <Columns/ColumnsCommon.h>
2019-08-21 02:28:04 +00:00
# include <Common/assert_cast.h>
2020-11-18 09:38:03 +00:00
# include <Common/quoteString.h>
2020-11-05 19:09:17 +00:00
# include <Core/AccurateComparison.h>
2019-12-12 14:16:59 +00:00
# include <Functions/IFunctionAdaptors.h>
2017-04-01 09:19:00 +00:00
# include <Functions/FunctionsMiscellaneous.h>
2017-07-21 06:35:58 +00:00
# include <Functions/FunctionHelpers.h>
2018-09-26 00:31:40 +00:00
# include <Functions/DateTimeTransforms.h>
2020-12-05 13:18:56 +00:00
# include <Functions/toFixedString.h>
2020-04-17 13:26:44 +00:00
# include <Functions/TransformDateTime64.h>
2018-09-27 15:55:22 +00:00
# include <DataTypes/DataTypeLowCardinality.h>
# include <Columns/ColumnLowCardinality.h>
2020-12-05 13:18:56 +00:00
# include <Interpreters/Context.h>
2011-10-15 23:40:56 +00:00
namespace DB
{
2016-01-12 02:21:15 +00:00
namespace ErrorCodes
{
2017-04-01 07:20:54 +00:00
extern const int ATTEMPT_TO_READ_AFTER_EOF ;
extern const int CANNOT_PARSE_NUMBER ;
extern const int CANNOT_READ_ARRAY_FROM_TEXT ;
extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED ;
extern const int CANNOT_PARSE_QUOTED_STRING ;
extern const int CANNOT_PARSE_ESCAPE_SEQUENCE ;
extern const int CANNOT_PARSE_DATE ;
extern const int CANNOT_PARSE_DATETIME ;
extern const int CANNOT_PARSE_TEXT ;
2017-06-15 09:12:32 +00:00
extern const int CANNOT_PARSE_UUID ;
2018-12-07 03:20:27 +00:00
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION ;
2018-02-18 02:22:32 +00:00
extern const int LOGICAL_ERROR ;
extern const int TYPE_MISMATCH ;
extern const int CANNOT_CONVERT_TYPE ;
extern const int ILLEGAL_COLUMN ;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH ;
extern const int ILLEGAL_TYPE_OF_ARGUMENT ;
extern const int NOT_IMPLEMENTED ;
2018-02-22 21:20:02 +00:00
extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN ;
2016-01-12 02:21:15 +00:00
}
2016-08-30 16:14:05 +00:00
2016-07-30 04:39:51 +00:00
/** Type conversion functions.
* toType - conversion in " natural way " ;
2011-10-15 23:40:56 +00:00
*/
2018-08-21 18:25:38 +00:00
inline UInt32 extractToDecimalScale ( const ColumnWithTypeAndName & named_column )
{
const auto * arg_type = named_column . type . get ( ) ;
bool ok = checkAndGetDataType < DataTypeUInt64 > ( arg_type )
| | checkAndGetDataType < DataTypeUInt32 > ( arg_type )
| | checkAndGetDataType < DataTypeUInt16 > ( arg_type )
| | checkAndGetDataType < DataTypeUInt8 > ( arg_type ) ;
if ( ! ok )
throw Exception ( " Illegal type of toDecimal() scale " + named_column.type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT) ;
Field field ;
named_column . column - > get ( 0 , field ) ;
return field . get < UInt32 > ( ) ;
}
2020-11-24 21:11:28 +00:00
/// Function toUnixTimestamp has exactly the same implementation as toDateTime of String type.
struct NameToUnixTimestamp { static constexpr auto name = " toUnixTimestamp " ; } ;
2020-12-05 13:18:56 +00:00
struct AccurateConvertStrategyAdditions
2020-11-05 19:09:17 +00:00
{
2020-11-12 11:27:02 +00:00
UInt32 scale { 0 } ;
2020-11-05 19:09:17 +00:00
} ;
2020-12-05 13:18:56 +00:00
struct AccurateOrNullConvertStrategyAdditions
2020-11-05 19:09:17 +00:00
{
2020-12-05 13:18:56 +00:00
UInt32 scale { 0 } ;
2020-11-05 19:09:17 +00:00
} ;
2011-10-15 23:40:56 +00:00
2020-12-23 21:57:33 +00:00
struct ConvertDefaultBehaviorTag { } ;
struct ConvertReturnNullOnErrorTag { } ;
2016-07-30 04:39:51 +00:00
/** Conversion of number types to each other, enums to numbers, dates and datetimes to numbers and back: done by straight assignment.
* ( Date is represented internally as number of days from some day ; DateTime - as unix timestamp )
2011-10-15 23:40:56 +00:00
*/
2020-12-23 21:57:33 +00:00
template < typename FromDataType , typename ToDataType , typename Name , typename SpecialTag = ConvertDefaultBehaviorTag >
2011-10-16 01:57:10 +00:00
struct ConvertImpl
2011-10-15 23:40:56 +00:00
{
2017-04-01 07:20:54 +00:00
using FromFieldType = typename FromDataType : : FieldType ;
using ToFieldType = typename ToDataType : : FieldType ;
2018-08-31 08:59:21 +00:00
template < typename Additions = void * >
2020-10-19 18:37:44 +00:00
static ColumnPtr NO_SANITIZE_UNDEFINED execute (
2021-05-03 19:56:40 +00:00
const ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type [[maybe_unused]] , size_t input_rows_count ,
2020-10-17 14:23:37 +00:00
Additions additions [[maybe_unused]] = Additions ( ) )
2017-04-01 07:20:54 +00:00
{
2020-10-17 14:23:37 +00:00
const ColumnWithTypeAndName & named_from = arguments [ 0 ] ;
2018-08-21 18:25:38 +00:00
2019-10-24 10:44:38 +00:00
using ColVecFrom = typename FromDataType : : ColumnType ;
using ColVecTo = typename ToDataType : : ColumnType ;
2018-08-27 16:16:16 +00:00
2020-11-24 21:11:28 +00:00
if ( std : : is_same_v < Name , NameToUnixTimestamp > )
{
2021-07-15 11:40:45 +00:00
if ( isDateOrDate32 ( named_from . type ) )
2020-11-28 12:02:51 +00:00
throw Exception ( " Illegal type " + named_from . type - > getName ( ) + " of first argument of function " + Name : : name ,
2020-11-24 21:11:28 +00:00
ErrorCodes : : ILLEGAL_COLUMN ) ;
}
2019-10-24 10:44:38 +00:00
if constexpr ( ( IsDataTypeDecimal < FromDataType > | | IsDataTypeDecimal < ToDataType > )
& & ! ( std : : is_same_v < DataTypeDateTime64 , FromDataType > | | std : : is_same_v < DataTypeDateTime64 , ToDataType > ) )
2018-09-25 16:03:50 +00:00
{
if constexpr ( ! IsDataTypeDecimalOrNumber < FromDataType > | | ! IsDataTypeDecimalOrNumber < ToDataType > )
2019-10-24 10:44:38 +00:00
{
2018-09-25 16:03:50 +00:00
throw Exception ( " Illegal column " + named_from . column - > getName ( ) + " of first argument of function " + Name : : name ,
ErrorCodes : : ILLEGAL_COLUMN ) ;
2019-10-24 10:44:38 +00:00
}
2018-09-25 16:03:50 +00:00
}
2018-08-27 16:16:16 +00:00
if ( const ColVecFrom * col_from = checkAndGetColumn < ColVecFrom > ( named_from . column . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
2018-08-27 16:16:16 +00:00
typename ColVecTo : : MutablePtr col_to = nullptr ;
2018-09-10 13:52:18 +00:00
if constexpr ( IsDataTypeDecimal < ToDataType > )
2018-08-21 18:25:38 +00:00
{
2020-11-05 19:09:17 +00:00
UInt32 scale ;
2020-12-05 13:18:56 +00:00
if constexpr ( std : : is_same_v < Additions , AccurateConvertStrategyAdditions >
| | std : : is_same_v < Additions , AccurateOrNullConvertStrategyAdditions > )
2020-11-05 19:09:17 +00:00
{
scale = additions . scale ;
}
else
{
scale = additions ;
}
2018-08-27 16:16:16 +00:00
col_to = ColVecTo : : create ( 0 , scale ) ;
2018-08-21 18:25:38 +00:00
}
2018-08-27 16:16:16 +00:00
else
col_to = ColVecTo : : create ( ) ;
2017-04-01 07:20:54 +00:00
2018-08-27 16:16:16 +00:00
const auto & vec_from = col_from - > getData ( ) ;
auto & vec_to = col_to - > getData ( ) ;
2021-05-03 19:56:40 +00:00
vec_to . resize ( input_rows_count ) ;
2017-04-01 07:20:54 +00:00
2020-11-05 19:09:17 +00:00
ColumnUInt8 : : MutablePtr col_null_map_to ;
ColumnUInt8 : : Container * vec_null_map_to [[maybe_unused]] = nullptr ;
2020-12-05 13:18:56 +00:00
if constexpr ( std : : is_same_v < Additions , AccurateOrNullConvertStrategyAdditions > )
2020-11-05 19:09:17 +00:00
{
2021-05-03 19:56:40 +00:00
col_null_map_to = ColumnUInt8 : : create ( input_rows_count , false ) ;
2020-11-05 19:09:17 +00:00
vec_null_map_to = & col_null_map_to - > getData ( ) ;
}
2021-05-03 19:56:40 +00:00
for ( size_t i = 0 ; i < input_rows_count ; + + i )
2018-08-21 18:25:38 +00:00
{
2021-05-03 15:41:37 +00:00
if constexpr ( std : : is_same_v < FromDataType , DataTypeUUID > ! = std : : is_same_v < ToDataType , DataTypeUUID > )
2021-01-04 22:31:34 +00:00
{
throw Exception ( " Conversion between numeric types and UUID is not supported " , ErrorCodes : : NOT_IMPLEMENTED ) ;
}
2020-12-05 14:55:37 +00:00
else
2020-08-19 11:52:17 +00:00
{
2020-12-05 13:18:56 +00:00
if constexpr ( IsDataTypeDecimal < FromDataType > | | IsDataTypeDecimal < ToDataType > )
2020-11-12 11:27:02 +00:00
{
2020-12-15 18:54:16 +00:00
if constexpr ( std : : is_same_v < Additions , AccurateOrNullConvertStrategyAdditions > )
{
ToFieldType result ;
bool convert_result = false ;
if constexpr ( IsDataTypeDecimal < FromDataType > & & IsDataTypeDecimal < ToDataType > )
2020-12-15 20:36:45 +00:00
convert_result = tryConvertDecimals < FromDataType , ToDataType > ( vec_from [ i ] , vec_from . getScale ( ) , vec_to . getScale ( ) , result ) ;
2020-12-15 18:54:16 +00:00
else if constexpr ( IsDataTypeDecimal < FromDataType > & & IsDataTypeNumber < ToDataType > )
2020-12-15 20:36:45 +00:00
convert_result = tryConvertFromDecimal < FromDataType , ToDataType > ( vec_from [ i ] , vec_from . getScale ( ) , result ) ;
2020-12-15 18:54:16 +00:00
else if constexpr ( IsDataTypeNumber < FromDataType > & & IsDataTypeDecimal < ToDataType > )
2020-12-15 20:36:45 +00:00
convert_result = tryConvertToDecimal < FromDataType , ToDataType > ( vec_from [ i ] , vec_to . getScale ( ) , result ) ;
2020-12-15 18:54:16 +00:00
if ( convert_result )
vec_to [ i ] = result ;
else
2021-01-22 13:24:51 +00:00
{
vec_to [ i ] = static_cast < ToFieldType > ( 0 ) ;
2020-12-15 18:54:16 +00:00
( * vec_null_map_to ) [ i ] = true ;
2021-01-22 13:24:51 +00:00
}
2020-12-15 18:54:16 +00:00
}
else
2020-11-12 11:27:02 +00:00
{
2020-12-05 13:18:56 +00:00
if constexpr ( IsDataTypeDecimal < FromDataType > & & IsDataTypeDecimal < ToDataType > )
vec_to [ i ] = convertDecimals < FromDataType , ToDataType > ( vec_from [ i ] , vec_from . getScale ( ) , vec_to . getScale ( ) ) ;
else if constexpr ( IsDataTypeDecimal < FromDataType > & & IsDataTypeNumber < ToDataType > )
vec_to [ i ] = convertFromDecimal < FromDataType , ToDataType > ( vec_from [ i ] , vec_from . getScale ( ) ) ;
else if constexpr ( IsDataTypeNumber < FromDataType > & & IsDataTypeDecimal < ToDataType > )
vec_to [ i ] = convertToDecimal < FromDataType , ToDataType > ( vec_from [ i ] , vec_to . getScale ( ) ) ;
else
throw Exception ( " Unsupported data type in conversion function " , ErrorCodes : : CANNOT_CONVERT_TYPE ) ;
2020-11-12 11:27:02 +00:00
}
2020-11-05 19:09:17 +00:00
}
2020-08-19 11:52:17 +00:00
else
2020-11-10 13:18:58 +00:00
{
2020-12-08 07:43:18 +00:00
/// If From Data is Nan or Inf and we convert to integer type, throw exception
if constexpr ( std : : is_floating_point_v < FromFieldType > & & ! std : : is_floating_point_v < ToFieldType > )
2020-11-10 13:18:58 +00:00
{
2020-12-05 13:18:56 +00:00
if ( ! isFinite ( vec_from [ i ] ) )
2020-11-10 13:18:58 +00:00
{
2020-12-05 13:18:56 +00:00
if constexpr ( std : : is_same_v < Additions , AccurateOrNullConvertStrategyAdditions > )
{
2021-01-22 13:24:51 +00:00
vec_to [ i ] = 0 ;
2020-12-08 10:54:33 +00:00
( * vec_null_map_to ) [ i ] = true ;
2020-12-05 13:18:56 +00:00
continue ;
}
else
2020-12-08 07:43:18 +00:00
throw Exception ( " Unexpected inf or nan to integer conversion " , ErrorCodes : : CANNOT_CONVERT_TYPE ) ;
2020-11-10 13:18:58 +00:00
}
}
2020-11-12 09:28:49 +00:00
2020-12-05 14:55:37 +00:00
if constexpr ( std : : is_same_v < Additions , AccurateOrNullConvertStrategyAdditions >
2020-12-05 13:18:56 +00:00
| | std : : is_same_v < Additions , AccurateConvertStrategyAdditions > )
2020-11-10 13:18:58 +00:00
{
2020-12-05 13:18:56 +00:00
bool convert_result = accurate : : convertNumeric ( vec_from [ i ] , vec_to [ i ] ) ;
2020-11-12 09:28:49 +00:00
2020-12-05 13:18:56 +00:00
if ( ! convert_result )
{
if ( std : : is_same_v < Additions , AccurateOrNullConvertStrategyAdditions > )
{
2021-01-22 13:24:51 +00:00
vec_to [ i ] = 0 ;
2020-12-05 13:18:56 +00:00
( * vec_null_map_to ) [ i ] = true ;
}
else
{
throw Exception (
" Value in column " + named_from . column - > getName ( ) + " cannot be safely converted into type "
+ result_type - > getName ( ) ,
ErrorCodes : : CANNOT_CONVERT_TYPE ) ;
}
}
}
else
2020-11-05 19:09:17 +00:00
{
2020-12-05 13:18:56 +00:00
vec_to [ i ] = static_cast < ToFieldType > ( vec_from [ i ] ) ;
2020-11-05 19:09:17 +00:00
}
}
2020-08-19 11:52:17 +00:00
}
2018-08-21 18:25:38 +00:00
}
2017-12-16 04:59:32 +00:00
2020-12-05 13:18:56 +00:00
if constexpr ( std : : is_same_v < Additions , AccurateOrNullConvertStrategyAdditions > )
2020-11-05 19:09:17 +00:00
return ColumnNullable : : create ( std : : move ( col_to ) , std : : move ( col_null_map_to ) ) ;
else
return col_to ;
2017-04-01 07:20:54 +00:00
}
else
2018-08-21 18:25:38 +00:00
throw Exception ( " Illegal column " + named_from . column - > getName ( ) + " of first argument of function " + Name : : name ,
2017-04-01 07:20:54 +00:00
ErrorCodes : : ILLEGAL_COLUMN ) ;
}
2011-10-16 01:57:10 +00:00
} ;
2019-10-30 13:59:38 +00:00
/** Conversion of DateTime to Date: throw off time component.
*/
2020-12-23 21:57:33 +00:00
template < typename Name > struct ConvertImpl < DataTypeDateTime , DataTypeDate , Name , ConvertDefaultBehaviorTag >
2019-10-30 13:59:38 +00:00
: DateTimeTransformImpl < DataTypeDateTime , DataTypeDate , ToDateImpl > { } ;
2021-07-15 11:40:45 +00:00
/** Conversion of DateTime to Date32: throw off time component.
*/
template < typename Name > struct ConvertImpl < DataTypeDateTime , DataTypeDate32 , Name , ConvertDefaultBehaviorTag >
: DateTimeTransformImpl < DataTypeDateTime , DataTypeDate32 , ToDate32Impl > { } ;
2011-10-15 23:40:56 +00:00
2016-07-30 04:39:51 +00:00
/** Conversion of Date to DateTime: adding 00:00:00 time component.
2011-10-16 01:57:10 +00:00
*/
2017-01-22 08:33:16 +00:00
struct ToDateTimeImpl
2011-10-16 01:57:10 +00:00
{
2017-10-29 04:18:48 +00:00
static constexpr auto name = " toDateTime " ;
2017-04-01 07:20:54 +00:00
static inline UInt32 execute ( UInt16 d , const DateLUTImpl & time_zone )
{
2018-05-25 13:29:15 +00:00
return time_zone . fromDayNum ( DayNum ( d ) ) ;
2017-04-01 07:20:54 +00:00
}
2019-10-30 13:59:38 +00:00
2021-07-15 11:40:45 +00:00
static inline UInt32 execute ( Int32 d , const DateLUTImpl & time_zone )
{
return time_zone . fromDayNum ( ExtendedDayNum ( d ) ) ;
}
2020-04-17 13:26:44 +00:00
static inline UInt32 execute ( UInt32 dt , const DateLUTImpl & /*time_zone*/ )
2019-10-30 13:59:38 +00:00
{
2020-04-17 13:26:44 +00:00
return dt ;
}
// TODO: return UInt32 ???
static inline Int64 execute ( Int64 dt64 , const DateLUTImpl & /*time_zone*/ )
{
return dt64 ;
2019-10-30 13:59:38 +00:00
}
2011-10-16 01:57:10 +00:00
} ;
2020-12-23 21:57:33 +00:00
template < typename Name > struct ConvertImpl < DataTypeDate , DataTypeDateTime , Name , ConvertDefaultBehaviorTag >
2019-09-26 15:12:40 +00:00
: DateTimeTransformImpl < DataTypeDate , DataTypeDateTime , ToDateTimeImpl > { } ;
2015-03-16 04:44:53 +00:00
2021-07-15 11:40:45 +00:00
template < typename Name > struct ConvertImpl < DataTypeDate32 , DataTypeDateTime , Name , ConvertDefaultBehaviorTag >
: DateTimeTransformImpl < DataTypeDate32 , DataTypeDateTime , ToDateTimeImpl > { } ;
2017-01-22 08:33:16 +00:00
/// Implementation of toDate function.
2015-03-16 04:44:53 +00:00
2015-10-22 15:31:42 +00:00
template < typename FromType , typename ToType >
struct ToDateTransform32Or64
{
2017-10-29 04:18:48 +00:00
static constexpr auto name = " toDate " ;
2018-12-27 00:40:10 +00:00
static inline NO_SANITIZE_UNDEFINED ToType execute ( const FromType & from , const DateLUTImpl & time_zone )
2017-04-01 07:20:54 +00:00
{
2020-04-17 13:26:44 +00:00
// since converting to Date, no need in values outside of default LUT range.
2021-07-15 11:40:45 +00:00
return ( from < DATE_LUT_MAX_DAY_NUM )
2020-08-07 19:52:21 +00:00
? from
: time_zone . toDayNum ( std : : min ( time_t ( from ) , time_t ( 0xFFFFFFFF ) ) ) ;
2017-04-01 07:20:54 +00:00
}
2015-10-22 15:31:42 +00:00
} ;
2020-08-07 17:38:42 +00:00
template < typename FromType , typename ToType >
struct ToDateTransform32Or64Signed
{
static constexpr auto name = " toDate " ;
static inline NO_SANITIZE_UNDEFINED ToType execute ( const FromType & from , const DateLUTImpl & time_zone )
{
2020-04-17 13:26:44 +00:00
// TODO: decide narrow or extended range based on FromType
2020-08-07 19:52:21 +00:00
/// The function should be monotonic (better for query optimizations), so we saturate instead of overflow.
if ( from < 0 )
return 0 ;
2021-07-15 11:40:45 +00:00
return ( from < DATE_LUT_MAX_DAY_NUM )
2020-08-07 19:52:21 +00:00
? from
: time_zone . toDayNum ( std : : min ( time_t ( from ) , time_t ( 0xFFFFFFFF ) ) ) ;
2020-08-07 17:38:42 +00:00
}
} ;
template < typename FromType , typename ToType >
struct ToDateTransform8Or16Signed
{
static constexpr auto name = " toDate " ;
static inline NO_SANITIZE_UNDEFINED ToType execute ( const FromType & from , const DateLUTImpl & )
{
2020-08-07 19:52:21 +00:00
if ( from < 0 )
return 0 ;
2020-08-07 17:38:42 +00:00
return from ;
}
} ;
2021-07-15 11:40:45 +00:00
/// Implementation of toDate32 function.
template < typename FromType , typename ToType >
struct ToDate32Transform32Or64
{
static constexpr auto name = " toDate32 " ;
static inline NO_SANITIZE_UNDEFINED ToType execute ( const FromType & from , const DateLUTImpl & time_zone )
{
return ( from < DATE_LUT_MAX_EXTEND_DAY_NUM )
? from
: time_zone . toDayNum ( std : : min ( time_t ( from ) , time_t ( 0xFFFFFFFF ) ) ) ;
}
} ;
template < typename FromType , typename ToType >
struct ToDate32Transform32Or64Signed
{
static constexpr auto name = " toDate32 " ;
static inline NO_SANITIZE_UNDEFINED ToType execute ( const FromType & from , const DateLUTImpl & time_zone )
{
static const Int32 daynum_min_offset = - static_cast < Int32 > ( DateLUT : : instance ( ) . getDayNumOffsetEpoch ( ) ) ;
if ( from < daynum_min_offset )
return daynum_min_offset ;
return ( from < DATE_LUT_MAX_EXTEND_DAY_NUM )
? from
: time_zone . toDayNum ( std : : min ( time_t ( from ) , time_t ( 0xFFFFFFFF ) ) ) ;
}
} ;
template < typename FromType , typename ToType >
struct ToDate32Transform8Or16Signed
{
static constexpr auto name = " toDate32 " ;
static inline NO_SANITIZE_UNDEFINED ToType execute ( const FromType & from , const DateLUTImpl & )
{
return from ;
}
} ;
2020-08-07 17:38:42 +00:00
/** Special case of converting Int8, Int16, (U)Int32 or (U)Int64 (and also, for convenience,
* Float32 , Float64 ) to Date . If the number is negative , saturate it to unix epoch time . If the
* number is less than 65536 , then it is treated as DayNum , and if it ' s greater or equals to 65536 ,
* then treated as unix timestamp . If the number exceeds UInt32 , saturate to MAX_UINT32 then as DayNum .
2016-07-30 04:39:51 +00:00
* It ' s a bit illogical , as we actually have two functions in one .
* But allows to support frequent case ,
* when user write toDate ( UInt32 ) , expecting conversion of unix timestamp to Date .
* ( otherwise such usage would be frequent mistake ) .
2015-10-22 15:31:42 +00:00
*/
2020-12-23 21:57:33 +00:00
template < typename Name > struct ConvertImpl < DataTypeUInt32 , DataTypeDate , Name , ConvertDefaultBehaviorTag >
2019-09-26 15:12:40 +00:00
: DateTimeTransformImpl < DataTypeUInt32 , DataTypeDate , ToDateTransform32Or64 < UInt32 , UInt16 > > { } ;
2020-12-23 21:57:33 +00:00
template < typename Name > struct ConvertImpl < DataTypeUInt64 , DataTypeDate , Name , ConvertDefaultBehaviorTag >
2019-09-26 15:12:40 +00:00
: DateTimeTransformImpl < DataTypeUInt64 , DataTypeDate , ToDateTransform32Or64 < UInt64 , UInt16 > > { } ;
2020-12-23 21:57:33 +00:00
template < typename Name > struct ConvertImpl < DataTypeInt8 , DataTypeDate , Name , ConvertDefaultBehaviorTag >
2020-08-07 17:38:42 +00:00
: DateTimeTransformImpl < DataTypeInt8 , DataTypeDate , ToDateTransform8Or16Signed < Int8 , UInt16 > > { } ;
2020-12-23 21:57:33 +00:00
template < typename Name > struct ConvertImpl < DataTypeInt16 , DataTypeDate , Name , ConvertDefaultBehaviorTag >
2020-08-07 17:38:42 +00:00
: DateTimeTransformImpl < DataTypeInt16 , DataTypeDate , ToDateTransform8Or16Signed < Int16 , UInt16 > > { } ;
2020-12-23 21:57:33 +00:00
template < typename Name > struct ConvertImpl < DataTypeInt32 , DataTypeDate , Name , ConvertDefaultBehaviorTag >
2020-08-07 17:38:42 +00:00
: DateTimeTransformImpl < DataTypeInt32 , DataTypeDate , ToDateTransform32Or64Signed < Int32 , UInt16 > > { } ;
2020-12-23 21:57:33 +00:00
template < typename Name > struct ConvertImpl < DataTypeInt64 , DataTypeDate , Name , ConvertDefaultBehaviorTag >
2020-08-07 17:38:42 +00:00
: DateTimeTransformImpl < DataTypeInt64 , DataTypeDate , ToDateTransform32Or64Signed < Int64 , UInt16 > > { } ;
2020-12-23 21:57:33 +00:00
template < typename Name > struct ConvertImpl < DataTypeFloat32 , DataTypeDate , Name , ConvertDefaultBehaviorTag >
2020-08-07 17:38:42 +00:00
: DateTimeTransformImpl < DataTypeFloat32 , DataTypeDate , ToDateTransform32Or64Signed < Float32 , UInt16 > > { } ;
2020-12-23 21:57:33 +00:00
template < typename Name > struct ConvertImpl < DataTypeFloat64 , DataTypeDate , Name , ConvertDefaultBehaviorTag >
2020-08-07 17:38:42 +00:00
: DateTimeTransformImpl < DataTypeFloat64 , DataTypeDate , ToDateTransform32Or64Signed < Float64 , UInt16 > > { } ;
2021-07-15 11:40:45 +00:00
template < typename Name > struct ConvertImpl < DataTypeUInt32 , DataTypeDate32 , Name , ConvertDefaultBehaviorTag >
: DateTimeTransformImpl < DataTypeUInt32 , DataTypeDate32 , ToDate32Transform32Or64 < UInt32 , Int32 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeUInt64 , DataTypeDate32 , Name , ConvertDefaultBehaviorTag >
: DateTimeTransformImpl < DataTypeUInt64 , DataTypeDate32 , ToDate32Transform32Or64 < UInt64 , Int32 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeInt8 , DataTypeDate32 , Name , ConvertDefaultBehaviorTag >
: DateTimeTransformImpl < DataTypeInt8 , DataTypeDate32 , ToDate32Transform8Or16Signed < Int8 , Int32 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeInt16 , DataTypeDate32 , Name , ConvertDefaultBehaviorTag >
: DateTimeTransformImpl < DataTypeInt16 , DataTypeDate32 , ToDate32Transform8Or16Signed < Int16 , Int32 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeInt32 , DataTypeDate32 , Name , ConvertDefaultBehaviorTag >
: DateTimeTransformImpl < DataTypeInt32 , DataTypeDate32 , ToDate32Transform32Or64Signed < Int32 , Int32 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeInt64 , DataTypeDate32 , Name , ConvertDefaultBehaviorTag >
: DateTimeTransformImpl < DataTypeInt64 , DataTypeDate32 , ToDate32Transform32Or64Signed < Int64 , Int32 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeFloat32 , DataTypeDate32 , Name , ConvertDefaultBehaviorTag >
: DateTimeTransformImpl < DataTypeFloat32 , DataTypeDate32 , ToDate32Transform32Or64Signed < Float32 , Int32 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeFloat64 , DataTypeDate32 , Name , ConvertDefaultBehaviorTag >
: DateTimeTransformImpl < DataTypeFloat64 , DataTypeDate32 , ToDate32Transform32Or64Signed < Float64 , Int32 > > { } ;
2020-08-07 17:38:42 +00:00
template < typename FromType , typename ToType >
struct ToDateTimeTransform64
{
static constexpr auto name = " toDateTime " ;
static inline NO_SANITIZE_UNDEFINED ToType execute ( const FromType & from , const DateLUTImpl & )
{
return std : : min ( time_t ( from ) , time_t ( 0xFFFFFFFF ) ) ;
}
} ;
template < typename FromType , typename ToType >
struct ToDateTimeTransformSigned
{
static constexpr auto name = " toDateTime " ;
static inline NO_SANITIZE_UNDEFINED ToType execute ( const FromType & from , const DateLUTImpl & )
{
2020-08-07 19:53:52 +00:00
if ( from < 0 )
return 0 ;
2020-08-07 17:38:42 +00:00
return from ;
}
} ;
template < typename FromType , typename ToType >
struct ToDateTimeTransform64Signed
{
static constexpr auto name = " toDateTime " ;
static inline NO_SANITIZE_UNDEFINED ToType execute ( const FromType & from , const DateLUTImpl & )
{
2020-08-07 19:53:52 +00:00
if ( from < 0 )
return 0 ;
2020-08-07 17:38:42 +00:00
return std : : min ( time_t ( from ) , time_t ( 0xFFFFFFFF ) ) ;
}
} ;
/** Special case of converting Int8, Int16, Int32 or (U)Int64 (and also, for convenience, Float32,
* Float64 ) to DateTime . If the number is negative , saturate it to unix epoch time . If the number
* exceeds UInt32 , saturate to MAX_UINT32 .
*/
template < typename Name > struct ConvertImpl < DataTypeInt8 , DataTypeDateTime , Name >
: DateTimeTransformImpl < DataTypeInt8 , DataTypeDateTime , ToDateTimeTransformSigned < Int8 , UInt32 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeInt16 , DataTypeDateTime , Name >
: DateTimeTransformImpl < DataTypeInt16 , DataTypeDateTime , ToDateTimeTransformSigned < Int16 , UInt32 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeInt32 , DataTypeDateTime , Name >
: DateTimeTransformImpl < DataTypeInt32 , DataTypeDateTime , ToDateTimeTransformSigned < Int32 , UInt32 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeInt64 , DataTypeDateTime , Name >
: DateTimeTransformImpl < DataTypeInt64 , DataTypeDateTime , ToDateTimeTransform64Signed < Int64 , UInt32 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeUInt64 , DataTypeDateTime , Name >
: DateTimeTransformImpl < DataTypeUInt64 , DataTypeDateTime , ToDateTimeTransform64 < UInt64 , UInt32 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeFloat32 , DataTypeDateTime , Name >
: DateTimeTransformImpl < DataTypeFloat32 , DataTypeDateTime , ToDateTimeTransform64Signed < Float32 , UInt32 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeFloat64 , DataTypeDateTime , Name >
: DateTimeTransformImpl < DataTypeFloat64 , DataTypeDateTime , ToDateTimeTransform64Signed < Float64 , UInt32 > > { } ;
2015-03-16 04:44:53 +00:00
2020-04-17 13:26:44 +00:00
const time_t LUT_MIN_TIME = - 1420070400l ; // 1925-01-01 UTC
const time_t LUT_MAX_TIME = 9877248000l ; // 2282-12-31 UTC
2019-10-30 13:59:38 +00:00
Fix DateTime64 initialization (to match DateTime behaviour)
There was no specializations for toDateTime64(<numeric>), and because of
this default decimal conversion was used, however this is not enough for
DateTime/DateTime64 types, since the date may overflow and the proper
check is required (like DateTime has), and this what UBsan found [1]:
../src/IO/WriteHelpers.h:812:33: runtime error: index 508 out of bounds for type 'const char [201]' Received signal -3 Received signal Unknown signal (-3)
Backtrace:
(gdb) bt
0 LocalDateTime::LocalDateTime (this=0x7fffffff8418, year_=1970, month_=1 '\001', day_=1 '\001', hour_=2 '\002', minute_=0 '\000', second_=254 '\376') at LocalDateTime.h:83
1 0x00000000138a5edb in DB::writeDateTimeText<(char)45, (char)58, (char)32, (char)46> (datetime64=..., scale=7, buf=..., date_lut=...) at WriteHelpers.h:852
2 0x0000000019c379b4 in DB::DataTypeDateTime64::serializeText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at DataTypeDateTime64.cpp:66
3 0x0000000019d297e4 in DB::IDataType::serializeAsText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at IDataType.cpp:387
[1]: https://clickhouse-test-reports.s3.yandex.net/19527/cea8ae162ffbf92e5ed29304ab010704c5d611c8/fuzzer_ubsan/report.html#fail1
Also fix CAST for DateTime64
2021-02-05 19:06:23 +00:00
/** Conversion of numeric to DateTime64
*/
template < typename FromType >
struct ToDateTime64TransformUnsigned
{
static constexpr auto name = " toDateTime64 " ;
const DateTime64 : : NativeType scale_multiplier = 1 ;
ToDateTime64TransformUnsigned ( UInt32 scale = 0 )
: scale_multiplier ( DecimalUtils : : scaleMultiplier < DateTime64 : : NativeType > ( scale ) )
{ }
2021-02-05 19:06:23 +00:00
inline NO_SANITIZE_UNDEFINED DateTime64 : : NativeType execute ( FromType from , const DateLUTImpl & ) const
Fix DateTime64 initialization (to match DateTime behaviour)
There was no specializations for toDateTime64(<numeric>), and because of
this default decimal conversion was used, however this is not enough for
DateTime/DateTime64 types, since the date may overflow and the proper
check is required (like DateTime has), and this what UBsan found [1]:
../src/IO/WriteHelpers.h:812:33: runtime error: index 508 out of bounds for type 'const char [201]' Received signal -3 Received signal Unknown signal (-3)
Backtrace:
(gdb) bt
0 LocalDateTime::LocalDateTime (this=0x7fffffff8418, year_=1970, month_=1 '\001', day_=1 '\001', hour_=2 '\002', minute_=0 '\000', second_=254 '\376') at LocalDateTime.h:83
1 0x00000000138a5edb in DB::writeDateTimeText<(char)45, (char)58, (char)32, (char)46> (datetime64=..., scale=7, buf=..., date_lut=...) at WriteHelpers.h:852
2 0x0000000019c379b4 in DB::DataTypeDateTime64::serializeText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at DataTypeDateTime64.cpp:66
3 0x0000000019d297e4 in DB::IDataType::serializeAsText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at IDataType.cpp:387
[1]: https://clickhouse-test-reports.s3.yandex.net/19527/cea8ae162ffbf92e5ed29304ab010704c5d611c8/fuzzer_ubsan/report.html#fail1
Also fix CAST for DateTime64
2021-02-05 19:06:23 +00:00
{
2020-04-17 13:26:44 +00:00
from = std : : min < time_t > ( from , LUT_MAX_TIME ) ;
Fix DateTime64 initialization (to match DateTime behaviour)
There was no specializations for toDateTime64(<numeric>), and because of
this default decimal conversion was used, however this is not enough for
DateTime/DateTime64 types, since the date may overflow and the proper
check is required (like DateTime has), and this what UBsan found [1]:
../src/IO/WriteHelpers.h:812:33: runtime error: index 508 out of bounds for type 'const char [201]' Received signal -3 Received signal Unknown signal (-3)
Backtrace:
(gdb) bt
0 LocalDateTime::LocalDateTime (this=0x7fffffff8418, year_=1970, month_=1 '\001', day_=1 '\001', hour_=2 '\002', minute_=0 '\000', second_=254 '\376') at LocalDateTime.h:83
1 0x00000000138a5edb in DB::writeDateTimeText<(char)45, (char)58, (char)32, (char)46> (datetime64=..., scale=7, buf=..., date_lut=...) at WriteHelpers.h:852
2 0x0000000019c379b4 in DB::DataTypeDateTime64::serializeText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at DataTypeDateTime64.cpp:66
3 0x0000000019d297e4 in DB::IDataType::serializeAsText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at IDataType.cpp:387
[1]: https://clickhouse-test-reports.s3.yandex.net/19527/cea8ae162ffbf92e5ed29304ab010704c5d611c8/fuzzer_ubsan/report.html#fail1
Also fix CAST for DateTime64
2021-02-05 19:06:23 +00:00
return DecimalUtils : : decimalFromComponentsWithMultiplier < DateTime64 > ( from , 0 , scale_multiplier ) ;
}
} ;
template < typename FromType >
struct ToDateTime64TransformSigned
{
static constexpr auto name = " toDateTime64 " ;
const DateTime64 : : NativeType scale_multiplier = 1 ;
ToDateTime64TransformSigned ( UInt32 scale = 0 )
: scale_multiplier ( DecimalUtils : : scaleMultiplier < DateTime64 : : NativeType > ( scale ) )
{ }
2021-02-05 19:06:23 +00:00
inline NO_SANITIZE_UNDEFINED DateTime64 : : NativeType execute ( FromType from , const DateLUTImpl & ) const
Fix DateTime64 initialization (to match DateTime behaviour)
There was no specializations for toDateTime64(<numeric>), and because of
this default decimal conversion was used, however this is not enough for
DateTime/DateTime64 types, since the date may overflow and the proper
check is required (like DateTime has), and this what UBsan found [1]:
../src/IO/WriteHelpers.h:812:33: runtime error: index 508 out of bounds for type 'const char [201]' Received signal -3 Received signal Unknown signal (-3)
Backtrace:
(gdb) bt
0 LocalDateTime::LocalDateTime (this=0x7fffffff8418, year_=1970, month_=1 '\001', day_=1 '\001', hour_=2 '\002', minute_=0 '\000', second_=254 '\376') at LocalDateTime.h:83
1 0x00000000138a5edb in DB::writeDateTimeText<(char)45, (char)58, (char)32, (char)46> (datetime64=..., scale=7, buf=..., date_lut=...) at WriteHelpers.h:852
2 0x0000000019c379b4 in DB::DataTypeDateTime64::serializeText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at DataTypeDateTime64.cpp:66
3 0x0000000019d297e4 in DB::IDataType::serializeAsText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at IDataType.cpp:387
[1]: https://clickhouse-test-reports.s3.yandex.net/19527/cea8ae162ffbf92e5ed29304ab010704c5d611c8/fuzzer_ubsan/report.html#fail1
Also fix CAST for DateTime64
2021-02-05 19:06:23 +00:00
{
2020-04-17 13:26:44 +00:00
from = std : : max < time_t > ( from , LUT_MIN_TIME ) ;
from = std : : min < time_t > ( from , LUT_MAX_TIME ) ;
Fix DateTime64 initialization (to match DateTime behaviour)
There was no specializations for toDateTime64(<numeric>), and because of
this default decimal conversion was used, however this is not enough for
DateTime/DateTime64 types, since the date may overflow and the proper
check is required (like DateTime has), and this what UBsan found [1]:
../src/IO/WriteHelpers.h:812:33: runtime error: index 508 out of bounds for type 'const char [201]' Received signal -3 Received signal Unknown signal (-3)
Backtrace:
(gdb) bt
0 LocalDateTime::LocalDateTime (this=0x7fffffff8418, year_=1970, month_=1 '\001', day_=1 '\001', hour_=2 '\002', minute_=0 '\000', second_=254 '\376') at LocalDateTime.h:83
1 0x00000000138a5edb in DB::writeDateTimeText<(char)45, (char)58, (char)32, (char)46> (datetime64=..., scale=7, buf=..., date_lut=...) at WriteHelpers.h:852
2 0x0000000019c379b4 in DB::DataTypeDateTime64::serializeText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at DataTypeDateTime64.cpp:66
3 0x0000000019d297e4 in DB::IDataType::serializeAsText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at IDataType.cpp:387
[1]: https://clickhouse-test-reports.s3.yandex.net/19527/cea8ae162ffbf92e5ed29304ab010704c5d611c8/fuzzer_ubsan/report.html#fail1
Also fix CAST for DateTime64
2021-02-05 19:06:23 +00:00
return DecimalUtils : : decimalFromComponentsWithMultiplier < DateTime64 > ( from , 0 , scale_multiplier ) ;
}
} ;
2021-02-21 18:09:51 +00:00
template < typename FromDataType , typename FromType >
struct ToDateTime64TransformFloat
{
static constexpr auto name = " toDateTime64 " ;
const UInt32 scale = 1 ;
ToDateTime64TransformFloat ( UInt32 scale_ = 0 )
: scale ( scale_ )
{ }
inline NO_SANITIZE_UNDEFINED DateTime64 : : NativeType execute ( FromType from , const DateLUTImpl & ) const
{
if ( from < 0 )
return 0 ;
from = std : : min < FromType > ( from , FromType ( 0xFFFFFFFF ) ) ;
return convertToDecimal < FromDataType , DataTypeDateTime64 > ( from , scale ) ;
}
} ;
Fix DateTime64 initialization (to match DateTime behaviour)
There was no specializations for toDateTime64(<numeric>), and because of
this default decimal conversion was used, however this is not enough for
DateTime/DateTime64 types, since the date may overflow and the proper
check is required (like DateTime has), and this what UBsan found [1]:
../src/IO/WriteHelpers.h:812:33: runtime error: index 508 out of bounds for type 'const char [201]' Received signal -3 Received signal Unknown signal (-3)
Backtrace:
(gdb) bt
0 LocalDateTime::LocalDateTime (this=0x7fffffff8418, year_=1970, month_=1 '\001', day_=1 '\001', hour_=2 '\002', minute_=0 '\000', second_=254 '\376') at LocalDateTime.h:83
1 0x00000000138a5edb in DB::writeDateTimeText<(char)45, (char)58, (char)32, (char)46> (datetime64=..., scale=7, buf=..., date_lut=...) at WriteHelpers.h:852
2 0x0000000019c379b4 in DB::DataTypeDateTime64::serializeText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at DataTypeDateTime64.cpp:66
3 0x0000000019d297e4 in DB::IDataType::serializeAsText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at IDataType.cpp:387
[1]: https://clickhouse-test-reports.s3.yandex.net/19527/cea8ae162ffbf92e5ed29304ab010704c5d611c8/fuzzer_ubsan/report.html#fail1
Also fix CAST for DateTime64
2021-02-05 19:06:23 +00:00
template < typename Name > struct ConvertImpl < DataTypeInt8 , DataTypeDateTime64 , Name >
: DateTimeTransformImpl < DataTypeInt8 , DataTypeDateTime64 , ToDateTime64TransformSigned < Int8 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeInt16 , DataTypeDateTime64 , Name >
: DateTimeTransformImpl < DataTypeInt16 , DataTypeDateTime64 , ToDateTime64TransformSigned < Int16 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeInt32 , DataTypeDateTime64 , Name >
: DateTimeTransformImpl < DataTypeInt32 , DataTypeDateTime64 , ToDateTime64TransformSigned < Int32 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeInt64 , DataTypeDateTime64 , Name >
: DateTimeTransformImpl < DataTypeInt64 , DataTypeDateTime64 , ToDateTime64TransformSigned < Int64 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeUInt64 , DataTypeDateTime64 , Name >
: DateTimeTransformImpl < DataTypeUInt64 , DataTypeDateTime64 , ToDateTime64TransformUnsigned < UInt64 > > { } ;
template < typename Name > struct ConvertImpl < DataTypeFloat32 , DataTypeDateTime64 , Name >
2021-02-21 18:09:51 +00:00
: DateTimeTransformImpl < DataTypeFloat32 , DataTypeDateTime64 , ToDateTime64TransformFloat < DataTypeFloat32 , Float32 > > { } ;
Fix DateTime64 initialization (to match DateTime behaviour)
There was no specializations for toDateTime64(<numeric>), and because of
this default decimal conversion was used, however this is not enough for
DateTime/DateTime64 types, since the date may overflow and the proper
check is required (like DateTime has), and this what UBsan found [1]:
../src/IO/WriteHelpers.h:812:33: runtime error: index 508 out of bounds for type 'const char [201]' Received signal -3 Received signal Unknown signal (-3)
Backtrace:
(gdb) bt
0 LocalDateTime::LocalDateTime (this=0x7fffffff8418, year_=1970, month_=1 '\001', day_=1 '\001', hour_=2 '\002', minute_=0 '\000', second_=254 '\376') at LocalDateTime.h:83
1 0x00000000138a5edb in DB::writeDateTimeText<(char)45, (char)58, (char)32, (char)46> (datetime64=..., scale=7, buf=..., date_lut=...) at WriteHelpers.h:852
2 0x0000000019c379b4 in DB::DataTypeDateTime64::serializeText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at DataTypeDateTime64.cpp:66
3 0x0000000019d297e4 in DB::IDataType::serializeAsText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at IDataType.cpp:387
[1]: https://clickhouse-test-reports.s3.yandex.net/19527/cea8ae162ffbf92e5ed29304ab010704c5d611c8/fuzzer_ubsan/report.html#fail1
Also fix CAST for DateTime64
2021-02-05 19:06:23 +00:00
template < typename Name > struct ConvertImpl < DataTypeFloat64 , DataTypeDateTime64 , Name >
2021-02-21 18:09:51 +00:00
: DateTimeTransformImpl < DataTypeFloat64 , DataTypeDateTime64 , ToDateTime64TransformFloat < DataTypeFloat64 , Float64 > > { } ;
Fix DateTime64 initialization (to match DateTime behaviour)
There was no specializations for toDateTime64(<numeric>), and because of
this default decimal conversion was used, however this is not enough for
DateTime/DateTime64 types, since the date may overflow and the proper
check is required (like DateTime has), and this what UBsan found [1]:
../src/IO/WriteHelpers.h:812:33: runtime error: index 508 out of bounds for type 'const char [201]' Received signal -3 Received signal Unknown signal (-3)
Backtrace:
(gdb) bt
0 LocalDateTime::LocalDateTime (this=0x7fffffff8418, year_=1970, month_=1 '\001', day_=1 '\001', hour_=2 '\002', minute_=0 '\000', second_=254 '\376') at LocalDateTime.h:83
1 0x00000000138a5edb in DB::writeDateTimeText<(char)45, (char)58, (char)32, (char)46> (datetime64=..., scale=7, buf=..., date_lut=...) at WriteHelpers.h:852
2 0x0000000019c379b4 in DB::DataTypeDateTime64::serializeText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at DataTypeDateTime64.cpp:66
3 0x0000000019d297e4 in DB::IDataType::serializeAsText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at IDataType.cpp:387
[1]: https://clickhouse-test-reports.s3.yandex.net/19527/cea8ae162ffbf92e5ed29304ab010704c5d611c8/fuzzer_ubsan/report.html#fail1
Also fix CAST for DateTime64
2021-02-05 19:06:23 +00:00
2020-04-17 13:26:44 +00:00
2019-10-31 08:23:55 +00:00
/** Conversion of DateTime64 to Date or DateTime: discards fractional part.
*/
template < typename Transform >
struct FromDateTime64Transform
{
2019-12-11 08:56:32 +00:00
static constexpr auto name = Transform : : name ;
2019-10-31 08:23:55 +00:00
const DateTime64 : : NativeType scale_multiplier = 1 ;
2020-10-19 18:37:44 +00:00
FromDateTime64Transform ( UInt32 scale )
2019-12-06 21:05:31 +00:00
: scale_multiplier ( DecimalUtils : : scaleMultiplier < DateTime64 : : NativeType > ( scale ) )
2019-10-31 08:23:55 +00:00
{ }
inline auto execute ( DateTime64 : : NativeType dt , const DateLUTImpl & time_zone ) const
{
2019-12-06 21:05:31 +00:00
const auto c = DecimalUtils : : splitWithScaleMultiplier ( DateTime64 ( dt ) , scale_multiplier ) ;
2019-10-31 08:23:55 +00:00
return Transform : : execute ( static_cast < UInt32 > ( c . whole ) , time_zone ) ;
}
} ;
2020-04-17 13:26:44 +00:00
/** Conversion of DateTime64 to Date or DateTime: discards fractional part.
*/
2020-12-23 21:57:33 +00:00
template < typename Name > struct ConvertImpl < DataTypeDateTime64 , DataTypeDate , Name , ConvertDefaultBehaviorTag >
2020-04-17 13:26:44 +00:00
: DateTimeTransformImpl < DataTypeDateTime64 , DataTypeDate , TransformDateTime64 < ToDateImpl > > { } ;
2020-12-23 21:57:33 +00:00
template < typename Name > struct ConvertImpl < DataTypeDateTime64 , DataTypeDateTime , Name , ConvertDefaultBehaviorTag >
2020-04-17 13:26:44 +00:00
: DateTimeTransformImpl < DataTypeDateTime64 , DataTypeDateTime , TransformDateTime64 < ToDateTimeImpl > > { } ;
struct ToDateTime64Transform
{
static constexpr auto name = " toDateTime64 " ;
const DateTime64 : : NativeType scale_multiplier = 1 ;
ToDateTime64Transform ( UInt32 scale = 0 )
: scale_multiplier ( DecimalUtils : : scaleMultiplier < DateTime64 : : NativeType > ( scale ) )
{ }
inline DateTime64 : : NativeType execute ( UInt16 d , const DateLUTImpl & time_zone ) const
{
const auto dt = ToDateTimeImpl : : execute ( d , time_zone ) ;
return execute ( dt , time_zone ) ;
}
inline DateTime64 : : NativeType execute ( UInt32 dt , const DateLUTImpl & /*time_zone*/ ) const
{
return DecimalUtils : : decimalFromComponentsWithMultiplier < DateTime64 > ( dt , 0 , scale_multiplier ) ;
}
} ;
/** Conversion of Date or DateTime to DateTime64: add zero sub-second part.
*/
template < typename Name > struct ConvertImpl < DataTypeDate , DataTypeDateTime64 , Name , ConvertDefaultBehaviorTag >
: DateTimeTransformImpl < DataTypeDate , DataTypeDateTime64 , ToDateTime64Transform > { } ;
template < typename Name > struct ConvertImpl < DataTypeDateTime , DataTypeDateTime64 , Name , ConvertDefaultBehaviorTag >
: DateTimeTransformImpl < DataTypeDateTime , DataTypeDateTime64 , ToDateTime64Transform > { } ;
2015-03-16 04:44:53 +00:00
2016-07-30 04:39:51 +00:00
/** Transformation of numbers, dates, datetimes to strings: through formatting.
2011-10-16 01:57:10 +00:00
*/
2016-07-30 04:39:51 +00:00
template < typename DataType >
struct FormatImpl
2015-12-22 13:07:40 +00:00
{
2021-07-12 13:02:15 +00:00
template < typename ReturnType = void >
static ReturnType execute ( const typename DataType : : FieldType x , WriteBuffer & wb , const DataType * , const DateLUTImpl * )
2017-04-01 07:20:54 +00:00
{
writeText ( x , wb ) ;
2021-07-12 13:02:15 +00:00
return ReturnType ( true ) ;
2017-04-01 07:20:54 +00:00
}
2015-12-22 13:07:40 +00:00
} ;
2016-07-30 04:39:51 +00:00
template < >
struct FormatImpl < DataTypeDate >
2015-12-22 13:07:40 +00:00
{
2021-07-12 13:02:15 +00:00
template < typename ReturnType = void >
static ReturnType execute ( const DataTypeDate : : FieldType x , WriteBuffer & wb , const DataTypeDate * , const DateLUTImpl * )
2017-04-01 07:20:54 +00:00
{
2018-05-25 13:29:15 +00:00
writeDateText ( DayNum ( x ) , wb ) ;
2021-07-12 13:02:15 +00:00
return ReturnType ( true ) ;
2017-04-01 07:20:54 +00:00
}
2015-12-22 13:07:40 +00:00
} ;
2021-07-15 11:40:45 +00:00
template < >
struct FormatImpl < DataTypeDate32 >
{
template < typename ReturnType = void >
static ReturnType execute ( const DataTypeDate : : FieldType x , WriteBuffer & wb , const DataTypeDate32 * , const DateLUTImpl * )
{
writeDateText ( ExtendedDayNum ( x ) , wb ) ;
return ReturnType ( true ) ;
}
} ;
2016-07-30 04:39:51 +00:00
template < >
struct FormatImpl < DataTypeDateTime >
2015-12-22 13:07:40 +00:00
{
2021-07-12 13:02:15 +00:00
template < typename ReturnType = void >
static ReturnType execute ( const DataTypeDateTime : : FieldType x , WriteBuffer & wb , const DataTypeDateTime * , const DateLUTImpl * time_zone )
2017-04-01 07:20:54 +00:00
{
writeDateTimeText ( x , wb , * time_zone ) ;
2021-07-12 13:02:15 +00:00
return ReturnType ( true ) ;
2017-04-01 07:20:54 +00:00
}
2015-12-22 13:07:40 +00:00
} ;
2019-05-04 00:07:54 +00:00
template < >
struct FormatImpl < DataTypeDateTime64 >
{
2021-07-12 13:02:15 +00:00
template < typename ReturnType = void >
static ReturnType execute ( const DataTypeDateTime64 : : FieldType x , WriteBuffer & wb , const DataTypeDateTime64 * type , const DateLUTImpl * time_zone )
2019-05-04 00:07:54 +00:00
{
2019-10-30 13:59:38 +00:00
writeDateTimeText ( DateTime64 ( x ) , type - > getScale ( ) , wb , * time_zone ) ;
2021-07-12 13:02:15 +00:00
return ReturnType ( true ) ;
2019-05-04 00:07:54 +00:00
}
} ;
2016-07-30 04:39:51 +00:00
template < typename FieldType >
struct FormatImpl < DataTypeEnum < FieldType > >
2015-12-22 13:07:40 +00:00
{
2021-07-12 13:02:15 +00:00
template < typename ReturnType = void >
static ReturnType execute ( const FieldType x , WriteBuffer & wb , const DataTypeEnum < FieldType > * type , const DateLUTImpl * )
2017-04-01 07:20:54 +00:00
{
2021-07-12 13:02:15 +00:00
static constexpr bool throw_exception = std : : is_same_v < ReturnType , void > ;
2011-10-16 03:05:15 +00:00
2021-07-12 13:02:15 +00:00
if constexpr ( throw_exception )
{
writeString ( type - > getNameForValue ( x ) , wb ) ;
}
else
{
StringRef res ;
bool is_ok = type - > getNameForValue ( x , res ) ;
if ( is_ok )
writeString ( res , wb ) ;
return ReturnType ( is_ok ) ;
}
2017-04-01 07:20:54 +00:00
}
2015-12-22 13:07:40 +00:00
} ;
2011-10-16 03:05:15 +00:00
2018-08-21 18:25:38 +00:00
template < typename FieldType >
struct FormatImpl < DataTypeDecimal < FieldType > >
{
2021-07-12 13:02:15 +00:00
template < typename ReturnType = void >
static ReturnType execute ( const FieldType x , WriteBuffer & wb , const DataTypeDecimal < FieldType > * type , const DateLUTImpl * )
2018-08-21 18:25:38 +00:00
{
2018-08-23 19:11:31 +00:00
writeText ( x , type - > getScale ( ) , wb ) ;
2021-07-12 13:02:15 +00:00
return ReturnType ( true ) ;
2018-08-21 18:25:38 +00:00
}
} ;
2015-12-30 11:53:12 +00:00
/// DataTypeEnum<T> to DataType<T> free conversion
template < typename FieldType , typename Name >
2020-12-23 21:57:33 +00:00
struct ConvertImpl < DataTypeEnum < FieldType > , DataTypeNumber < FieldType > , Name , ConvertDefaultBehaviorTag >
2015-12-30 11:53:12 +00:00
{
2020-11-17 13:24:45 +00:00
static ColumnPtr execute ( const ColumnsWithTypeAndName & arguments , const DataTypePtr & , size_t /*input_rows_count*/ )
2017-04-01 07:20:54 +00:00
{
2020-10-17 14:23:37 +00:00
return arguments [ 0 ] . column ;
2017-04-01 07:20:54 +00:00
}
2015-12-30 11:53:12 +00:00
} ;
2021-07-12 09:18:04 +00:00
static ColumnUInt8 : : MutablePtr copyNullMap ( ColumnPtr col )
{
ColumnUInt8 : : MutablePtr null_map = nullptr ;
if ( const auto * col_null = checkAndGetColumn < ColumnNullable > ( col . get ( ) ) )
{
null_map = ColumnUInt8 : : create ( ) ;
null_map - > insertRangeFrom ( col_null - > getNullMapColumn ( ) , 0 , col_null - > size ( ) ) ;
}
return null_map ;
}
2015-12-30 11:53:12 +00:00
2011-10-16 01:57:10 +00:00
template < typename FromDataType , typename Name >
2020-12-23 21:57:33 +00:00
struct ConvertImpl < FromDataType , std : : enable_if_t < ! std : : is_same_v < FromDataType , DataTypeString > , DataTypeString > , Name , ConvertDefaultBehaviorTag >
2011-10-16 01:57:10 +00:00
{
2017-04-01 07:20:54 +00:00
using FromFieldType = typename FromDataType : : FieldType ;
2018-09-12 17:50:51 +00:00
using ColVecType = std : : conditional_t < IsDecimalNumber < FromFieldType > , ColumnDecimal < FromFieldType > , ColumnVector < FromFieldType > > ;
2017-04-01 07:20:54 +00:00
2020-11-17 13:24:45 +00:00
static ColumnPtr execute ( const ColumnsWithTypeAndName & arguments , const DataTypePtr & , size_t /*input_rows_count*/ )
2017-04-01 07:20:54 +00:00
{
2021-07-12 09:18:04 +00:00
ColumnUInt8 : : MutablePtr null_map = copyNullMap ( arguments [ 0 ] . column ) ;
const auto & col_with_type_and_name = columnGetNested ( arguments [ 0 ] ) ;
2017-04-01 07:20:54 +00:00
const auto & type = static_cast < const FromDataType & > ( * col_with_type_and_name . type ) ;
const DateLUTImpl * time_zone = nullptr ;
/// For argument of DateTime type, second argument with time zone could be specified.
2019-05-04 00:07:54 +00:00
if constexpr ( std : : is_same_v < FromDataType , DataTypeDateTime > | | std : : is_same_v < FromDataType , DataTypeDateTime64 > )
2021-07-12 13:04:59 +00:00
{
auto non_null_args = createBlockWithNestedColumns ( arguments ) ;
time_zone = & extractTimeZoneFromFunctionArguments ( non_null_args , 1 , 0 ) ;
}
2017-04-01 07:20:54 +00:00
2018-09-12 17:50:51 +00:00
if ( const auto col_from = checkAndGetColumn < ColVecType > ( col_with_type_and_name . column . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
2017-12-14 01:43:19 +00:00
auto col_to = ColumnString : : create ( ) ;
2017-04-01 07:20:54 +00:00
2018-09-12 17:50:51 +00:00
const typename ColVecType : : Container & vec_from = col_from - > getData ( ) ;
2018-11-25 00:08:50 +00:00
ColumnString : : Chars & data_to = col_to - > getChars ( ) ;
2017-12-15 21:32:25 +00:00
ColumnString : : Offsets & offsets_to = col_to - > getOffsets ( ) ;
2017-04-01 07:20:54 +00:00
size_t size = vec_from . size ( ) ;
2017-11-17 23:45:49 +00:00
2017-12-25 04:01:46 +00:00
if constexpr ( std : : is_same_v < FromDataType , DataTypeDate > )
2017-11-17 23:45:49 +00:00
data_to . resize ( size * ( strlen ( " YYYY-MM-DD " ) + 1 ) ) ;
2021-07-15 11:41:52 +00:00
else if constexpr ( std : : is_same_v < FromDataType , DataTypeDate32 > )
data_to . resize ( size * ( strlen ( " YYYY-MM-DD " ) + 1 ) ) ;
2017-12-25 04:01:46 +00:00
else if constexpr ( std : : is_same_v < FromDataType , DataTypeDateTime > )
2017-11-17 23:45:49 +00:00
data_to . resize ( size * ( strlen ( " YYYY-MM-DD hh:mm:ss " ) + 1 ) ) ;
2019-05-04 00:07:54 +00:00
else if constexpr ( std : : is_same_v < FromDataType , DataTypeDateTime64 > )
2019-11-20 22:04:18 +00:00
data_to . resize ( size * ( strlen ( " YYYY-MM-DD hh:mm:ss. " ) + vec_from . getScale ( ) + 1 ) ) ;
2017-11-17 23:45:49 +00:00
else
2020-08-08 00:47:03 +00:00
data_to . resize ( size * 3 ) ; /// Arbitrary
2017-11-17 23:45:49 +00:00
2017-04-01 07:20:54 +00:00
offsets_to . resize ( size ) ;
2018-11-25 00:08:50 +00:00
WriteBufferFromVector < ColumnString : : Chars > write_buffer ( data_to ) ;
2017-04-01 07:20:54 +00:00
2021-07-13 06:52:31 +00:00
if ( null_map )
2017-04-01 07:20:54 +00:00
{
2021-07-13 06:52:31 +00:00
for ( size_t i = 0 ; i < size ; + + i )
2021-07-12 09:18:04 +00:00
{
2021-07-12 13:02:15 +00:00
bool is_ok = FormatImpl < FromDataType > : : template execute < bool > ( vec_from [ i ] , write_buffer , & type , time_zone ) ;
2021-07-12 09:55:34 +00:00
null_map - > getData ( ) [ i ] | = ! is_ok ;
2021-07-13 06:52:31 +00:00
writeChar ( 0 , write_buffer ) ;
offsets_to [ i ] = write_buffer . count ( ) ;
2021-07-12 09:18:04 +00:00
}
2021-07-13 06:52:31 +00:00
}
else
{
for ( size_t i = 0 ; i < size ; + + i )
2021-07-12 09:18:04 +00:00
{
2021-07-12 13:02:15 +00:00
FormatImpl < FromDataType > : : template execute < void > ( vec_from [ i ] , write_buffer , & type , time_zone ) ;
2021-07-13 06:52:31 +00:00
writeChar ( 0 , write_buffer ) ;
offsets_to [ i ] = write_buffer . count ( ) ;
2021-07-12 09:18:04 +00:00
}
2017-04-01 07:20:54 +00:00
}
2020-01-10 21:42:26 +00:00
write_buffer . finalize ( ) ;
2021-07-12 09:18:04 +00:00
if ( null_map )
return ColumnNullable : : create ( std : : move ( col_to ) , std : : move ( null_map ) ) ;
2020-10-17 14:23:37 +00:00
return col_to ;
2017-04-01 07:20:54 +00:00
}
else
2020-10-17 14:23:37 +00:00
throw Exception ( " Illegal column " + arguments [ 0 ] . column - > getName ( )
2017-04-01 07:20:54 +00:00
+ " of first argument of function " + Name : : name ,
ErrorCodes : : ILLEGAL_COLUMN ) ;
}
2011-10-16 01:57:10 +00:00
} ;
2016-07-30 04:39:51 +00:00
/// Generic conversion of any type to String.
struct ConvertImplGenericToString
{
2021-07-12 10:15:30 +00:00
static ColumnPtr execute ( const ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type )
2017-04-01 07:20:54 +00:00
{
2021-07-12 10:15:30 +00:00
ColumnUInt8 : : MutablePtr null_map = copyNullMap ( arguments [ 0 ] . column ) ;
const auto & col_with_type_and_name = columnGetNested ( arguments [ 0 ] ) ;
2017-04-01 07:20:54 +00:00
const IDataType & type = * col_with_type_and_name . type ;
const IColumn & col_from = * col_with_type_and_name . column ;
size_t size = col_from . size ( ) ;
2017-12-14 01:43:19 +00:00
auto col_to = ColumnString : : create ( ) ;
2017-04-01 07:20:54 +00:00
2018-11-25 00:08:50 +00:00
ColumnString : : Chars & data_to = col_to - > getChars ( ) ;
2017-12-15 21:32:25 +00:00
ColumnString : : Offsets & offsets_to = col_to - > getOffsets ( ) ;
2017-04-01 07:20:54 +00:00
2018-10-13 14:33:43 +00:00
data_to . resize ( size * 2 ) ; /// Using coefficient 2 for initial size is arbitrary.
2017-12-09 10:14:45 +00:00
offsets_to . resize ( size ) ;
2017-04-01 07:20:54 +00:00
2018-11-25 00:08:50 +00:00
WriteBufferFromVector < ColumnString : : Chars > write_buffer ( data_to ) ;
2017-04-01 07:20:54 +00:00
2018-06-08 01:51:55 +00:00
FormatSettings format_settings ;
2021-03-13 18:05:18 +00:00
auto serialization = type . getDefaultSerialization ( ) ;
2017-12-09 10:14:45 +00:00
for ( size_t i = 0 ; i < size ; + + i )
2017-04-01 07:20:54 +00:00
{
2021-03-13 18:05:18 +00:00
serialization - > serializeText ( col_from , i , write_buffer , format_settings ) ;
2017-12-09 10:14:45 +00:00
writeChar ( 0 , write_buffer ) ;
offsets_to [ i ] = write_buffer . count ( ) ;
2017-04-01 07:20:54 +00:00
}
2017-12-09 10:14:45 +00:00
2020-01-10 21:42:26 +00:00
write_buffer . finalize ( ) ;
2021-07-12 10:15:30 +00:00
if ( result_type - > isNullable ( ) & & null_map )
return ColumnNullable : : create ( std : : move ( col_to ) , std : : move ( null_map ) ) ;
2020-10-17 14:23:37 +00:00
return col_to ;
2017-04-01 07:20:54 +00:00
}
2016-07-30 04:39:51 +00:00
} ;
2017-01-22 08:33:16 +00:00
/** Conversion of strings to numbers, dates, datetimes: through parsing.
2015-07-03 11:55:51 +00:00
*/
2018-07-11 21:43:09 +00:00
template < typename DataType >
void parseImpl ( typename DataType : : FieldType & x , ReadBuffer & rb , const DateLUTImpl * )
2015-07-01 16:07:29 +00:00
{
2017-04-01 07:20:54 +00:00
readText ( x , rb ) ;
2015-07-01 16:07:29 +00:00
}
2018-07-11 21:43:09 +00:00
template < >
inline void parseImpl < DataTypeDate > ( DataTypeDate : : FieldType & x , ReadBuffer & rb , const DateLUTImpl * )
2011-10-16 03:05:15 +00:00
{
2018-05-25 13:29:15 +00:00
DayNum tmp ( 0 ) ;
2017-04-01 07:20:54 +00:00
readDateText ( tmp , rb ) ;
x = tmp ;
2011-10-16 03:05:15 +00:00
}
2021-07-15 11:41:52 +00:00
template < >
inline void parseImpl < DataTypeDate32 > ( DataTypeDate32 : : FieldType & x , ReadBuffer & rb , const DateLUTImpl * )
{
ExtendedDayNum tmp ( 0 ) ;
readDateText ( tmp , rb ) ;
x = tmp ;
}
2020-04-17 13:26:44 +00:00
// NOTE: no need of extra overload of DateTime64, since readDateTimeText64 has different signature and that case is explicitly handled in the calling code.
2018-07-11 21:43:09 +00:00
template < >
inline void parseImpl < DataTypeDateTime > ( DataTypeDateTime : : FieldType & x , ReadBuffer & rb , const DateLUTImpl * time_zone )
2011-10-16 03:05:15 +00:00
{
2021-03-15 20:40:33 +00:00
time_t time = 0 ;
readDateTimeText ( time , rb , * time_zone ) ;
if ( time < 0 )
time = 0 ;
x = time ;
2011-10-16 03:05:15 +00:00
}
2020-04-17 13:26:44 +00:00
2018-07-16 00:28:44 +00:00
template < >
inline void parseImpl < DataTypeUUID > ( DataTypeUUID : : FieldType & x , ReadBuffer & rb , const DateLUTImpl * )
2017-07-06 14:42:27 +00:00
{
UUID tmp ;
2020-10-25 08:45:29 +00:00
readUUIDText ( tmp , rb ) ;
2021-04-25 09:30:43 +00:00
x = tmp . toUnderType ( ) ;
2017-07-06 14:42:27 +00:00
}
2016-08-02 05:32:09 +00:00
2017-12-25 02:57:54 +00:00
template < typename DataType >
2018-07-11 21:43:09 +00:00
bool tryParseImpl ( typename DataType : : FieldType & x , ReadBuffer & rb , const DateLUTImpl * )
2017-12-25 02:57:54 +00:00
{
2019-06-16 18:12:14 +00:00
if constexpr ( std : : is_floating_point_v < typename DataType : : FieldType > )
2017-12-25 02:57:54 +00:00
return tryReadFloatText ( x , rb ) ;
2020-08-19 11:52:17 +00:00
else /*if constexpr (is_integer_v<typename DataType::FieldType>)*/
2019-06-16 18:12:14 +00:00
return tryReadIntText ( x , rb ) ;
2018-07-11 21:43:09 +00:00
}
template < >
inline bool tryParseImpl < DataTypeDate > ( DataTypeDate : : FieldType & x , ReadBuffer & rb , const DateLUTImpl * )
{
DayNum tmp ( 0 ) ;
2018-07-14 23:39:00 +00:00
if ( ! tryReadDateText ( tmp , rb ) )
return false ;
2018-07-11 21:43:09 +00:00
x = tmp ;
return true ;
}
2021-07-15 11:41:52 +00:00
template < >
inline bool tryParseImpl < DataTypeDate32 > ( DataTypeDate32 : : FieldType & x , ReadBuffer & rb , const DateLUTImpl * )
{
ExtendedDayNum tmp ( 0 ) ;
if ( ! tryReadDateText ( tmp , rb ) )
2021-07-15 11:40:45 +00:00
{
2021-07-15 11:41:52 +00:00
return false ;
2021-07-15 11:40:45 +00:00
}
2021-07-15 11:41:52 +00:00
x = tmp ;
return true ;
}
2018-07-11 21:43:09 +00:00
template < >
inline bool tryParseImpl < DataTypeDateTime > ( DataTypeDateTime : : FieldType & x , ReadBuffer & rb , const DateLUTImpl * time_zone )
{
time_t tmp = 0 ;
2018-07-14 23:39:00 +00:00
if ( ! tryReadDateTimeText ( tmp , rb , * time_zone ) )
return false ;
2018-07-11 21:43:09 +00:00
x = tmp ;
return true ;
2017-12-25 02:57:54 +00:00
}
2020-10-25 08:45:29 +00:00
template < >
inline bool tryParseImpl < DataTypeUUID > ( DataTypeUUID : : FieldType & x , ReadBuffer & rb , const DateLUTImpl * )
{
UUID tmp ;
if ( ! tryReadUUIDText ( tmp , rb ) )
return false ;
2021-04-25 09:30:43 +00:00
x = tmp . toUnderType ( ) ;
2020-10-25 08:45:29 +00:00
return true ;
}
2017-12-25 02:57:54 +00:00
2016-08-17 07:40:06 +00:00
/** Throw exception with verbose message when string value is not parsed completely.
*/
2020-10-17 14:23:37 +00:00
[[noreturn]] inline void throwExceptionForIncompletelyParsedValue ( ReadBuffer & read_buffer , const DataTypePtr result_type )
2019-12-25 19:17:41 +00:00
{
2020-10-17 14:23:37 +00:00
const IDataType & to_type = * result_type ;
2019-12-25 19:17:41 +00:00
WriteBufferFromOwnString message_buf ;
message_buf < < " Cannot parse string " < < quote < < String ( read_buffer . buffer ( ) . begin ( ) , read_buffer . buffer ( ) . size ( ) )
< < " as " < < to_type . getName ( )
< < " : syntax error " ;
if ( read_buffer . offset ( ) )
message_buf < < " at position " < < read_buffer . offset ( )
< < " (parsed just " < < quote < < String ( read_buffer . buffer ( ) . begin ( ) , read_buffer . offset ( ) ) < < " ) " ;
else
message_buf < < " at begin of string " ;
if ( isNativeNumber ( to_type ) )
message_buf < < " . Note: there are to " < < to_type . getName ( ) < < " OrZero and to " < < to_type . getName ( ) < < " OrNull functions, which returns zero/NULL instead of throwing exception. " ;
throw Exception ( message_buf . str ( ) , ErrorCodes : : CANNOT_PARSE_TEXT ) ;
}
2016-08-17 07:40:06 +00:00
2017-12-25 02:57:54 +00:00
enum class ConvertFromStringExceptionMode
{
Throw , /// Throw exception if value cannot be parsed.
Zero , /// Fill with zero or default if value cannot be parsed.
Null /// Return ColumnNullable with NULLs when value cannot be parsed.
} ;
2018-02-11 23:57:07 +00:00
enum class ConvertFromStringParsingMode
{
Normal ,
2020-07-05 18:09:20 +00:00
BestEffort , /// Only applicable for DateTime. Will use sophisticated method, that is slower.
BestEffortUS
2018-02-11 23:57:07 +00:00
} ;
template < typename FromDataType , typename ToDataType , typename Name ,
ConvertFromStringExceptionMode exception_mode , ConvertFromStringParsingMode parsing_mode >
2017-12-25 02:57:54 +00:00
struct ConvertThroughParsing
2011-10-16 01:57:10 +00:00
{
2017-12-25 02:57:54 +00:00
static_assert ( std : : is_same_v < FromDataType , DataTypeString > | | std : : is_same_v < FromDataType , DataTypeFixedString > ,
2017-12-25 04:10:43 +00:00
" ConvertThroughParsing is only applicable for String or FixedString data types " ) ;
2017-12-25 02:57:54 +00:00
2019-11-20 22:05:09 +00:00
static constexpr bool to_datetime64 = std : : is_same_v < ToDataType , DataTypeDateTime64 > ;
2020-10-17 14:23:37 +00:00
// using ToFieldType = typename ToDataType::FieldType;
2017-04-01 07:20:54 +00:00
2017-12-25 02:57:54 +00:00
static bool isAllRead ( ReadBuffer & in )
{
/// In case of FixedString, skip zero bytes at end.
if constexpr ( std : : is_same_v < FromDataType , DataTypeFixedString > )
while ( ! in . eof ( ) & & * in . position ( ) = = 0 )
+ + in . position ( ) ;
if ( in . eof ( ) )
return true ;
/// Special case, that allows to parse string with DateTime as Date.
2017-12-25 04:01:46 +00:00
if ( std : : is_same_v < ToDataType , DataTypeDate > & & ( in . buffer ( ) . size ( ) ) = = strlen ( " YYYY-MM-DD hh:mm:ss " ) )
2017-12-25 02:57:54 +00:00
return true ;
return false ;
}
2018-08-31 08:59:21 +00:00
template < typename Additions = void * >
2020-11-17 13:24:45 +00:00
static ColumnPtr execute ( const ColumnsWithTypeAndName & arguments , const DataTypePtr & res_type , size_t input_rows_count ,
2018-08-31 08:59:21 +00:00
Additions additions [[maybe_unused]] = Additions ( ) )
2017-04-01 07:20:54 +00:00
{
2019-10-24 10:44:38 +00:00
using ColVecTo = typename ToDataType : : ColumnType ;
2018-08-27 16:16:16 +00:00
2018-02-11 23:57:07 +00:00
const DateLUTImpl * local_time_zone [[maybe_unused]] = nullptr ;
const DateLUTImpl * utc_time_zone [[maybe_unused]] = nullptr ;
2017-04-01 07:20:54 +00:00
/// For conversion to DateTime type, second argument with time zone could be specified.
2019-11-20 22:05:09 +00:00
if constexpr ( std : : is_same_v < ToDataType , DataTypeDateTime > | | to_datetime64 )
2018-02-11 23:57:07 +00:00
{
2020-10-17 14:23:37 +00:00
const auto result_type = removeNullable ( res_type ) ;
2020-08-08 00:47:03 +00:00
// Time zone is already figured out during result type resolution, no need to do it here.
2019-11-05 11:59:46 +00:00
if ( const auto dt_col = checkAndGetDataType < ToDataType > ( result_type . get ( ) ) )
2019-11-03 05:10:11 +00:00
local_time_zone = & dt_col - > getTimeZone ( ) ;
else
{
2020-10-17 14:23:37 +00:00
local_time_zone = & extractTimeZoneFromFunctionArguments ( arguments , 1 , 0 ) ;
2019-11-03 05:10:11 +00:00
}
2018-02-11 23:57:07 +00:00
2020-07-05 18:09:20 +00:00
if constexpr ( parsing_mode = = ConvertFromStringParsingMode : : BestEffort | | parsing_mode = = ConvertFromStringParsingMode : : BestEffortUS )
2018-02-11 23:57:07 +00:00
utc_time_zone = & DateLUT : : instance ( " UTC " ) ;
}
2017-04-01 07:20:54 +00:00
2020-10-17 14:23:37 +00:00
const IColumn * col_from = arguments [ 0 ] . column . get ( ) ;
2017-12-25 02:57:54 +00:00
const ColumnString * col_from_string = checkAndGetColumn < ColumnString > ( col_from ) ;
const ColumnFixedString * col_from_fixed_string = checkAndGetColumn < ColumnFixedString > ( col_from ) ;
if ( std : : is_same_v < FromDataType , DataTypeString > & & ! col_from_string )
throw Exception ( " Illegal column " + col_from - > getName ( )
+ " of first argument of function " + Name : : name ,
ErrorCodes : : ILLEGAL_COLUMN ) ;
if ( std : : is_same_v < FromDataType , DataTypeFixedString > & & ! col_from_fixed_string )
throw Exception ( " Illegal column " + col_from - > getName ( )
+ " of first argument of function " + Name : : name ,
ErrorCodes : : ILLEGAL_COLUMN ) ;
2018-04-24 07:16:39 +00:00
size_t size = input_rows_count ;
2018-08-27 16:16:16 +00:00
typename ColVecTo : : MutablePtr col_to = nullptr ;
2017-12-25 02:57:54 +00:00
2018-09-10 13:52:18 +00:00
if constexpr ( IsDataTypeDecimal < ToDataType > )
2018-08-21 18:25:38 +00:00
{
2018-08-31 08:59:21 +00:00
UInt32 scale = additions ;
2019-11-20 22:05:09 +00:00
if constexpr ( to_datetime64 )
2019-10-12 07:03:38 +00:00
{
ToDataType check_bounds_in_ctor ( scale , local_time_zone ? local_time_zone - > getTimeZone ( ) : String { } ) ;
}
else
{
ToDataType check_bounds_in_ctor ( ToDataType : : maxPrecision ( ) , scale ) ;
}
2018-08-31 08:59:21 +00:00
col_to = ColVecTo : : create ( size , scale ) ;
2018-08-21 18:25:38 +00:00
}
2018-08-27 16:16:16 +00:00
else
col_to = ColVecTo : : create ( size ) ;
typename ColVecTo : : Container & vec_to = col_to - > getData ( ) ;
2017-12-25 02:57:54 +00:00
ColumnUInt8 : : MutablePtr col_null_map_to ;
2017-12-25 07:18:27 +00:00
ColumnUInt8 : : Container * vec_null_map_to [[maybe_unused]] = nullptr ;
2018-02-11 23:57:07 +00:00
if constexpr ( exception_mode = = ConvertFromStringExceptionMode : : Null )
2017-04-01 07:20:54 +00:00
{
2017-12-25 02:57:54 +00:00
col_null_map_to = ColumnUInt8 : : create ( size ) ;
vec_null_map_to = & col_null_map_to - > getData ( ) ;
}
2017-04-01 07:20:54 +00:00
2018-11-25 00:08:50 +00:00
const ColumnString : : Chars * chars = nullptr ;
2017-12-25 02:57:54 +00:00
const IColumn : : Offsets * offsets = nullptr ;
size_t fixed_string_size = 0 ;
if constexpr ( std : : is_same_v < FromDataType , DataTypeString > )
{
chars = & col_from_string - > getChars ( ) ;
offsets = & col_from_string - > getOffsets ( ) ;
}
else
{
chars = & col_from_fixed_string - > getChars ( ) ;
fixed_string_size = col_from_fixed_string - > getN ( ) ;
}
2017-04-01 07:20:54 +00:00
2017-12-25 02:57:54 +00:00
size_t current_offset = 0 ;
2017-04-01 07:20:54 +00:00
2017-12-25 02:57:54 +00:00
for ( size_t i = 0 ; i < size ; + + i )
{
size_t next_offset = std : : is_same_v < FromDataType , DataTypeString > ? ( * offsets ) [ i ] : ( current_offset + fixed_string_size ) ;
size_t string_size = std : : is_same_v < FromDataType , DataTypeString > ? next_offset - current_offset - 1 : fixed_string_size ;
2017-04-01 07:20:54 +00:00
2017-12-25 02:57:54 +00:00
ReadBufferFromMemory read_buffer ( & ( * chars ) [ current_offset ] , string_size ) ;
2017-04-01 07:20:54 +00:00
2019-05-15 18:50:35 +00:00
if constexpr ( exception_mode = = ConvertFromStringExceptionMode : : Throw )
2017-12-25 02:57:54 +00:00
{
2018-02-11 23:57:07 +00:00
if constexpr ( parsing_mode = = ConvertFromStringParsingMode : : BestEffort )
{
2019-11-20 22:05:09 +00:00
if constexpr ( to_datetime64 )
2019-10-12 07:03:38 +00:00
{
2019-11-11 05:18:34 +00:00
DateTime64 res = 0 ;
2019-10-12 07:03:38 +00:00
parseDateTime64BestEffort ( res , vec_to . getScale ( ) , read_buffer , * local_time_zone , * utc_time_zone ) ;
vec_to [ i ] = res ;
}
else
{
time_t res ;
parseDateTimeBestEffort ( res , read_buffer , * local_time_zone , * utc_time_zone ) ;
vec_to [ i ] = res ;
}
2018-02-11 23:57:07 +00:00
}
2020-07-05 18:09:20 +00:00
else if constexpr ( parsing_mode = = ConvertFromStringParsingMode : : BestEffortUS )
{
2020-04-17 13:26:44 +00:00
if constexpr ( to_datetime64 )
{
DateTime64 res = 0 ;
parseDateTime64BestEffortUS ( res , vec_to . getScale ( ) , read_buffer , * local_time_zone , * utc_time_zone ) ;
vec_to [ i ] = res ;
}
else
{
time_t res ;
parseDateTimeBestEffortUS ( res , read_buffer , * local_time_zone , * utc_time_zone ) ;
vec_to [ i ] = res ;
}
2020-07-05 18:09:20 +00:00
}
2018-02-11 23:57:07 +00:00
else
{
2019-11-20 22:05:09 +00:00
if constexpr ( to_datetime64 )
2019-10-12 07:03:38 +00:00
{
2019-11-11 05:18:34 +00:00
DateTime64 value = 0 ;
2019-10-14 08:38:03 +00:00
readDateTime64Text ( value , vec_to . getScale ( ) , read_buffer , * local_time_zone ) ;
vec_to [ i ] = value ;
2019-10-12 07:03:38 +00:00
}
else if constexpr ( IsDataTypeDecimal < ToDataType > )
2021-06-17 22:42:33 +00:00
SerializationDecimal < typename ToDataType : : FieldType > : : readText (
vec_to [ i ] , read_buffer , ToDataType : : maxPrecision ( ) , vec_to . getScale ( ) ) ;
2019-05-15 18:50:35 +00:00
else
2021-07-15 11:40:45 +00:00
{
2019-05-15 18:50:35 +00:00
parseImpl < ToDataType > ( vec_to [ i ] , read_buffer , local_time_zone ) ;
2021-07-15 11:40:45 +00:00
}
2018-02-11 23:57:07 +00:00
}
2017-04-01 07:20:54 +00:00
2017-12-25 02:57:54 +00:00
if ( ! isAllRead ( read_buffer ) )
2020-10-17 14:23:37 +00:00
throwExceptionForIncompletelyParsedValue ( read_buffer , res_type ) ;
2017-12-25 02:57:54 +00:00
}
else
{
2018-02-11 23:57:07 +00:00
bool parsed ;
if constexpr ( parsing_mode = = ConvertFromStringParsingMode : : BestEffort )
{
2019-11-20 22:05:09 +00:00
if constexpr ( to_datetime64 )
2019-10-12 07:03:38 +00:00
{
2019-11-11 05:18:34 +00:00
DateTime64 res = 0 ;
2019-10-12 07:03:38 +00:00
parsed = tryParseDateTime64BestEffort ( res , vec_to . getScale ( ) , read_buffer , * local_time_zone , * utc_time_zone ) ;
vec_to [ i ] = res ;
}
else
{
time_t res ;
parsed = tryParseDateTimeBestEffort ( res , read_buffer , * local_time_zone , * utc_time_zone ) ;
vec_to [ i ] = res ;
}
2018-02-11 23:57:07 +00:00
}
2021-01-27 14:10:06 +00:00
else if constexpr ( parsing_mode = = ConvertFromStringParsingMode : : BestEffortUS )
{
time_t res ;
parsed = tryParseDateTimeBestEffortUS ( res , read_buffer , * local_time_zone , * utc_time_zone ) ;
vec_to [ i ] = res ;
}
2018-02-11 23:57:07 +00:00
else
{
2019-11-20 22:05:09 +00:00
if constexpr ( to_datetime64 )
2019-10-12 07:03:38 +00:00
{
2019-11-11 05:18:34 +00:00
DateTime64 value = 0 ;
2019-10-12 07:03:38 +00:00
parsed = tryReadDateTime64Text ( value , vec_to . getScale ( ) , read_buffer , * local_time_zone ) ;
vec_to [ i ] = value ;
}
else if constexpr ( IsDataTypeDecimal < ToDataType > )
2021-06-17 22:42:33 +00:00
parsed = SerializationDecimal < typename ToDataType : : FieldType > : : tryReadText (
vec_to [ i ] , read_buffer , ToDataType : : maxPrecision ( ) , vec_to . getScale ( ) ) ;
2019-05-15 18:50:35 +00:00
else
parsed = tryParseImpl < ToDataType > ( vec_to [ i ] , read_buffer , local_time_zone ) ;
2018-02-11 23:57:07 +00:00
}
2017-12-25 05:22:33 +00:00
2021-06-17 22:42:33 +00:00
if ( ! isAllRead ( read_buffer ) )
parsed = false ;
2020-06-14 01:45:03 +00:00
2017-12-25 05:22:33 +00:00
if ( ! parsed )
2021-07-15 11:40:45 +00:00
{
if constexpr ( std : : is_same_v < ToDataType , DataTypeDate32 > )
{
vec_to [ i ] = - static_cast < Int32 > ( DateLUT : : instance ( ) . getDayNumOffsetEpoch ( ) ) ;
}
else
{
vec_to [ i ] = static_cast < typename ToDataType : : FieldType > ( 0 ) ;
}
}
2017-12-25 05:22:33 +00:00
2018-02-11 23:57:07 +00:00
if constexpr ( exception_mode = = ConvertFromStringExceptionMode : : Null )
2017-12-25 05:22:33 +00:00
( * vec_null_map_to ) [ i ] = ! parsed ;
2017-04-01 07:20:54 +00:00
}
2017-12-16 04:59:32 +00:00
2017-12-25 02:57:54 +00:00
current_offset = next_offset ;
2017-04-01 07:20:54 +00:00
}
2017-12-25 02:57:54 +00:00
2018-02-11 23:57:07 +00:00
if constexpr ( exception_mode = = ConvertFromStringExceptionMode : : Null )
2020-10-17 14:23:37 +00:00
return ColumnNullable : : create ( std : : move ( col_to ) , std : : move ( col_null_map_to ) ) ;
2017-04-01 07:20:54 +00:00
else
2020-10-20 15:56:05 +00:00
return col_to ;
2017-04-01 07:20:54 +00:00
}
2016-08-02 05:32:09 +00:00
} ;
2017-12-25 02:57:54 +00:00
template < typename ToDataType , typename Name >
2020-12-23 21:57:33 +00:00
struct ConvertImpl < std : : enable_if_t < ! std : : is_same_v < ToDataType , DataTypeString > , DataTypeString > , ToDataType , Name , ConvertDefaultBehaviorTag >
2018-02-11 23:57:07 +00:00
: ConvertThroughParsing < DataTypeString , ToDataType , Name , ConvertFromStringExceptionMode : : Throw , ConvertFromStringParsingMode : : Normal > { } ;
2017-12-25 02:57:54 +00:00
template < typename ToDataType , typename Name >
2020-12-23 21:57:33 +00:00
struct ConvertImpl < std : : enable_if_t < ! std : : is_same_v < ToDataType , DataTypeFixedString > , DataTypeFixedString > , ToDataType , Name , ConvertDefaultBehaviorTag >
2018-02-11 23:57:07 +00:00
: ConvertThroughParsing < DataTypeFixedString , ToDataType , Name , ConvertFromStringExceptionMode : : Throw , ConvertFromStringParsingMode : : Normal > { } ;
2011-10-16 01:57:10 +00:00
2020-12-23 21:57:33 +00:00
template < typename ToDataType , typename Name >
struct ConvertImpl < std : : enable_if_t < ! std : : is_same_v < ToDataType , DataTypeString > , DataTypeString > , ToDataType , Name , ConvertReturnNullOnErrorTag >
: ConvertThroughParsing < DataTypeString , ToDataType , Name , ConvertFromStringExceptionMode : : Null , ConvertFromStringParsingMode : : Normal > { } ;
template < typename ToDataType , typename Name >
struct ConvertImpl < std : : enable_if_t < ! std : : is_same_v < ToDataType , DataTypeFixedString > , DataTypeFixedString > , ToDataType , Name , ConvertReturnNullOnErrorTag >
: ConvertThroughParsing < DataTypeFixedString , ToDataType , Name , ConvertFromStringExceptionMode : : Null , ConvertFromStringParsingMode : : Normal > { } ;
2016-08-02 03:30:34 +00:00
/// Generic conversion of any type from String. Used for complex types: Array and Tuple.
struct ConvertImplGenericFromString
{
2020-10-17 14:23:37 +00:00
static ColumnPtr execute ( ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type )
2017-04-01 07:20:54 +00:00
{
2020-10-17 14:23:37 +00:00
const IColumn & col_from = * arguments [ 0 ] . column ;
2017-04-01 07:20:54 +00:00
size_t size = col_from . size ( ) ;
2016-08-02 03:30:34 +00:00
2020-10-17 14:23:37 +00:00
const IDataType & data_type_to = * result_type ;
2016-08-02 03:30:34 +00:00
2017-07-21 06:35:58 +00:00
if ( const ColumnString * col_from_string = checkAndGetColumn < ColumnString > ( & col_from ) )
2017-04-01 07:20:54 +00:00
{
2017-12-16 04:59:32 +00:00
auto res = data_type_to . createColumn ( ) ;
2016-08-02 03:30:34 +00:00
2017-12-16 04:59:32 +00:00
IColumn & column_to = * res ;
2017-04-01 07:20:54 +00:00
column_to . reserve ( size ) ;
2016-08-02 03:30:34 +00:00
2018-11-25 00:08:50 +00:00
const ColumnString : : Chars & chars = col_from_string - > getChars ( ) ;
2017-12-15 21:32:25 +00:00
const IColumn : : Offsets & offsets = col_from_string - > getOffsets ( ) ;
2016-08-02 03:30:34 +00:00
2017-04-01 07:20:54 +00:00
size_t current_offset = 0 ;
2016-08-02 03:30:34 +00:00
2018-06-08 01:51:55 +00:00
FormatSettings format_settings ;
2021-03-13 18:05:18 +00:00
auto serialization = data_type_to . getDefaultSerialization ( ) ;
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < size ; + + i )
{
ReadBufferFromMemory read_buffer ( & chars [ current_offset ] , offsets [ i ] - current_offset - 1 ) ;
2016-08-02 03:30:34 +00:00
2021-03-13 18:05:18 +00:00
serialization - > deserializeWholeText ( column_to , read_buffer , format_settings ) ;
2016-08-02 03:30:34 +00:00
2017-04-01 07:20:54 +00:00
if ( ! read_buffer . eof ( ) )
2020-10-17 14:23:37 +00:00
throwExceptionForIncompletelyParsedValue ( read_buffer , result_type ) ;
2016-08-02 03:30:34 +00:00
2017-04-01 07:20:54 +00:00
current_offset = offsets [ i ] ;
}
2017-12-16 04:59:32 +00:00
2020-10-17 14:23:37 +00:00
return res ;
2017-04-01 07:20:54 +00:00
}
else
2020-10-17 14:23:37 +00:00
throw Exception ( " Illegal column " + arguments [ 0 ] . column - > getName ( )
2017-04-01 07:20:54 +00:00
+ " of first argument of conversion function from string " ,
ErrorCodes : : ILLEGAL_COLUMN ) ;
}
2016-08-02 03:30:34 +00:00
} ;
2016-07-31 03:53:16 +00:00
template < >
2020-12-23 21:57:33 +00:00
struct ConvertImpl < DataTypeString , DataTypeUInt32 , NameToUnixTimestamp , ConvertDefaultBehaviorTag >
: ConvertImpl < DataTypeString , DataTypeDateTime , NameToUnixTimestamp , ConvertDefaultBehaviorTag > { } ;
2015-07-01 17:32:04 +00:00
2020-12-24 00:13:53 +00:00
template < >
struct ConvertImpl < DataTypeString , DataTypeUInt32 , NameToUnixTimestamp , ConvertReturnNullOnErrorTag >
: ConvertImpl < DataTypeString , DataTypeDateTime , NameToUnixTimestamp , ConvertReturnNullOnErrorTag > { } ;
2015-07-03 00:09:57 +00:00
2016-07-31 03:53:16 +00:00
/** If types are identical, just take reference to column.
2011-10-16 03:05:15 +00:00
*/
2017-10-30 02:18:06 +00:00
template < typename T , typename Name >
2020-12-23 21:57:33 +00:00
struct ConvertImpl < std : : enable_if_t < ! T : : is_parametric , T > , T , Name , ConvertDefaultBehaviorTag >
2011-10-16 01:57:10 +00:00
{
2020-11-05 19:09:17 +00:00
template < typename Additions = void * >
static ColumnPtr execute ( const ColumnsWithTypeAndName & arguments , const DataTypePtr & , size_t /*input_rows_count*/ ,
Additions additions [[maybe_unused]] = Additions ( ) )
2017-04-01 07:20:54 +00:00
{
2020-10-17 14:23:37 +00:00
return arguments [ 0 ] . column ;
2017-04-01 07:20:54 +00:00
}
2011-10-16 01:57:10 +00:00
} ;
2016-07-31 03:53:16 +00:00
/** Conversion from FixedString to String.
* Cutting sequences of zero bytes from end of strings .
2014-04-13 13:58:54 +00:00
*/
template < typename Name >
2020-12-23 21:57:33 +00:00
struct ConvertImpl < DataTypeFixedString , DataTypeString , Name , ConvertDefaultBehaviorTag >
2014-04-13 13:58:54 +00:00
{
2020-11-17 13:24:45 +00:00
static ColumnPtr execute ( const ColumnsWithTypeAndName & arguments , const DataTypePtr & , size_t /*input_rows_count*/ )
2017-04-01 07:20:54 +00:00
{
2020-10-17 14:23:37 +00:00
if ( const ColumnFixedString * col_from = checkAndGetColumn < ColumnFixedString > ( arguments [ 0 ] . column . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
2017-12-14 01:43:19 +00:00
auto col_to = ColumnString : : create ( ) ;
2017-04-01 07:20:54 +00:00
2018-11-25 00:08:50 +00:00
const ColumnFixedString : : Chars & data_from = col_from - > getChars ( ) ;
ColumnString : : Chars & data_to = col_to - > getChars ( ) ;
2017-12-15 21:32:25 +00:00
ColumnString : : Offsets & offsets_to = col_to - > getOffsets ( ) ;
2017-04-01 07:20:54 +00:00
size_t size = col_from - > size ( ) ;
size_t n = col_from - > getN ( ) ;
2017-10-30 02:18:06 +00:00
data_to . resize ( size * ( n + 1 ) ) ; /// + 1 - zero terminator
2017-04-01 07:20:54 +00:00
offsets_to . resize ( size ) ;
size_t offset_from = 0 ;
size_t offset_to = 0 ;
for ( size_t i = 0 ; i < size ; + + i )
{
size_t bytes_to_copy = n ;
while ( bytes_to_copy > 0 & & data_from [ offset_from + bytes_to_copy - 1 ] = = 0 )
- - bytes_to_copy ;
memcpy ( & data_to [ offset_to ] , & data_from [ offset_from ] , bytes_to_copy ) ;
offset_from + = n ;
offset_to + = bytes_to_copy ;
data_to [ offset_to ] = 0 ;
+ + offset_to ;
offsets_to [ i ] = offset_to ;
}
data_to . resize ( offset_to ) ;
2020-10-17 14:23:37 +00:00
return col_to ;
2017-04-01 07:20:54 +00:00
}
else
2020-10-17 14:23:37 +00:00
throw Exception ( " Illegal column " + arguments [ 0 ] . column - > getName ( )
2017-04-01 07:20:54 +00:00
+ " of first argument of function " + Name : : name ,
ErrorCodes : : ILLEGAL_COLUMN ) ;
}
2014-04-13 13:58:54 +00:00
} ;
2016-07-31 03:53:16 +00:00
/// Declared early because used below.
2017-10-30 02:18:06 +00:00
struct NameToDate { static constexpr auto name = " toDate " ; } ;
2021-07-15 11:41:52 +00:00
struct NameToDate32 { static constexpr auto name = " toDate32 " ; } ;
2017-11-05 05:32:22 +00:00
struct NameToDateTime { static constexpr auto name = " toDateTime " ; } ;
2020-08-15 13:29:24 +00:00
struct NameToDateTime32 { static constexpr auto name = " toDateTime32 " ; } ;
2019-10-12 07:03:38 +00:00
struct NameToDateTime64 { static constexpr auto name = " toDateTime64 " ; } ;
2017-10-30 02:18:06 +00:00
struct NameToString { static constexpr auto name = " toString " ; } ;
2018-08-22 16:16:39 +00:00
struct NameToDecimal32 { static constexpr auto name = " toDecimal32 " ; } ;
struct NameToDecimal64 { static constexpr auto name = " toDecimal64 " ; } ;
struct NameToDecimal128 { static constexpr auto name = " toDecimal128 " ; } ;
2020-08-19 11:52:17 +00:00
struct NameToDecimal256 { static constexpr auto name = " toDecimal256 " ; } ;
2017-10-30 02:18:06 +00:00
# define DEFINE_NAME_TO_INTERVAL(INTERVAL_KIND) \
struct NameToInterval # # INTERVAL_KIND \
{ \
static constexpr auto name = " toInterval " # INTERVAL_KIND ; \
2019-11-16 13:54:52 +00:00
static constexpr auto kind = IntervalKind : : INTERVAL_KIND ; \
2017-10-30 02:18:06 +00:00
} ;
DEFINE_NAME_TO_INTERVAL ( Second )
DEFINE_NAME_TO_INTERVAL ( Minute )
DEFINE_NAME_TO_INTERVAL ( Hour )
DEFINE_NAME_TO_INTERVAL ( Day )
DEFINE_NAME_TO_INTERVAL ( Week )
DEFINE_NAME_TO_INTERVAL ( Month )
2018-12-18 13:16:48 +00:00
DEFINE_NAME_TO_INTERVAL ( Quarter )
2017-10-30 02:18:06 +00:00
DEFINE_NAME_TO_INTERVAL ( Year )
# undef DEFINE_NAME_TO_INTERVAL
2011-10-16 01:57:10 +00:00
2020-08-20 14:41:03 +00:00
struct NameParseDateTimeBestEffort ;
struct NameParseDateTimeBestEffortOrZero ;
struct NameParseDateTimeBestEffortOrNull ;
2020-08-20 11:18:29 +00:00
template < typename Name , typename ToDataType >
2020-10-17 14:23:37 +00:00
static inline bool isDateTime64 ( const ColumnsWithTypeAndName & arguments )
2020-08-20 11:18:29 +00:00
{
if constexpr ( std : : is_same_v < ToDataType , DataTypeDateTime64 > )
return true ;
2020-08-20 14:41:03 +00:00
else if constexpr ( std : : is_same_v < Name , NameToDateTime > | | std : : is_same_v < Name , NameParseDateTimeBestEffort >
| | std : : is_same_v < Name , NameParseDateTimeBestEffortOrZero > | | std : : is_same_v < Name , NameParseDateTimeBestEffortOrNull > )
2020-08-20 11:18:29 +00:00
{
2020-10-17 14:23:37 +00:00
return ( arguments . size ( ) = = 2 & & isUnsignedInteger ( arguments [ 1 ] . type ) ) | | arguments . size ( ) = = 3 ;
2020-08-20 11:18:29 +00:00
}
return false ;
}
2016-07-30 04:39:51 +00:00
2015-12-29 13:51:12 +00:00
template < typename ToDataType , typename Name , typename MonotonicityImpl >
2011-10-16 01:57:10 +00:00
class FunctionConvert : public IFunction
{
2011-10-15 23:40:56 +00:00
public :
2017-04-01 07:20:54 +00:00
using Monotonic = MonotonicityImpl ;
static constexpr auto name = Name : : name ;
2018-08-21 18:25:38 +00:00
static constexpr bool to_decimal =
2020-08-19 11:52:17 +00:00
std : : is_same_v < Name , NameToDecimal32 > | | std : : is_same_v < Name , NameToDecimal64 >
| | std : : is_same_v < Name , NameToDecimal128 > | | std : : is_same_v < Name , NameToDecimal256 > ;
2018-08-21 18:25:38 +00:00
2019-11-20 22:05:09 +00:00
static constexpr bool to_datetime64 = std : : is_same_v < ToDataType , DataTypeDateTime64 > ;
2020-12-23 21:57:33 +00:00
static constexpr bool to_string_or_fixed_string = std : : is_same_v < ToDataType , DataTypeFixedString > | |
std : : is_same_v < ToDataType , DataTypeString > ;
2021-04-10 23:33:54 +00:00
static FunctionPtr create ( ContextPtr ) { return std : : make_shared < FunctionConvert > ( ) ; }
2019-12-25 19:17:41 +00:00
static FunctionPtr create ( ) { return std : : make_shared < FunctionConvert > ( ) ; }
2017-04-01 07:20:54 +00:00
String getName ( ) const override
{
return name ;
}
bool isVariadic ( ) const override { return true ; }
size_t getNumberOfArguments ( ) const override { return 0 ; }
2020-10-09 07:41:28 +00:00
bool isInjective ( const ColumnsWithTypeAndName & ) const override { return std : : is_same_v < Name , NameToString > ; }
2021-04-29 14:48:26 +00:00
bool isSuitableForShortCircuitArgumentsExecution ( ) const override { return false ; }
2017-04-01 07:20:54 +00:00
2021-05-16 15:17:05 +00:00
using DefaultReturnTypeGetter = std : : function < DataTypePtr ( const ColumnsWithTypeAndName & ) > ;
static DataTypePtr getReturnTypeDefaultImplementationForNulls ( const ColumnsWithTypeAndName & arguments , const DefaultReturnTypeGetter & getter )
2020-12-23 21:57:33 +00:00
{
2021-05-15 17:33:15 +00:00
NullPresence null_presence = getNullPresense ( arguments ) ;
if ( null_presence . has_null_constant )
{
2021-05-16 15:17:05 +00:00
return makeNullable ( std : : make_shared < DataTypeNothing > ( ) ) ;
2021-05-15 17:33:15 +00:00
}
if ( null_presence . has_nullable )
{
auto nested_columns = Block ( createBlockWithNestedColumns ( arguments ) ) ;
2021-05-16 15:17:05 +00:00
auto return_type = getter ( ColumnsWithTypeAndName ( nested_columns . begin ( ) , nested_columns . end ( ) ) ) ;
return makeNullable ( return_type ) ;
2021-05-15 17:33:15 +00:00
}
2021-05-16 15:17:05 +00:00
return getter ( arguments ) ;
}
2021-05-15 17:33:15 +00:00
2021-05-16 15:17:05 +00:00
DataTypePtr getReturnTypeImpl ( const ColumnsWithTypeAndName & arguments ) const override
{
auto getter = [ & ] ( const auto & args ) { return getReturnTypeImplRemovedNullable ( args ) ; } ;
auto res = getReturnTypeDefaultImplementationForNulls ( arguments , getter ) ;
2020-12-23 21:57:33 +00:00
to_nullable = res - > isNullable ( ) ;
2020-12-24 00:13:53 +00:00
checked_return_type = true ;
2020-12-23 21:57:33 +00:00
return res ;
}
DataTypePtr getReturnTypeImplRemovedNullable ( const ColumnsWithTypeAndName & arguments ) const
2017-04-01 07:20:54 +00:00
{
2019-12-25 14:50:13 +00:00
FunctionArgumentDescriptors mandatory_args = { { " Value " , nullptr , nullptr , nullptr } } ;
2019-12-23 14:54:06 +00:00
FunctionArgumentDescriptors optional_args ;
2019-10-14 10:15:10 +00:00
2020-08-20 11:18:29 +00:00
if constexpr ( to_decimal )
2018-08-21 18:25:38 +00:00
{
2019-12-23 14:54:06 +00:00
mandatory_args . push_back ( { " scale " , & isNativeInteger , & isColumnConst , " const Integer " } ) ;
2019-10-14 10:15:10 +00:00
}
2020-08-15 13:29:24 +00:00
2020-08-20 11:18:29 +00:00
if ( ! to_decimal & & isDateTime64 < Name , ToDataType > ( arguments ) )
2020-08-15 13:29:24 +00:00
{
2020-08-20 11:18:29 +00:00
mandatory_args . push_back ( { " scale " , & isNativeInteger , & isColumnConst , " const Integer " } ) ;
2020-08-15 13:29:24 +00:00
}
2019-12-23 14:54:06 +00:00
// toString(DateTime or DateTime64, [timezone: String])
2020-10-17 14:23:37 +00:00
if ( ( std : : is_same_v < Name , NameToString > & & ! arguments . empty ( ) & & ( isDateTime64 ( arguments [ 0 ] . type ) | | isDateTime ( arguments [ 0 ] . type ) ) )
2019-12-25 14:50:13 +00:00
// toUnixTimestamp(value[, timezone : String])
| | std : : is_same_v < Name , NameToUnixTimestamp >
// toDate(value[, timezone : String])
2020-06-27 19:05:00 +00:00
| | std : : is_same_v < ToDataType , DataTypeDate > // TODO: shall we allow timestamp argument for toDate? DateTime knows nothing about timezones and this argument is ignored below.
2021-07-15 11:40:45 +00:00
// toDate(value[, timezone : String])
| | std : : is_same_v < ToDataType , DataTypeDate32 >
2019-12-25 14:50:13 +00:00
// toDateTime(value[, timezone: String])
| | std : : is_same_v < ToDataType , DataTypeDateTime >
// toDateTime64(value, scale : Integer[, timezone: String])
| | std : : is_same_v < ToDataType , DataTypeDateTime64 > )
2018-08-21 18:25:38 +00:00
{
2019-12-23 14:54:06 +00:00
optional_args . push_back ( { " timezone " , & isString , & isColumnConst , " const String " } ) ;
2018-08-21 18:25:38 +00:00
}
2019-10-14 10:15:10 +00:00
validateFunctionArgumentTypes ( * this , arguments , mandatory_args , optional_args ) ;
2018-02-11 23:57:07 +00:00
2017-12-25 04:01:46 +00:00
if constexpr ( std : : is_same_v < ToDataType , DataTypeInterval > )
2017-11-05 05:32:22 +00:00
{
2019-11-16 13:54:52 +00:00
return std : : make_shared < DataTypeInterval > ( Name : : kind ) ;
2017-12-02 02:47:12 +00:00
}
2018-08-21 18:25:38 +00:00
else if constexpr ( to_decimal )
{
UInt64 scale = extractToDecimalScale ( arguments [ 1 ] ) ;
2018-08-22 16:16:39 +00:00
if constexpr ( std : : is_same_v < Name , NameToDecimal32 > )
2020-08-26 12:26:12 +00:00
return createDecimalMaxPrecision < Decimal32 > ( scale ) ;
2018-08-22 16:16:39 +00:00
else if constexpr ( std : : is_same_v < Name , NameToDecimal64 > )
2020-08-26 12:26:12 +00:00
return createDecimalMaxPrecision < Decimal64 > ( scale ) ;
2018-11-24 01:48:06 +00:00
else if constexpr ( std : : is_same_v < Name , NameToDecimal128 > )
2020-08-26 12:26:12 +00:00
return createDecimalMaxPrecision < Decimal128 > ( scale ) ;
2020-08-19 11:52:17 +00:00
else if constexpr ( std : : is_same_v < Name , NameToDecimal256 > )
2020-08-26 12:26:12 +00:00
return createDecimalMaxPrecision < Decimal256 > ( scale ) ;
2018-08-21 18:25:38 +00:00
2021-02-21 12:34:48 +00:00
throw Exception ( " Unexpected branch in code of conversion function: it is a bug. " , ErrorCodes : : LOGICAL_ERROR ) ;
2018-08-21 18:25:38 +00:00
}
2017-12-02 02:47:12 +00:00
else
{
2019-12-11 08:56:32 +00:00
// Optional second argument with time zone for DateTime.
2019-10-14 10:15:10 +00:00
UInt8 timezone_arg_position = 1 ;
2019-11-07 10:56:13 +00:00
UInt32 scale [[maybe_unused]] = DataTypeDateTime64 : : default_scale ;
2017-11-05 05:32:22 +00:00
2019-10-08 04:59:38 +00:00
// DateTime64 requires more arguments: scale and timezone. Since timezone is optional, scale should be first.
2020-08-20 11:18:29 +00:00
if ( isDateTime64 < Name , ToDataType > ( arguments ) )
2017-11-05 05:32:22 +00:00
{
2019-10-14 10:15:10 +00:00
timezone_arg_position + = 1 ;
scale = static_cast < UInt32 > ( arguments [ 1 ] . column - > get64 ( 0 ) ) ;
2017-11-05 05:32:22 +00:00
2020-08-20 11:18:29 +00:00
if ( to_datetime64 | | scale ! = 0 ) /// toDateTime('xxxx-xx-xx xx:xx:xx', 0) return DateTime
return std : : make_shared < DataTypeDateTime64 > ( scale ,
extractTimeZoneNameFromFunctionArguments ( arguments , timezone_arg_position , 0 ) ) ;
return std : : make_shared < DataTypeDateTime > ( extractTimeZoneNameFromFunctionArguments ( arguments , timezone_arg_position , 0 ) ) ;
2017-11-05 05:32:22 +00:00
}
2019-10-14 10:15:10 +00:00
if constexpr ( std : : is_same_v < ToDataType , DataTypeDateTime > )
return std : : make_shared < DataTypeDateTime > ( extractTimeZoneNameFromFunctionArguments ( arguments , timezone_arg_position , 0 ) ) ;
2020-08-20 11:18:29 +00:00
else if constexpr ( std : : is_same_v < ToDataType , DataTypeDateTime64 > )
2021-02-21 12:34:48 +00:00
throw Exception ( " Unexpected branch in code of conversion function: it is a bug. " , ErrorCodes : : LOGICAL_ERROR ) ;
2017-12-02 02:47:12 +00:00
else
2018-02-02 08:33:36 +00:00
return std : : make_shared < ToDataType > ( ) ;
2017-12-02 02:47:12 +00:00
}
2017-11-05 05:32:22 +00:00
}
2020-12-24 00:13:53 +00:00
/// Function actually uses default implementation for nulls,
/// but we need to know if return type is Nullable or not,
/// so we use checked_return_type only to intercept the first call to getReturnTypeImpl(...).
2021-07-12 09:18:04 +00:00
bool useDefaultImplementationForNulls ( ) const override
{
bool to_nullable_string = to_nullable & & std : : is_same_v < ToDataType , DataTypeString > ;
return checked_return_type & & ! to_nullable_string ;
}
2020-12-24 00:13:53 +00:00
2017-07-23 08:40:43 +00:00
bool useDefaultImplementationForConstants ( ) const override { return true ; }
2021-02-05 19:06:23 +00:00
ColumnNumbers getArgumentsThatAreAlwaysConstant ( ) const override
{
if constexpr ( std : : is_same_v < ToDataType , DataTypeDateTime64 > )
return { 2 } ;
return { 1 } ;
}
2018-08-08 16:05:50 +00:00
bool canBeExecutedOnDefaultArguments ( ) const override { return false ; }
2017-07-23 08:40:43 +00:00
2020-11-17 13:24:45 +00:00
ColumnPtr executeImpl ( const ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , size_t input_rows_count ) const override
2017-04-01 07:20:54 +00:00
{
try
{
2020-10-17 14:23:37 +00:00
return executeInternal ( arguments , result_type , input_rows_count ) ;
2017-04-01 07:20:54 +00:00
}
catch ( Exception & e )
{
/// More convenient error message.
if ( e . code ( ) = = ErrorCodes : : ATTEMPT_TO_READ_AFTER_EOF )
{
e . addMessage ( " Cannot parse "
2020-10-17 14:23:37 +00:00
+ result_type - > getName ( ) + " from "
+ arguments [ 0 ] . type - > getName ( )
2017-04-01 07:20:54 +00:00
+ " , because value is too short " ) ;
}
else if ( e . code ( ) = = ErrorCodes : : CANNOT_PARSE_NUMBER
| | e . code ( ) = = ErrorCodes : : CANNOT_READ_ARRAY_FROM_TEXT
| | e . code ( ) = = ErrorCodes : : CANNOT_PARSE_INPUT_ASSERTION_FAILED
| | e . code ( ) = = ErrorCodes : : CANNOT_PARSE_QUOTED_STRING
| | e . code ( ) = = ErrorCodes : : CANNOT_PARSE_ESCAPE_SEQUENCE
| | e . code ( ) = = ErrorCodes : : CANNOT_PARSE_DATE
2017-06-15 09:12:32 +00:00
| | e . code ( ) = = ErrorCodes : : CANNOT_PARSE_DATETIME
| | e . code ( ) = = ErrorCodes : : CANNOT_PARSE_UUID )
2017-04-01 07:20:54 +00:00
{
e . addMessage ( " Cannot parse "
2020-10-17 14:23:37 +00:00
+ result_type - > getName ( ) + " from "
+ arguments [ 0 ] . type - > getName ( ) ) ;
2017-04-01 07:20:54 +00:00
}
throw ;
}
}
bool hasInformationAboutMonotonicity ( ) const override
{
return Monotonic : : has ( ) ;
}
Monotonicity getMonotonicityForRange ( const IDataType & type , const Field & left , const Field & right ) const override
{
return Monotonic : : get ( type , left , right ) ;
}
2016-08-17 06:55:47 +00:00
private :
2020-12-24 00:13:53 +00:00
mutable bool checked_return_type = false ;
2020-12-23 21:57:33 +00:00
mutable bool to_nullable = false ;
2020-11-17 13:24:45 +00:00
ColumnPtr executeInternal ( const ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , size_t input_rows_count ) const
2017-04-01 07:20:54 +00:00
{
2020-10-17 14:23:37 +00:00
if ( arguments . empty ( ) )
2021-05-03 19:56:40 +00:00
throw Exception { " Function " + getName ( ) + " expects at least 1 argument " ,
2018-12-07 03:20:27 +00:00
ErrorCodes : : TOO_FEW_ARGUMENTS_FOR_FUNCTION } ;
2017-12-13 21:09:31 +00:00
2021-07-12 13:04:59 +00:00
if ( result_type - > onlyNull ( ) )
return result_type - > createColumnConstWithDefaultValue ( input_rows_count ) ;
2021-07-12 09:18:04 +00:00
const DataTypePtr from_type = removeNullable ( arguments [ 0 ] . type ) ;
2020-10-17 14:23:37 +00:00
ColumnPtr result_column ;
2017-07-21 06:35:58 +00:00
2020-12-23 21:57:33 +00:00
auto call = [ & ] ( const auto & types , const auto & tag ) - > bool
2018-08-20 15:17:55 +00:00
{
using Types = std : : decay_t < decltype ( types ) > ;
using LeftDataType = typename Types : : LeftType ;
using RightDataType = typename Types : : RightType ;
2020-12-23 21:57:33 +00:00
using SpecialTag = std : : decay_t < decltype ( tag ) > ;
2018-08-20 15:17:55 +00:00
2018-09-10 13:52:18 +00:00
if constexpr ( IsDataTypeDecimal < RightDataType > )
2018-08-31 08:59:21 +00:00
{
2019-10-12 07:03:38 +00:00
if constexpr ( std : : is_same_v < RightDataType , DataTypeDateTime64 > )
{
2021-05-03 19:56:40 +00:00
/// Account for optional timezone argument.
2019-10-12 07:03:38 +00:00
if ( arguments . size ( ) ! = 2 & & arguments . size ( ) ! = 3 )
throw Exception { " Function " + getName ( ) + " expects 2 or 3 arguments for DataTypeDateTime64. " ,
ErrorCodes : : TOO_FEW_ARGUMENTS_FOR_FUNCTION } ;
}
else if ( arguments . size ( ) ! = 2 )
{
2018-08-31 08:59:21 +00:00
throw Exception { " Function " + getName ( ) + " expects 2 arguments for Decimal. " ,
2018-12-07 03:20:27 +00:00
ErrorCodes : : TOO_FEW_ARGUMENTS_FOR_FUNCTION } ;
2019-10-12 07:03:38 +00:00
}
2018-08-31 08:59:21 +00:00
2020-10-17 14:23:37 +00:00
const ColumnWithTypeAndName & scale_column = arguments [ 1 ] ;
2018-08-31 08:59:21 +00:00
UInt32 scale = extractToDecimalScale ( scale_column ) ;
2020-12-24 00:13:53 +00:00
result_column = ConvertImpl < LeftDataType , RightDataType , Name , SpecialTag > : : execute ( arguments , result_type , input_rows_count , scale ) ;
2018-08-31 08:59:21 +00:00
}
2019-10-31 08:23:55 +00:00
else if constexpr ( IsDataTypeDateOrDateTime < RightDataType > & & std : : is_same_v < LeftDataType , DataTypeDateTime64 > )
{
2020-10-17 14:23:37 +00:00
const auto * dt64 = assert_cast < const DataTypeDateTime64 * > ( arguments [ 0 ] . type . get ( ) ) ;
2020-12-24 00:13:53 +00:00
result_column = ConvertImpl < LeftDataType , RightDataType , Name , SpecialTag > : : execute ( arguments , result_type , input_rows_count , dt64 - > getScale ( ) ) ;
2019-10-31 08:23:55 +00:00
}
2020-08-19 11:52:17 +00:00
else if constexpr ( IsDataTypeDecimalOrNumber < LeftDataType > & & IsDataTypeDecimalOrNumber < RightDataType > )
{
using LeftT = typename LeftDataType : : FieldType ;
using RightT = typename RightDataType : : FieldType ;
static constexpr bool bad_left =
IsDecimalNumber < LeftT > | | std : : is_floating_point_v < LeftT > | | is_big_int_v < LeftT > | | is_signed_v < LeftT > ;
static constexpr bool bad_right =
IsDecimalNumber < RightT > | | std : : is_floating_point_v < RightT > | | is_big_int_v < RightT > | | is_signed_v < RightT > ;
/// Disallow int vs UUID conversion (but support int vs UInt128 conversion)
if constexpr ( ( bad_left & & std : : is_same_v < RightDataType , DataTypeUUID > ) | |
( bad_right & & std : : is_same_v < LeftDataType , DataTypeUUID > ) )
{
throw Exception ( " Wrong UUID conversion " , ErrorCodes : : CANNOT_CONVERT_TYPE ) ;
}
else
2020-12-23 21:57:33 +00:00
result_column = ConvertImpl < LeftDataType , RightDataType , Name , SpecialTag > : : execute ( arguments , result_type , input_rows_count ) ;
2020-08-19 11:52:17 +00:00
}
2018-08-31 08:59:21 +00:00
else
2021-07-15 11:40:45 +00:00
{
2020-12-23 21:57:33 +00:00
result_column = ConvertImpl < LeftDataType , RightDataType , Name , SpecialTag > : : execute ( arguments , result_type , input_rows_count ) ;
2021-07-15 11:40:45 +00:00
}
2019-05-04 00:07:54 +00:00
2018-08-20 15:17:55 +00:00
return true ;
} ;
2020-10-17 14:23:37 +00:00
if ( isDateTime64 < Name , ToDataType > ( arguments ) )
2020-08-15 13:29:24 +00:00
{
/// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64
2020-10-17 14:23:37 +00:00
const ColumnWithTypeAndName & scale_column = arguments [ 1 ] ;
2020-08-20 11:18:29 +00:00
UInt32 scale = extractToDecimalScale ( scale_column ) ;
2020-08-15 17:08:03 +00:00
2020-08-21 05:16:50 +00:00
if ( to_datetime64 | | scale ! = 0 ) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64
2020-08-20 11:18:29 +00:00
{
2020-12-23 21:57:33 +00:00
if ( ! callOnIndexAndDataType < DataTypeDateTime64 > ( from_type - > getTypeId ( ) , call , ConvertDefaultBehaviorTag { } ) )
2020-10-17 14:23:37 +00:00
throw Exception ( " Illegal type " + arguments [ 0 ] . type - > getName ( ) + " of argument of function " + getName ( ) ,
2020-08-20 11:18:29 +00:00
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
2020-08-15 17:08:03 +00:00
2020-10-17 14:23:37 +00:00
return result_column ;
2020-08-15 13:29:24 +00:00
}
}
2020-12-23 21:57:33 +00:00
bool done ;
if constexpr ( to_string_or_fixed_string )
{
done = callOnIndexAndDataType < ToDataType > ( from_type - > getTypeId ( ) , call , ConvertDefaultBehaviorTag { } ) ;
}
else
{
2020-12-24 00:13:53 +00:00
/// We should use ConvertFromStringExceptionMode::Null mode when converting from String (or FixedString)
/// to Nullable type, to avoid 'value is too short' error on attempt to parse empty string from NULL values.
2020-12-23 21:57:33 +00:00
if ( to_nullable & & WhichDataType ( from_type ) . isStringOrFixedString ( ) )
done = callOnIndexAndDataType < ToDataType > ( from_type - > getTypeId ( ) , call , ConvertReturnNullOnErrorTag { } ) ;
else
done = callOnIndexAndDataType < ToDataType > ( from_type - > getTypeId ( ) , call , ConvertDefaultBehaviorTag { } ) ;
}
2018-08-20 15:17:55 +00:00
if ( ! done )
2017-04-01 07:20:54 +00:00
{
/// Generic conversion of any type to String.
2017-12-25 04:01:46 +00:00
if ( std : : is_same_v < ToDataType , DataTypeString > )
2017-04-01 07:20:54 +00:00
{
2021-07-12 10:15:30 +00:00
return ConvertImplGenericToString : : execute ( arguments , result_type ) ;
2017-04-01 07:20:54 +00:00
}
else
2020-10-17 14:23:37 +00:00
throw Exception ( " Illegal type " + arguments [ 0 ] . type - > getName ( ) + " of argument of function " + getName ( ) ,
2017-04-01 07:20:54 +00:00
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
}
2020-10-19 18:37:44 +00:00
return result_column ;
2017-04-01 07:20:54 +00:00
}
2011-10-15 23:40:56 +00:00
} ;
2018-02-11 23:57:07 +00:00
/** Function toTOrZero (where T is number of date or datetime type):
2016-08-02 05:32:09 +00:00
* try to convert from String to type T through parsing ,
* if cannot parse , return default value instead of throwing exception .
2018-02-11 23:57:07 +00:00
* Function toTOrNull will return Nullable type with NULL when cannot parse .
* NOTE Also need to implement tryToUnixTimestamp with timezone .
2016-08-02 05:32:09 +00:00
*/
2018-02-11 23:57:07 +00:00
template < typename ToDataType , typename Name ,
ConvertFromStringExceptionMode exception_mode ,
ConvertFromStringParsingMode parsing_mode = ConvertFromStringParsingMode : : Normal >
class FunctionConvertFromString : public IFunction
2016-08-02 05:32:09 +00:00
{
public :
2017-04-01 07:20:54 +00:00
static constexpr auto name = Name : : name ;
2018-08-31 08:59:21 +00:00
static constexpr bool to_decimal =
std : : is_same_v < ToDataType , DataTypeDecimal < Decimal32 > > | |
std : : is_same_v < ToDataType , DataTypeDecimal < Decimal64 > > | |
2020-08-19 11:52:17 +00:00
std : : is_same_v < ToDataType , DataTypeDecimal < Decimal128 > > | |
std : : is_same_v < ToDataType , DataTypeDecimal < Decimal256 > > ;
2018-08-31 08:59:21 +00:00
2020-05-16 12:11:17 +00:00
static constexpr bool to_datetime64 = std : : is_same_v < ToDataType , DataTypeDateTime64 > ;
2021-04-10 23:33:54 +00:00
static FunctionPtr create ( ContextPtr ) { return std : : make_shared < FunctionConvertFromString > ( ) ; }
2019-12-25 19:17:41 +00:00
static FunctionPtr create ( ) { return std : : make_shared < FunctionConvertFromString > ( ) ; }
2017-04-01 07:20:54 +00:00
String getName ( ) const override
{
return name ;
}
2018-02-11 23:57:07 +00:00
bool isVariadic ( ) const override { return true ; }
2021-04-29 14:48:26 +00:00
bool isSuitableForShortCircuitArgumentsExecution ( ) const override { return true ; }
2018-02-11 23:57:07 +00:00
size_t getNumberOfArguments ( ) const override { return 0 ; }
2017-04-01 07:20:54 +00:00
2017-07-23 08:40:43 +00:00
bool useDefaultImplementationForConstants ( ) const override { return true ; }
2018-02-11 23:57:07 +00:00
ColumnNumbers getArgumentsThatAreAlwaysConstant ( ) const override { return { 1 } ; }
2017-07-23 08:40:43 +00:00
2018-02-11 23:57:07 +00:00
DataTypePtr getReturnTypeImpl ( const ColumnsWithTypeAndName & arguments ) const override
2017-04-01 07:20:54 +00:00
{
2020-05-16 12:11:17 +00:00
DataTypePtr res ;
2020-08-20 14:41:03 +00:00
if ( isDateTime64 < Name , ToDataType > ( arguments ) )
2019-11-21 15:46:25 +00:00
{
2020-05-16 12:11:17 +00:00
validateFunctionArgumentTypes ( * this , arguments ,
FunctionArgumentDescriptors { { " string " , isStringOrFixedString , nullptr , " String or FixedString " } } ,
// optional
FunctionArgumentDescriptors {
{ " precision " , isUInt8 , isColumnConst , " const UInt8 " } ,
{ " timezone " , isStringOrFixedString , isColumnConst , " const String or FixedString " } ,
} ) ;
2017-12-25 05:22:33 +00:00
2020-08-20 14:41:03 +00:00
UInt64 scale = to_datetime64 ? DataTypeDateTime64 : : default_scale : 0 ;
2020-05-16 12:11:17 +00:00
if ( arguments . size ( ) > 1 )
scale = extractToDecimalScale ( arguments [ 1 ] ) ;
const auto timezone = extractTimeZoneNameFromFunctionArguments ( arguments , 2 , 0 ) ;
2020-08-20 14:41:03 +00:00
res = scale = = 0 ? res = std : : make_shared < DataTypeDateTime > ( timezone ) : std : : make_shared < DataTypeDateTime64 > ( scale , timezone ) ;
2020-05-16 12:11:17 +00:00
}
else
2018-02-11 23:57:07 +00:00
{
2020-05-16 12:11:17 +00:00
if ( ( arguments . size ( ) ! = 1 & & arguments . size ( ) ! = 2 ) | | ( to_decimal & & arguments . size ( ) ! = 2 ) )
throw Exception ( " Number of arguments for function " + getName ( ) + " doesn't match: passed " + toString ( arguments . size ( ) ) +
" , should be 1 or 2. Second argument only make sense for DateTime (time zone, optional) and Decimal (scale). " ,
ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ) ;
if ( ! isStringOrFixedString ( arguments [ 0 ] . type ) )
2019-05-15 18:50:35 +00:00
{
2020-05-16 12:11:17 +00:00
if ( this - > getName ( ) . find ( " OrZero " ) ! = std : : string : : npos | |
this - > getName ( ) . find ( " OrNull " ) ! = std : : string : : npos )
throw Exception ( " Illegal type " + arguments [ 0 ] . type - > getName ( ) + " of first argument of function " + getName ( ) +
" . Conversion functions with postfix 'OrZero' or 'OrNull' should take String argument " ,
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
else
throw Exception ( " Illegal type " + arguments [ 0 ] . type - > getName ( ) + " of first argument of function " + getName ( ) ,
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
2019-05-15 18:50:35 +00:00
}
2020-05-16 12:11:17 +00:00
if ( arguments . size ( ) = = 2 )
2019-05-15 18:50:35 +00:00
{
2020-05-16 12:11:17 +00:00
if constexpr ( std : : is_same_v < ToDataType , DataTypeDateTime > )
{
if ( ! isString ( arguments [ 1 ] . type ) )
throw Exception ( " Illegal type " + arguments [ 1 ] . type - > getName ( ) + " of 2nd argument of function " + getName ( ) ,
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
}
else if constexpr ( to_decimal )
{
if ( ! isInteger ( arguments [ 1 ] . type ) )
throw Exception ( " Illegal type " + arguments [ 1 ] . type - > getName ( ) + " of 2nd argument of function " + getName ( ) ,
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
if ( ! arguments [ 1 ] . column )
throw Exception ( " Second argument for function " + getName ( ) + " must be constant " , ErrorCodes : : ILLEGAL_COLUMN ) ;
}
else
{
throw Exception ( " Number of arguments for function " + getName ( ) + " doesn't match: passed "
+ toString ( arguments . size ( ) ) + " , should be 1. Second argument makes sense only for DateTime and Decimal. " ,
ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ) ;
}
2019-05-15 18:50:35 +00:00
}
2017-12-25 05:22:33 +00:00
2020-05-16 12:11:17 +00:00
if constexpr ( std : : is_same_v < ToDataType , DataTypeDateTime > )
res = std : : make_shared < DataTypeDateTime > ( extractTimeZoneNameFromFunctionArguments ( arguments , 1 , 0 ) ) ;
2020-08-20 14:41:03 +00:00
else if constexpr ( std : : is_same_v < ToDataType , DataTypeDateTime64 > )
throw Exception ( " LOGICAL ERROR: It is a bug. " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-05-16 12:11:17 +00:00
else if constexpr ( to_decimal )
{
UInt64 scale = extractToDecimalScale ( arguments [ 1 ] ) ;
2020-08-26 12:26:12 +00:00
res = createDecimalMaxPrecision < typename ToDataType : : FieldType > ( scale ) ;
2020-05-16 12:11:17 +00:00
if ( ! res )
2020-08-08 00:47:03 +00:00
throw Exception ( " Something wrong with toDecimalNNOrZero() or toDecimalNNOrNull ( ) " , ErrorCodes::LOGICAL_ERROR) ;
2020-05-16 12:11:17 +00:00
}
else
res = std : : make_shared < ToDataType > ( ) ;
2019-12-11 08:56:32 +00:00
}
2018-02-12 00:55:46 +00:00
if constexpr ( exception_mode = = ConvertFromStringExceptionMode : : Null )
res = std : : make_shared < DataTypeNullable > ( res ) ;
return res ;
2017-12-25 05:22:33 +00:00
}
2020-08-20 14:41:03 +00:00
template < typename ConvertToDataType >
2020-11-17 13:24:45 +00:00
ColumnPtr executeInternal ( const ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , size_t input_rows_count , UInt32 scale = 0 ) const
2017-12-25 05:22:33 +00:00
{
2020-10-17 14:23:37 +00:00
const IDataType * from_type = arguments [ 0 ] . type . get ( ) ;
2017-12-25 05:22:33 +00:00
2020-08-20 14:41:03 +00:00
if ( checkAndGetDataType < DataTypeString > ( from_type ) )
2019-05-15 18:50:35 +00:00
{
2020-10-17 14:23:37 +00:00
return ConvertThroughParsing < DataTypeString , ConvertToDataType , Name , exception_mode , parsing_mode > : : execute (
arguments , result_type , input_rows_count , scale ) ;
2019-05-15 18:50:35 +00:00
}
2020-08-20 14:41:03 +00:00
else if ( checkAndGetDataType < DataTypeFixedString > ( from_type ) )
{
2020-10-17 14:23:37 +00:00
return ConvertThroughParsing < DataTypeFixedString , ConvertToDataType , Name , exception_mode , parsing_mode > : : execute (
arguments , result_type , input_rows_count , scale ) ;
2019-05-15 18:50:35 +00:00
}
2020-08-20 14:41:03 +00:00
2020-10-17 14:23:37 +00:00
return nullptr ;
2020-08-20 14:41:03 +00:00
}
2020-11-17 13:24:45 +00:00
ColumnPtr executeImpl ( const ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , size_t input_rows_count ) const override
2020-08-20 14:41:03 +00:00
{
2020-10-17 14:23:37 +00:00
ColumnPtr result_column ;
2020-08-20 14:41:03 +00:00
if constexpr ( to_decimal )
2020-10-17 14:23:37 +00:00
result_column = executeInternal < ToDataType > ( arguments , result_type , input_rows_count ,
assert_cast < const ToDataType & > ( * removeNullable ( result_type ) ) . getScale ( ) ) ;
2017-12-25 05:22:33 +00:00
else
2019-05-15 18:50:35 +00:00
{
2020-10-17 14:23:37 +00:00
if ( isDateTime64 < Name , ToDataType > ( arguments ) )
2019-05-15 18:50:35 +00:00
{
2020-08-20 14:41:03 +00:00
UInt64 scale = to_datetime64 ? DataTypeDateTime64 : : default_scale : 0 ;
if ( arguments . size ( ) > 1 )
2020-10-17 14:23:37 +00:00
scale = extractToDecimalScale ( arguments [ 1 ] ) ;
2020-08-20 14:41:03 +00:00
if ( scale = = 0 )
2020-10-17 14:23:37 +00:00
result_column = executeInternal < DataTypeDateTime > ( arguments , result_type , input_rows_count ) ;
2020-08-20 14:41:03 +00:00
else
{
2020-10-17 14:23:37 +00:00
result_column = executeInternal < DataTypeDateTime64 > ( arguments , result_type , input_rows_count , static_cast < UInt32 > ( scale ) ) ;
2020-08-20 14:41:03 +00:00
}
2019-05-15 18:50:35 +00:00
}
2020-08-20 14:41:03 +00:00
else
2019-05-15 18:50:35 +00:00
{
2020-10-17 14:23:37 +00:00
result_column = executeInternal < ToDataType > ( arguments , result_type , input_rows_count ) ;
2019-05-15 18:50:35 +00:00
}
}
2020-10-17 14:23:37 +00:00
if ( ! result_column )
throw Exception ( " Illegal type " + arguments [ 0 ] . type - > getName ( ) + " of argument of function " + getName ( )
2018-08-31 08:59:21 +00:00
+ " . Only String or FixedString argument is accepted for try-conversion function. "
+ " For other arguments, use function without 'orZero' or 'orNull'. " ,
2017-04-01 07:20:54 +00:00
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
2020-10-17 14:23:37 +00:00
return result_column ;
2017-04-01 07:20:54 +00:00
}
2016-08-02 05:32:09 +00:00
} ;
2013-12-03 10:19:57 +00:00
2016-07-23 23:02:14 +00:00
/// Monotonicity.
2015-11-29 08:06:29 +00:00
2015-11-29 11:58:44 +00:00
struct PositiveMonotonicity
2015-11-29 08:06:29 +00:00
{
2017-04-01 07:20:54 +00:00
static bool has ( ) { return true ; }
2017-12-02 02:47:12 +00:00
static IFunction : : Monotonicity get ( const IDataType & , const Field & , const Field & )
2017-04-01 07:20:54 +00:00
{
return { true } ;
}
2015-11-29 08:06:29 +00:00
} ;
2018-08-22 17:26:11 +00:00
struct UnknownMonotonicity
{
static bool has ( ) { return false ; }
static IFunction : : Monotonicity get ( const IDataType & , const Field & , const Field & )
{
return { false } ;
}
} ;
2015-11-29 11:58:44 +00:00
template < typename T >
2019-08-08 08:41:38 +00:00
struct ToNumberMonotonicity
2015-11-29 08:06:29 +00:00
{
2017-04-01 07:20:54 +00:00
static bool has ( ) { return true ; }
2017-12-02 02:47:12 +00:00
static UInt64 divideByRangeOfType ( UInt64 x )
{
if constexpr ( sizeof ( T ) < sizeof ( UInt64 ) )
return x > > ( sizeof ( T ) * 8 ) ;
else
return 0 ;
}
2017-04-01 07:20:54 +00:00
static IFunction : : Monotonicity get ( const IDataType & type , const Field & left , const Field & right )
{
2018-12-20 11:42:45 +00:00
if ( ! type . isValueRepresentedByNumber ( ) )
2018-12-19 06:57:37 +00:00
return { } ;
2018-12-24 15:46:55 +00:00
/// If type is same, the conversion is always monotonic.
/// (Enum has separate case, because it is different data type)
2018-12-21 11:37:33 +00:00
if ( checkAndGetDataType < DataTypeNumber < T > > ( & type ) | |
2018-09-07 14:37:26 +00:00
checkAndGetDataType < DataTypeEnum < T > > ( & type ) )
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
return { true , true , true } ;
2017-04-01 07:20:54 +00:00
2018-12-24 15:46:55 +00:00
/// Float cases.
2018-12-24 13:25:48 +00:00
2018-12-24 15:46:55 +00:00
/// When converting to Float, the conversion is always monotonic.
if ( std : : is_floating_point_v < T > )
return { true , true , true } ;
/// If converting from Float, for monotonicity, arguments must fit in range of result type.
2018-09-07 14:37:26 +00:00
if ( WhichDataType ( type ) . isFloat ( ) )
2017-04-01 07:20:54 +00:00
{
2018-12-24 15:46:55 +00:00
if ( left . isNull ( ) | | right . isNull ( ) )
return { } ;
2017-04-01 07:20:54 +00:00
Float64 left_float = left . get < Float64 > ( ) ;
Float64 right_float = right . get < Float64 > ( ) ;
2020-08-19 11:52:17 +00:00
if ( left_float > = static_cast < Float64 > ( std : : numeric_limits < T > : : min ( ) )
& & left_float < = static_cast < Float64 > ( std : : numeric_limits < T > : : max ( ) )
& & right_float > = static_cast < Float64 > ( std : : numeric_limits < T > : : min ( ) )
& & right_float < = static_cast < Float64 > ( std : : numeric_limits < T > : : max ( ) ) )
2017-04-01 07:20:54 +00:00
return { true } ;
return { } ;
}
2018-12-24 15:46:55 +00:00
/// Integer cases.
2018-12-25 18:40:47 +00:00
const bool from_is_unsigned = type . isValueRepresentedByUnsignedInteger ( ) ;
2019-11-02 05:55:06 +00:00
const bool to_is_unsigned = is_unsigned_v < T > ;
2018-12-24 15:46:55 +00:00
2018-12-25 18:40:47 +00:00
const size_t size_of_from = type . getSizeOfValueInMemory ( ) ;
const size_t size_of_to = sizeof ( T ) ;
2018-12-24 15:46:55 +00:00
2018-12-25 18:40:47 +00:00
const bool left_in_first_half = left . isNull ( )
2018-12-24 15:46:55 +00:00
? from_is_unsigned
2018-12-25 18:40:47 +00:00
: ( left . get < Int64 > ( ) > = 0 ) ;
2018-12-24 15:46:55 +00:00
2018-12-25 18:40:47 +00:00
const bool right_in_first_half = right . isNull ( )
2018-12-24 15:46:55 +00:00
? ! from_is_unsigned
2018-12-25 18:40:47 +00:00
: ( right . get < Int64 > ( ) > = 0 ) ;
2018-12-25 18:06:38 +00:00
2018-12-24 15:46:55 +00:00
/// Size of type is the same.
if ( size_of_from = = size_of_to )
{
if ( from_is_unsigned = = to_is_unsigned )
return { true , true , true } ;
if ( left_in_first_half = = right_in_first_half )
return { true } ;
2017-04-01 07:20:54 +00:00
return { } ;
2018-12-24 15:46:55 +00:00
}
2017-04-01 07:20:54 +00:00
2018-12-24 15:46:55 +00:00
/// Size of type is expanded.
if ( size_of_from < size_of_to )
{
if ( from_is_unsigned = = to_is_unsigned )
return { true , true , true } ;
if ( ! to_is_unsigned )
return { true , true , true } ;
/// signed -> unsigned. If arguments from the same half, then function is monotonic.
if ( left_in_first_half = = right_in_first_half )
return { true } ;
2018-12-25 18:40:47 +00:00
return { } ;
2018-12-24 15:46:55 +00:00
}
2020-08-08 00:47:03 +00:00
/// Size of type is shrunk.
2018-12-24 15:46:55 +00:00
if ( size_of_from > size_of_to )
{
/// Function cannot be monotonic on unbounded ranges.
if ( left . isNull ( ) | | right . isNull ( ) )
return { } ;
2020-09-14 03:34:14 +00:00
/// Function cannot be monotonic when left and right are not on the same ranges.
if ( divideByRangeOfType ( left . get < UInt64 > ( ) ) ! = divideByRangeOfType ( right . get < UInt64 > ( ) ) )
2018-12-24 15:46:55 +00:00
return { } ;
2020-09-14 03:34:14 +00:00
if ( to_is_unsigned )
return { true } ;
else
// If To is signed, it's possible that the signedness is different after conversion. So we check it explicitly.
return { ( T ( left . get < UInt64 > ( ) ) > = 0 ) = = ( T ( right . get < UInt64 > ( ) ) > = 0 ) } ;
2018-12-24 15:46:55 +00:00
}
__builtin_unreachable ( ) ;
2017-04-01 07:20:54 +00:00
}
2015-11-29 08:06:29 +00:00
} ;
2020-08-07 17:38:42 +00:00
struct ToDateMonotonicity
{
static bool has ( ) { return true ; }
static IFunction : : Monotonicity get ( const IDataType & type , const Field & left , const Field & right )
{
auto which = WhichDataType ( type ) ;
2021-07-15 11:40:45 +00:00
if ( which . isDateOrDate32 ( ) | | which . isDateTime ( ) | | which . isDateTime64 ( ) | | which . isInt8 ( ) | | which . isInt16 ( ) | | which . isUInt8 ( ) | | which . isUInt16 ( ) )
2020-08-07 17:38:42 +00:00
return { true , true , true } ;
else if (
( which . isUInt ( ) & & ( ( left . isNull ( ) | | left . get < UInt64 > ( ) < 0xFFFF ) & & ( right . isNull ( ) | | right . get < UInt64 > ( ) > = 0xFFFF ) ) )
| | ( which . isInt ( ) & & ( ( left . isNull ( ) | | left . get < Int64 > ( ) < 0xFFFF ) & & ( right . isNull ( ) | | right . get < Int64 > ( ) > = 0xFFFF ) ) )
2020-08-08 06:30:50 +00:00
| | ( which . isFloat ( ) & & ( ( left . isNull ( ) | | left . get < Float64 > ( ) < 0xFFFF ) & & ( right . isNull ( ) | | right . get < Float64 > ( ) > = 0xFFFF ) ) )
| | ! type . isValueRepresentedByNumber ( ) )
2020-08-07 17:38:42 +00:00
return { } ;
else
return { true , true , true } ;
}
} ;
2020-08-08 06:30:50 +00:00
struct ToDateTimeMonotonicity
{
static bool has ( ) { return true ; }
static IFunction : : Monotonicity get ( const IDataType & type , const Field & , const Field & )
{
if ( type . isValueRepresentedByNumber ( ) )
return { true , true , true } ;
else
return { } ;
}
} ;
2017-05-13 22:19:04 +00:00
/** The monotonicity for the `toString` function is mainly determined for test purposes.
* It is doubtful that anyone is looking to optimize queries with conditions ` toString ( CounterID ) = 34 ` .
2015-11-29 11:58:44 +00:00
*/
struct ToStringMonotonicity
2015-11-29 08:06:29 +00:00
{
2017-04-01 07:20:54 +00:00
static bool has ( ) { return true ; }
static IFunction : : Monotonicity get ( const IDataType & type , const Field & left , const Field & right )
{
IFunction : : Monotonicity positive ( true , true ) ;
IFunction : : Monotonicity not_monotonic ;
2020-10-17 14:23:37 +00:00
const auto * type_ptr = & type ;
if ( const auto * low_cardinality_type = checkAndGetDataType < DataTypeLowCardinality > ( type_ptr ) )
2020-04-08 05:27:46 +00:00
type_ptr = low_cardinality_type - > getDictionaryType ( ) . get ( ) ;
2017-04-01 07:20:54 +00:00
2021-07-15 11:40:45 +00:00
/// `toString` function is monotonous if the argument is Date or Date32 or DateTime or String, or non-negative numbers with the same number of symbols.
if ( checkDataTypes < DataTypeDate , DataTypeDate32 , DataTypeDateTime , DataTypeString > ( type_ptr ) )
2017-04-01 07:20:54 +00:00
return positive ;
if ( left . isNull ( ) | | right . isNull ( ) )
return { } ;
if ( left . getType ( ) = = Field : : Types : : UInt64
& & right . getType ( ) = = Field : : Types : : UInt64 )
{
return ( left . get < Int64 > ( ) = = 0 & & right . get < Int64 > ( ) = = 0 )
| | ( floor ( log10 ( left . get < UInt64 > ( ) ) ) = = floor ( log10 ( right . get < UInt64 > ( ) ) ) )
? positive : not_monotonic ;
}
if ( left . getType ( ) = = Field : : Types : : Int64
& & right . getType ( ) = = Field : : Types : : Int64 )
{
return ( left . get < Int64 > ( ) = = 0 & & right . get < Int64 > ( ) = = 0 )
| | ( left . get < Int64 > ( ) > 0 & & right . get < Int64 > ( ) > 0 & & floor ( log10 ( left . get < Int64 > ( ) ) ) = = floor ( log10 ( right . get < Int64 > ( ) ) ) )
? positive : not_monotonic ;
}
return not_monotonic ;
}
2015-11-29 08:06:29 +00:00
} ;
2017-10-30 02:18:06 +00:00
struct NameToUInt8 { static constexpr auto name = " toUInt8 " ; } ;
struct NameToUInt16 { static constexpr auto name = " toUInt16 " ; } ;
struct NameToUInt32 { static constexpr auto name = " toUInt32 " ; } ;
struct NameToUInt64 { static constexpr auto name = " toUInt64 " ; } ;
2021-05-03 15:41:37 +00:00
struct NameToUInt128 { static constexpr auto name = " toUInt128 " ; } ;
2020-08-19 11:52:17 +00:00
struct NameToUInt256 { static constexpr auto name = " toUInt256 " ; } ;
2017-10-30 02:18:06 +00:00
struct NameToInt8 { static constexpr auto name = " toInt8 " ; } ;
struct NameToInt16 { static constexpr auto name = " toInt16 " ; } ;
struct NameToInt32 { static constexpr auto name = " toInt32 " ; } ;
struct NameToInt64 { static constexpr auto name = " toInt64 " ; } ;
2020-08-19 11:52:17 +00:00
struct NameToInt128 { static constexpr auto name = " toInt128 " ; } ;
struct NameToInt256 { static constexpr auto name = " toInt256 " ; } ;
2017-10-30 02:18:06 +00:00
struct NameToFloat32 { static constexpr auto name = " toFloat32 " ; } ;
struct NameToFloat64 { static constexpr auto name = " toFloat64 " ; } ;
struct NameToUUID { static constexpr auto name = " toUUID " ; } ;
2019-08-08 08:41:38 +00:00
using FunctionToUInt8 = FunctionConvert < DataTypeUInt8 , NameToUInt8 , ToNumberMonotonicity < UInt8 > > ;
using FunctionToUInt16 = FunctionConvert < DataTypeUInt16 , NameToUInt16 , ToNumberMonotonicity < UInt16 > > ;
using FunctionToUInt32 = FunctionConvert < DataTypeUInt32 , NameToUInt32 , ToNumberMonotonicity < UInt32 > > ;
using FunctionToUInt64 = FunctionConvert < DataTypeUInt64 , NameToUInt64 , ToNumberMonotonicity < UInt64 > > ;
2021-05-03 15:41:37 +00:00
using FunctionToUInt128 = FunctionConvert < DataTypeUInt128 , NameToUInt128 , ToNumberMonotonicity < UInt128 > > ;
2020-09-01 09:54:50 +00:00
using FunctionToUInt256 = FunctionConvert < DataTypeUInt256 , NameToUInt256 , ToNumberMonotonicity < UInt256 > > ;
2019-08-08 08:41:38 +00:00
using FunctionToInt8 = FunctionConvert < DataTypeInt8 , NameToInt8 , ToNumberMonotonicity < Int8 > > ;
using FunctionToInt16 = FunctionConvert < DataTypeInt16 , NameToInt16 , ToNumberMonotonicity < Int16 > > ;
using FunctionToInt32 = FunctionConvert < DataTypeInt32 , NameToInt32 , ToNumberMonotonicity < Int32 > > ;
using FunctionToInt64 = FunctionConvert < DataTypeInt64 , NameToInt64 , ToNumberMonotonicity < Int64 > > ;
2020-08-19 11:52:17 +00:00
using FunctionToInt128 = FunctionConvert < DataTypeInt128 , NameToInt128 , ToNumberMonotonicity < Int128 > > ;
2020-09-01 09:54:50 +00:00
using FunctionToInt256 = FunctionConvert < DataTypeInt256 , NameToInt256 , ToNumberMonotonicity < Int256 > > ;
2019-08-08 08:41:38 +00:00
using FunctionToFloat32 = FunctionConvert < DataTypeFloat32 , NameToFloat32 , ToNumberMonotonicity < Float32 > > ;
using FunctionToFloat64 = FunctionConvert < DataTypeFloat64 , NameToFloat64 , ToNumberMonotonicity < Float64 > > ;
2020-08-07 17:38:42 +00:00
using FunctionToDate = FunctionConvert < DataTypeDate , NameToDate , ToDateMonotonicity > ;
2021-07-15 11:41:52 +00:00
using FunctionToDate32 = FunctionConvert < DataTypeDate32 , NameToDate32 , ToDateMonotonicity > ;
2020-08-08 06:30:50 +00:00
using FunctionToDateTime = FunctionConvert < DataTypeDateTime , NameToDateTime , ToDateTimeMonotonicity > ;
2020-08-15 13:29:24 +00:00
using FunctionToDateTime32 = FunctionConvert < DataTypeDateTime , NameToDateTime32 , ToDateTimeMonotonicity > ;
2019-10-16 07:26:28 +00:00
using FunctionToDateTime64 = FunctionConvert < DataTypeDateTime64 , NameToDateTime64 , UnknownMonotonicity > ;
2019-08-08 08:41:38 +00:00
using FunctionToUUID = FunctionConvert < DataTypeUUID , NameToUUID , ToNumberMonotonicity < UInt128 > > ;
2017-10-30 02:18:06 +00:00
using FunctionToString = FunctionConvert < DataTypeString , NameToString , ToStringMonotonicity > ;
2019-08-08 08:41:38 +00:00
using FunctionToUnixTimestamp = FunctionConvert < DataTypeUInt32 , NameToUnixTimestamp , ToNumberMonotonicity < UInt32 > > ;
2018-08-22 17:26:11 +00:00
using FunctionToDecimal32 = FunctionConvert < DataTypeDecimal < Decimal32 > , NameToDecimal32 , UnknownMonotonicity > ;
using FunctionToDecimal64 = FunctionConvert < DataTypeDecimal < Decimal64 > , NameToDecimal64 , UnknownMonotonicity > ;
using FunctionToDecimal128 = FunctionConvert < DataTypeDecimal < Decimal128 > , NameToDecimal128 , UnknownMonotonicity > ;
2020-08-19 11:52:17 +00:00
using FunctionToDecimal256 = FunctionConvert < DataTypeDecimal < Decimal256 > , NameToDecimal256 , UnknownMonotonicity > ;
2017-10-30 02:18:06 +00:00
2011-10-15 23:40:56 +00:00
2015-12-29 13:51:12 +00:00
template < typename DataType > struct FunctionTo ;
2017-10-30 02:18:06 +00:00
2015-12-29 13:51:12 +00:00
template < > struct FunctionTo < DataTypeUInt8 > { using Type = FunctionToUInt8 ; } ;
template < > struct FunctionTo < DataTypeUInt16 > { using Type = FunctionToUInt16 ; } ;
template < > struct FunctionTo < DataTypeUInt32 > { using Type = FunctionToUInt32 ; } ;
template < > struct FunctionTo < DataTypeUInt64 > { using Type = FunctionToUInt64 ; } ;
2021-05-03 15:41:37 +00:00
template < > struct FunctionTo < DataTypeUInt128 > { using Type = FunctionToUInt128 ; } ;
2020-08-19 11:52:17 +00:00
template < > struct FunctionTo < DataTypeUInt256 > { using Type = FunctionToUInt256 ; } ;
2015-12-29 13:51:12 +00:00
template < > struct FunctionTo < DataTypeInt8 > { using Type = FunctionToInt8 ; } ;
template < > struct FunctionTo < DataTypeInt16 > { using Type = FunctionToInt16 ; } ;
template < > struct FunctionTo < DataTypeInt32 > { using Type = FunctionToInt32 ; } ;
template < > struct FunctionTo < DataTypeInt64 > { using Type = FunctionToInt64 ; } ;
2020-08-19 11:52:17 +00:00
template < > struct FunctionTo < DataTypeInt128 > { using Type = FunctionToInt128 ; } ;
template < > struct FunctionTo < DataTypeInt256 > { using Type = FunctionToInt256 ; } ;
2015-12-29 13:51:12 +00:00
template < > struct FunctionTo < DataTypeFloat32 > { using Type = FunctionToFloat32 ; } ;
template < > struct FunctionTo < DataTypeFloat64 > { using Type = FunctionToFloat64 ; } ;
template < > struct FunctionTo < DataTypeDate > { using Type = FunctionToDate ; } ;
2021-07-15 11:40:45 +00:00
template < > struct FunctionTo < DataTypeDate32 > { using Type = FunctionToDate32 ; } ;
2015-12-29 13:51:12 +00:00
template < > struct FunctionTo < DataTypeDateTime > { using Type = FunctionToDateTime ; } ;
2019-10-12 07:03:38 +00:00
template < > struct FunctionTo < DataTypeDateTime64 > { using Type = FunctionToDateTime64 ; } ;
2017-07-04 10:42:53 +00:00
template < > struct FunctionTo < DataTypeUUID > { using Type = FunctionToUUID ; } ;
2015-12-29 13:51:12 +00:00
template < > struct FunctionTo < DataTypeString > { using Type = FunctionToString ; } ;
template < > struct FunctionTo < DataTypeFixedString > { using Type = FunctionToFixedString ; } ;
2018-08-22 16:16:39 +00:00
template < > struct FunctionTo < DataTypeDecimal < Decimal32 > > { using Type = FunctionToDecimal32 ; } ;
template < > struct FunctionTo < DataTypeDecimal < Decimal64 > > { using Type = FunctionToDecimal64 ; } ;
template < > struct FunctionTo < DataTypeDecimal < Decimal128 > > { using Type = FunctionToDecimal128 ; } ;
2020-08-19 11:52:17 +00:00
template < > struct FunctionTo < DataTypeDecimal < Decimal256 > > { using Type = FunctionToDecimal256 ; } ;
2017-10-30 02:18:06 +00:00
2015-12-29 13:51:12 +00:00
template < typename FieldType > struct FunctionTo < DataTypeEnum < FieldType > >
2017-04-01 07:20:54 +00:00
: FunctionTo < DataTypeNumber < FieldType > >
2015-12-29 13:51:12 +00:00
{
} ;
2015-12-25 15:43:38 +00:00
2017-10-30 02:18:06 +00:00
struct NameToUInt8OrZero { static constexpr auto name = " toUInt8OrZero " ; } ;
struct NameToUInt16OrZero { static constexpr auto name = " toUInt16OrZero " ; } ;
struct NameToUInt32OrZero { static constexpr auto name = " toUInt32OrZero " ; } ;
struct NameToUInt64OrZero { static constexpr auto name = " toUInt64OrZero " ; } ;
2021-05-03 15:41:37 +00:00
struct NameToUInt128OrZero { static constexpr auto name = " toUInt128OrZero " ; } ;
2020-08-19 11:52:17 +00:00
struct NameToUInt256OrZero { static constexpr auto name = " toUInt256OrZero " ; } ;
2017-10-30 02:18:06 +00:00
struct NameToInt8OrZero { static constexpr auto name = " toInt8OrZero " ; } ;
struct NameToInt16OrZero { static constexpr auto name = " toInt16OrZero " ; } ;
struct NameToInt32OrZero { static constexpr auto name = " toInt32OrZero " ; } ;
struct NameToInt64OrZero { static constexpr auto name = " toInt64OrZero " ; } ;
2020-08-19 11:52:17 +00:00
struct NameToInt128OrZero { static constexpr auto name = " toInt128OrZero " ; } ;
struct NameToInt256OrZero { static constexpr auto name = " toInt256OrZero " ; } ;
2017-10-30 02:18:06 +00:00
struct NameToFloat32OrZero { static constexpr auto name = " toFloat32OrZero " ; } ;
struct NameToFloat64OrZero { static constexpr auto name = " toFloat64OrZero " ; } ;
2018-02-11 23:57:07 +00:00
struct NameToDateOrZero { static constexpr auto name = " toDateOrZero " ; } ;
2021-07-15 11:41:52 +00:00
struct NameToDate32OrZero { static constexpr auto name = " toDate32OrZero " ; } ;
2018-02-11 23:57:07 +00:00
struct NameToDateTimeOrZero { static constexpr auto name = " toDateTimeOrZero " ; } ;
2019-10-12 07:03:38 +00:00
struct NameToDateTime64OrZero { static constexpr auto name = " toDateTime64OrZero " ; } ;
2019-05-15 18:50:35 +00:00
struct NameToDecimal32OrZero { static constexpr auto name = " toDecimal32OrZero " ; } ;
struct NameToDecimal64OrZero { static constexpr auto name = " toDecimal64OrZero " ; } ;
struct NameToDecimal128OrZero { static constexpr auto name = " toDecimal128OrZero " ; } ;
2020-08-19 11:52:17 +00:00
struct NameToDecimal256OrZero { static constexpr auto name = " toDecimal256OrZero " ; } ;
2020-10-25 08:45:29 +00:00
struct NameToUUIDOrZero { static constexpr auto name = " toUUIDOrZero " ; } ;
2018-02-11 23:57:07 +00:00
using FunctionToUInt8OrZero = FunctionConvertFromString < DataTypeUInt8 , NameToUInt8OrZero , ConvertFromStringExceptionMode : : Zero > ;
using FunctionToUInt16OrZero = FunctionConvertFromString < DataTypeUInt16 , NameToUInt16OrZero , ConvertFromStringExceptionMode : : Zero > ;
using FunctionToUInt32OrZero = FunctionConvertFromString < DataTypeUInt32 , NameToUInt32OrZero , ConvertFromStringExceptionMode : : Zero > ;
using FunctionToUInt64OrZero = FunctionConvertFromString < DataTypeUInt64 , NameToUInt64OrZero , ConvertFromStringExceptionMode : : Zero > ;
2021-05-03 15:41:37 +00:00
using FunctionToUInt128OrZero = FunctionConvertFromString < DataTypeUInt128 , NameToUInt128OrZero , ConvertFromStringExceptionMode : : Zero > ;
2020-08-19 11:52:17 +00:00
using FunctionToUInt256OrZero = FunctionConvertFromString < DataTypeUInt256 , NameToUInt256OrZero , ConvertFromStringExceptionMode : : Zero > ;
2018-02-11 23:57:07 +00:00
using FunctionToInt8OrZero = FunctionConvertFromString < DataTypeInt8 , NameToInt8OrZero , ConvertFromStringExceptionMode : : Zero > ;
using FunctionToInt16OrZero = FunctionConvertFromString < DataTypeInt16 , NameToInt16OrZero , ConvertFromStringExceptionMode : : Zero > ;
using FunctionToInt32OrZero = FunctionConvertFromString < DataTypeInt32 , NameToInt32OrZero , ConvertFromStringExceptionMode : : Zero > ;
using FunctionToInt64OrZero = FunctionConvertFromString < DataTypeInt64 , NameToInt64OrZero , ConvertFromStringExceptionMode : : Zero > ;
2020-08-19 11:52:17 +00:00
using FunctionToInt128OrZero = FunctionConvertFromString < DataTypeInt128 , NameToInt128OrZero , ConvertFromStringExceptionMode : : Zero > ;
using FunctionToInt256OrZero = FunctionConvertFromString < DataTypeInt256 , NameToInt256OrZero , ConvertFromStringExceptionMode : : Zero > ;
2018-02-11 23:57:07 +00:00
using FunctionToFloat32OrZero = FunctionConvertFromString < DataTypeFloat32 , NameToFloat32OrZero , ConvertFromStringExceptionMode : : Zero > ;
using FunctionToFloat64OrZero = FunctionConvertFromString < DataTypeFloat64 , NameToFloat64OrZero , ConvertFromStringExceptionMode : : Zero > ;
using FunctionToDateOrZero = FunctionConvertFromString < DataTypeDate , NameToDateOrZero , ConvertFromStringExceptionMode : : Zero > ;
2021-07-15 11:41:52 +00:00
using FunctionToDate32OrZero = FunctionConvertFromString < DataTypeDate32 , NameToDate32OrZero , ConvertFromStringExceptionMode : : Zero > ;
2018-02-11 23:57:07 +00:00
using FunctionToDateTimeOrZero = FunctionConvertFromString < DataTypeDateTime , NameToDateTimeOrZero , ConvertFromStringExceptionMode : : Zero > ;
2019-10-12 07:03:38 +00:00
using FunctionToDateTime64OrZero = FunctionConvertFromString < DataTypeDateTime64 , NameToDateTime64OrZero , ConvertFromStringExceptionMode : : Zero > ;
2019-05-15 18:50:35 +00:00
using FunctionToDecimal32OrZero = FunctionConvertFromString < DataTypeDecimal < Decimal32 > , NameToDecimal32OrZero , ConvertFromStringExceptionMode : : Zero > ;
using FunctionToDecimal64OrZero = FunctionConvertFromString < DataTypeDecimal < Decimal64 > , NameToDecimal64OrZero , ConvertFromStringExceptionMode : : Zero > ;
using FunctionToDecimal128OrZero = FunctionConvertFromString < DataTypeDecimal < Decimal128 > , NameToDecimal128OrZero , ConvertFromStringExceptionMode : : Zero > ;
2020-08-19 11:52:17 +00:00
using FunctionToDecimal256OrZero = FunctionConvertFromString < DataTypeDecimal < Decimal256 > , NameToDecimal256OrZero , ConvertFromStringExceptionMode : : Zero > ;
2020-10-25 08:45:29 +00:00
using FunctionToUUIDOrZero = FunctionConvertFromString < DataTypeUUID , NameToUUIDOrZero , ConvertFromStringExceptionMode : : Zero > ;
2016-08-02 05:32:09 +00:00
2017-12-25 05:22:33 +00:00
struct NameToUInt8OrNull { static constexpr auto name = " toUInt8OrNull " ; } ;
struct NameToUInt16OrNull { static constexpr auto name = " toUInt16OrNull " ; } ;
struct NameToUInt32OrNull { static constexpr auto name = " toUInt32OrNull " ; } ;
struct NameToUInt64OrNull { static constexpr auto name = " toUInt64OrNull " ; } ;
2021-05-03 15:41:37 +00:00
struct NameToUInt128OrNull { static constexpr auto name = " toUInt128OrNull " ; } ;
2020-08-19 11:52:17 +00:00
struct NameToUInt256OrNull { static constexpr auto name = " toUInt256OrNull " ; } ;
2017-12-25 05:22:33 +00:00
struct NameToInt8OrNull { static constexpr auto name = " toInt8OrNull " ; } ;
struct NameToInt16OrNull { static constexpr auto name = " toInt16OrNull " ; } ;
struct NameToInt32OrNull { static constexpr auto name = " toInt32OrNull " ; } ;
struct NameToInt64OrNull { static constexpr auto name = " toInt64OrNull " ; } ;
2020-08-19 11:52:17 +00:00
struct NameToInt128OrNull { static constexpr auto name = " toInt128OrNull " ; } ;
struct NameToInt256OrNull { static constexpr auto name = " toInt256OrNull " ; } ;
2017-12-25 05:22:33 +00:00
struct NameToFloat32OrNull { static constexpr auto name = " toFloat32OrNull " ; } ;
struct NameToFloat64OrNull { static constexpr auto name = " toFloat64OrNull " ; } ;
2018-02-11 23:57:07 +00:00
struct NameToDateOrNull { static constexpr auto name = " toDateOrNull " ; } ;
2021-07-15 11:41:52 +00:00
struct NameToDate32OrNull { static constexpr auto name = " toDate32OrNull " ; } ;
2018-02-11 23:57:07 +00:00
struct NameToDateTimeOrNull { static constexpr auto name = " toDateTimeOrNull " ; } ;
2019-10-12 07:03:38 +00:00
struct NameToDateTime64OrNull { static constexpr auto name = " toDateTime64OrNull " ; } ;
2019-05-15 18:50:35 +00:00
struct NameToDecimal32OrNull { static constexpr auto name = " toDecimal32OrNull " ; } ;
struct NameToDecimal64OrNull { static constexpr auto name = " toDecimal64OrNull " ; } ;
struct NameToDecimal128OrNull { static constexpr auto name = " toDecimal128OrNull " ; } ;
2020-08-19 11:52:17 +00:00
struct NameToDecimal256OrNull { static constexpr auto name = " toDecimal256OrNull " ; } ;
2020-10-25 08:45:29 +00:00
struct NameToUUIDOrNull { static constexpr auto name = " toUUIDOrNull " ; } ;
2018-02-11 23:57:07 +00:00
using FunctionToUInt8OrNull = FunctionConvertFromString < DataTypeUInt8 , NameToUInt8OrNull , ConvertFromStringExceptionMode : : Null > ;
using FunctionToUInt16OrNull = FunctionConvertFromString < DataTypeUInt16 , NameToUInt16OrNull , ConvertFromStringExceptionMode : : Null > ;
using FunctionToUInt32OrNull = FunctionConvertFromString < DataTypeUInt32 , NameToUInt32OrNull , ConvertFromStringExceptionMode : : Null > ;
using FunctionToUInt64OrNull = FunctionConvertFromString < DataTypeUInt64 , NameToUInt64OrNull , ConvertFromStringExceptionMode : : Null > ;
2021-05-03 15:41:37 +00:00
using FunctionToUInt128OrNull = FunctionConvertFromString < DataTypeUInt128 , NameToUInt128OrNull , ConvertFromStringExceptionMode : : Null > ;
2020-08-19 11:52:17 +00:00
using FunctionToUInt256OrNull = FunctionConvertFromString < DataTypeUInt256 , NameToUInt256OrNull , ConvertFromStringExceptionMode : : Null > ;
2018-02-11 23:57:07 +00:00
using FunctionToInt8OrNull = FunctionConvertFromString < DataTypeInt8 , NameToInt8OrNull , ConvertFromStringExceptionMode : : Null > ;
using FunctionToInt16OrNull = FunctionConvertFromString < DataTypeInt16 , NameToInt16OrNull , ConvertFromStringExceptionMode : : Null > ;
using FunctionToInt32OrNull = FunctionConvertFromString < DataTypeInt32 , NameToInt32OrNull , ConvertFromStringExceptionMode : : Null > ;
using FunctionToInt64OrNull = FunctionConvertFromString < DataTypeInt64 , NameToInt64OrNull , ConvertFromStringExceptionMode : : Null > ;
2020-08-19 11:52:17 +00:00
using FunctionToInt128OrNull = FunctionConvertFromString < DataTypeInt128 , NameToInt128OrNull , ConvertFromStringExceptionMode : : Null > ;
using FunctionToInt256OrNull = FunctionConvertFromString < DataTypeInt256 , NameToInt256OrNull , ConvertFromStringExceptionMode : : Null > ;
2018-02-11 23:57:07 +00:00
using FunctionToFloat32OrNull = FunctionConvertFromString < DataTypeFloat32 , NameToFloat32OrNull , ConvertFromStringExceptionMode : : Null > ;
using FunctionToFloat64OrNull = FunctionConvertFromString < DataTypeFloat64 , NameToFloat64OrNull , ConvertFromStringExceptionMode : : Null > ;
using FunctionToDateOrNull = FunctionConvertFromString < DataTypeDate , NameToDateOrNull , ConvertFromStringExceptionMode : : Null > ;
2021-07-15 11:41:52 +00:00
using FunctionToDate32OrNull = FunctionConvertFromString < DataTypeDate32 , NameToDate32OrNull , ConvertFromStringExceptionMode : : Null > ;
2018-02-11 23:57:07 +00:00
using FunctionToDateTimeOrNull = FunctionConvertFromString < DataTypeDateTime , NameToDateTimeOrNull , ConvertFromStringExceptionMode : : Null > ;
2019-10-12 07:03:38 +00:00
using FunctionToDateTime64OrNull = FunctionConvertFromString < DataTypeDateTime64 , NameToDateTime64OrNull , ConvertFromStringExceptionMode : : Null > ;
2019-05-15 18:50:35 +00:00
using FunctionToDecimal32OrNull = FunctionConvertFromString < DataTypeDecimal < Decimal32 > , NameToDecimal32OrNull , ConvertFromStringExceptionMode : : Null > ;
using FunctionToDecimal64OrNull = FunctionConvertFromString < DataTypeDecimal < Decimal64 > , NameToDecimal64OrNull , ConvertFromStringExceptionMode : : Null > ;
using FunctionToDecimal128OrNull = FunctionConvertFromString < DataTypeDecimal < Decimal128 > , NameToDecimal128OrNull , ConvertFromStringExceptionMode : : Null > ;
2020-08-19 11:52:17 +00:00
using FunctionToDecimal256OrNull = FunctionConvertFromString < DataTypeDecimal < Decimal256 > , NameToDecimal256OrNull , ConvertFromStringExceptionMode : : Null > ;
2020-10-25 08:45:29 +00:00
using FunctionToUUIDOrNull = FunctionConvertFromString < DataTypeUUID , NameToUUIDOrNull , ConvertFromStringExceptionMode : : Null > ;
2018-02-11 23:57:07 +00:00
struct NameParseDateTimeBestEffort { static constexpr auto name = " parseDateTimeBestEffort " ; } ;
struct NameParseDateTimeBestEffortOrZero { static constexpr auto name = " parseDateTimeBestEffortOrZero " ; } ;
struct NameParseDateTimeBestEffortOrNull { static constexpr auto name = " parseDateTimeBestEffortOrNull " ; } ;
2021-01-27 14:10:06 +00:00
struct NameParseDateTimeBestEffortUS { static constexpr auto name = " parseDateTimeBestEffortUS " ; } ;
struct NameParseDateTimeBestEffortUSOrZero { static constexpr auto name = " parseDateTimeBestEffortUSOrZero " ; } ;
struct NameParseDateTimeBestEffortUSOrNull { static constexpr auto name = " parseDateTimeBestEffortUSOrNull " ; } ;
2020-08-20 14:41:03 +00:00
struct NameParseDateTime32BestEffort { static constexpr auto name = " parseDateTime32BestEffort " ; } ;
struct NameParseDateTime32BestEffortOrZero { static constexpr auto name = " parseDateTime32BestEffortOrZero " ; } ;
struct NameParseDateTime32BestEffortOrNull { static constexpr auto name = " parseDateTime32BestEffortOrNull " ; } ;
2019-10-12 07:03:38 +00:00
struct NameParseDateTime64BestEffort { static constexpr auto name = " parseDateTime64BestEffort " ; } ;
struct NameParseDateTime64BestEffortOrZero { static constexpr auto name = " parseDateTime64BestEffortOrZero " ; } ;
struct NameParseDateTime64BestEffortOrNull { static constexpr auto name = " parseDateTime64BestEffortOrNull " ; } ;
2018-02-11 23:57:07 +00:00
using FunctionParseDateTimeBestEffort = FunctionConvertFromString <
DataTypeDateTime , NameParseDateTimeBestEffort , ConvertFromStringExceptionMode : : Throw , ConvertFromStringParsingMode : : BestEffort > ;
using FunctionParseDateTimeBestEffortOrZero = FunctionConvertFromString <
DataTypeDateTime , NameParseDateTimeBestEffortOrZero , ConvertFromStringExceptionMode : : Zero , ConvertFromStringParsingMode : : BestEffort > ;
using FunctionParseDateTimeBestEffortOrNull = FunctionConvertFromString <
DataTypeDateTime , NameParseDateTimeBestEffortOrNull , ConvertFromStringExceptionMode : : Null , ConvertFromStringParsingMode : : BestEffort > ;
2017-12-25 05:22:33 +00:00
2021-01-27 14:10:06 +00:00
using FunctionParseDateTimeBestEffortUS = FunctionConvertFromString <
DataTypeDateTime , NameParseDateTimeBestEffortUS , ConvertFromStringExceptionMode : : Throw , ConvertFromStringParsingMode : : BestEffortUS > ;
using FunctionParseDateTimeBestEffortUSOrZero = FunctionConvertFromString <
DataTypeDateTime , NameParseDateTimeBestEffortUSOrZero , ConvertFromStringExceptionMode : : Zero , ConvertFromStringParsingMode : : BestEffortUS > ;
using FunctionParseDateTimeBestEffortUSOrNull = FunctionConvertFromString <
DataTypeDateTime , NameParseDateTimeBestEffortUSOrNull , ConvertFromStringExceptionMode : : Null , ConvertFromStringParsingMode : : BestEffortUS > ;
2020-08-21 05:06:06 +00:00
using FunctionParseDateTime32BestEffort = FunctionConvertFromString <
DataTypeDateTime , NameParseDateTime32BestEffort , ConvertFromStringExceptionMode : : Throw , ConvertFromStringParsingMode : : BestEffort > ;
using FunctionParseDateTime32BestEffortOrZero = FunctionConvertFromString <
DataTypeDateTime , NameParseDateTime32BestEffortOrZero , ConvertFromStringExceptionMode : : Zero , ConvertFromStringParsingMode : : BestEffort > ;
using FunctionParseDateTime32BestEffortOrNull = FunctionConvertFromString <
DataTypeDateTime , NameParseDateTime32BestEffortOrNull , ConvertFromStringExceptionMode : : Null , ConvertFromStringParsingMode : : BestEffort > ;
2019-10-12 07:03:38 +00:00
using FunctionParseDateTime64BestEffort = FunctionConvertFromString <
DataTypeDateTime64 , NameParseDateTime64BestEffort , ConvertFromStringExceptionMode : : Throw , ConvertFromStringParsingMode : : BestEffort > ;
using FunctionParseDateTime64BestEffortOrZero = FunctionConvertFromString <
DataTypeDateTime64 , NameParseDateTime64BestEffortOrZero , ConvertFromStringExceptionMode : : Zero , ConvertFromStringParsingMode : : BestEffort > ;
using FunctionParseDateTime64BestEffortOrNull = FunctionConvertFromString <
DataTypeDateTime64 , NameParseDateTime64BestEffortOrNull , ConvertFromStringExceptionMode : : Null , ConvertFromStringParsingMode : : BestEffort > ;
2015-12-25 15:43:38 +00:00
2021-05-15 17:33:15 +00:00
class ExecutableFunctionCast : public IExecutableFunction
2015-12-23 08:11:11 +00:00
{
2017-12-07 12:09:55 +00:00
public :
2020-10-20 13:11:57 +00:00
using WrapperType = std : : function < ColumnPtr ( ColumnsWithTypeAndName & , const DataTypePtr & , const ColumnNullable * , size_t ) > ;
2018-02-02 08:33:36 +00:00
2020-11-18 09:35:32 +00:00
struct Diagnostic
{
std : : string column_from ;
std : : string column_to ;
} ;
explicit ExecutableFunctionCast (
WrapperType & & wrapper_function_ , const char * name_ , std : : optional < Diagnostic > diagnostic_ )
: wrapper_function ( std : : move ( wrapper_function_ ) ) , name ( name_ ) , diagnostic ( std : : move ( diagnostic_ ) ) { }
2018-02-02 08:33:36 +00:00
String getName ( ) const override { return name ; }
protected :
2021-05-15 17:33:15 +00:00
ColumnPtr executeImpl ( const ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , size_t input_rows_count ) const override
2018-02-02 08:33:36 +00:00
{
/// drop second argument, pass others
2020-10-17 14:23:37 +00:00
ColumnsWithTypeAndName new_arguments { arguments . front ( ) } ;
2018-02-02 08:33:36 +00:00
if ( arguments . size ( ) > 2 )
new_arguments . insert ( std : : end ( new_arguments ) , std : : next ( std : : begin ( arguments ) , 2 ) , std : : end ( arguments ) ) ;
2020-11-18 09:35:32 +00:00
try
{
return wrapper_function ( new_arguments , result_type , nullptr , input_rows_count ) ;
}
catch ( Exception & e )
{
if ( diagnostic )
e . addMessage ( " while converting source column " + backQuoteIfNeed ( diagnostic - > column_from ) +
" to destination column " + backQuoteIfNeed ( diagnostic - > column_to ) ) ;
throw ;
}
2018-02-02 08:33:36 +00:00
}
bool useDefaultImplementationForNulls ( ) const override { return false ; }
bool useDefaultImplementationForConstants ( ) const override { return true ; }
2018-09-27 15:55:22 +00:00
bool useDefaultImplementationForLowCardinalityColumns ( ) const override { return false ; }
2018-02-02 08:33:36 +00:00
ColumnNumbers getArgumentsThatAreAlwaysConstant ( ) const override { return { 1 } ; }
2017-12-07 12:09:55 +00:00
private :
2018-02-02 08:33:36 +00:00
WrapperType wrapper_function ;
const char * name ;
2020-11-18 09:35:32 +00:00
std : : optional < Diagnostic > diagnostic ;
2018-02-02 08:33:36 +00:00
} ;
2018-07-11 19:51:18 +00:00
struct NameCast { static constexpr auto name = " CAST " ; } ;
2020-12-05 14:55:37 +00:00
enum class CastType
{
2020-12-05 13:18:56 +00:00
nonAccurate ,
accurate ,
accurateOrNull
} ;
2021-05-15 17:33:15 +00:00
class FunctionCast final : public IFunctionBase
2018-02-02 08:33:36 +00:00
{
public :
2020-10-20 13:11:57 +00:00
using WrapperType = std : : function < ColumnPtr ( ColumnsWithTypeAndName & , const DataTypePtr & , const ColumnNullable * , size_t ) > ;
2018-02-06 19:34:53 +00:00
using MonotonicityForRange = std : : function < Monotonicity ( const IDataType & , const Field & , const Field & ) > ;
2020-11-18 09:35:32 +00:00
using Diagnostic = ExecutableFunctionCast : : Diagnostic ;
2018-02-02 08:33:36 +00:00
2019-12-25 19:17:41 +00:00
FunctionCast ( const char * name_ , MonotonicityForRange & & monotonicity_for_range_
2020-11-05 19:09:17 +00:00
, const DataTypes & argument_types_ , const DataTypePtr & return_type_
2020-12-05 13:18:56 +00:00
, std : : optional < Diagnostic > diagnostic_ , CastType cast_type_ )
2020-11-05 19:09:17 +00:00
: name ( name_ ) , monotonicity_for_range ( std : : move ( monotonicity_for_range_ ) )
2020-11-18 09:35:32 +00:00
, argument_types ( argument_types_ ) , return_type ( return_type_ ) , diagnostic ( std : : move ( diagnostic_ ) )
2020-12-05 13:18:56 +00:00
, cast_type ( cast_type_ )
2018-02-02 08:33:36 +00:00
{
}
const DataTypes & getArgumentTypes ( ) const override { return argument_types ; }
2020-10-17 14:23:37 +00:00
const DataTypePtr & getResultType ( ) const override { return return_type ; }
2018-02-02 08:33:36 +00:00
2021-05-15 17:33:15 +00:00
ExecutableFunctionPtr prepare ( const ColumnsWithTypeAndName & /*sample_columns*/ ) const override
2018-02-02 08:33:36 +00:00
{
2020-11-18 09:45:46 +00:00
try
{
return std : : make_unique < ExecutableFunctionCast > (
prepareUnpackDictionaries ( getArgumentTypes ( ) [ 0 ] , getResultType ( ) ) , name , diagnostic ) ;
}
catch ( Exception & e )
{
if ( diagnostic )
e . addMessage ( " while converting source column " + backQuoteIfNeed ( diagnostic - > column_from ) +
" to destination column " + backQuoteIfNeed ( diagnostic - > column_to ) ) ;
throw ;
}
2018-02-02 08:33:36 +00:00
}
String getName ( ) const override { return name ; }
2019-10-10 14:38:08 +00:00
bool isDeterministic ( ) const override { return true ; }
bool isDeterministicInScopeOfQuery ( ) const override { return true ; }
2021-04-29 14:48:26 +00:00
bool isSuitableForShortCircuitArgumentsExecution ( ) const override { return false ; }
2019-10-10 14:38:08 +00:00
2018-02-02 08:33:36 +00:00
bool hasInformationAboutMonotonicity ( ) const override
{
return static_cast < bool > ( monotonicity_for_range ) ;
}
Monotonicity getMonotonicityForRange ( const IDataType & type , const Field & left , const Field & right ) const override
{
return monotonicity_for_range ( type , left , right ) ;
}
private :
const char * name ;
MonotonicityForRange monotonicity_for_range ;
DataTypes argument_types ;
DataTypePtr return_type ;
2015-12-23 08:11:11 +00:00
2020-11-18 09:35:32 +00:00
std : : optional < Diagnostic > diagnostic ;
2020-12-05 13:18:56 +00:00
CastType cast_type ;
2020-11-18 09:35:32 +00:00
2021-04-10 23:33:54 +00:00
static WrapperType createFunctionAdaptor ( FunctionPtr function , const DataTypePtr & from_type )
2017-04-01 07:20:54 +00:00
{
2021-05-15 17:33:15 +00:00
auto function_adaptor = std : : make_unique < FunctionToOverloadResolverAdaptor > ( function ) - > build ( { ColumnWithTypeAndName { nullptr , from_type , " " } } ) ;
2017-04-01 07:20:54 +00:00
2020-11-10 13:18:58 +00:00
return [ function_adaptor ]
( ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , const ColumnNullable * , size_t input_rows_count )
2018-07-11 19:51:18 +00:00
{
2020-10-17 14:23:37 +00:00
return function_adaptor - > execute ( arguments , result_type , input_rows_count ) ;
2018-07-11 19:51:18 +00:00
} ;
}
2020-12-05 13:18:56 +00:00
static WrapperType createToNullableColumnWrapper ( )
2018-07-11 19:51:18 +00:00
{
2020-11-05 19:09:17 +00:00
return [ ] ( ColumnsWithTypeAndName & , const DataTypePtr & result_type , const ColumnNullable * , size_t input_rows_count )
2017-11-05 05:32:22 +00:00
{
2020-11-05 19:09:17 +00:00
ColumnPtr res = result_type - > createColumn ( ) ;
ColumnUInt8 : : Ptr col_null_map_to = ColumnUInt8 : : create ( input_rows_count , true ) ;
return ColumnNullable : : create ( res - > cloneResized ( input_rows_count ) , std : : move ( col_null_map_to ) ) ;
2017-04-01 07:20:54 +00:00
} ;
}
2020-11-05 19:09:17 +00:00
template < typename ToDataType >
2020-11-12 11:27:02 +00:00
WrapperType createWrapper ( const DataTypePtr & from_type , const ToDataType * const to_type , bool requested_result_is_nullable ) const
2017-04-01 07:20:54 +00:00
{
2020-11-05 19:09:17 +00:00
TypeIndex from_type_index = from_type - > getTypeId ( ) ;
WhichDataType which ( from_type_index ) ;
2020-12-05 13:18:56 +00:00
bool can_apply_accurate_cast = ( cast_type = = CastType : : accurate | | cast_type = = CastType : : accurateOrNull )
& & ( which . isInt ( ) | | which . isUInt ( ) | | which . isFloat ( ) ) ;
2017-04-01 07:20:54 +00:00
2020-11-05 19:09:17 +00:00
if ( requested_result_is_nullable & & checkAndGetDataType < DataTypeString > ( from_type . get ( ) ) )
{
/// In case when converting to Nullable type, we apply different parsing rule,
/// that will not throw an exception but return NULL in case of malformed input.
FunctionPtr function = FunctionConvertFromString < ToDataType , NameCast , ConvertFromStringExceptionMode : : Null > : : create ( ) ;
return createFunctionAdaptor ( function , from_type ) ;
}
else if ( ! can_apply_accurate_cast )
2017-04-01 07:20:54 +00:00
{
2020-11-05 19:09:17 +00:00
FunctionPtr function = FunctionTo < ToDataType > : : Type : : create ( ) ;
return createFunctionAdaptor ( function , from_type ) ;
}
2017-04-01 07:20:54 +00:00
2020-12-05 13:18:56 +00:00
auto wrapper_cast_type = cast_type ;
2020-11-12 11:27:02 +00:00
2020-12-05 13:18:56 +00:00
return [ wrapper_cast_type , from_type_index , to_type ]
2020-11-15 19:24:15 +00:00
( ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , const ColumnNullable * column_nullable , size_t input_rows_count )
2017-04-01 07:20:54 +00:00
{
2020-11-05 19:09:17 +00:00
ColumnPtr result_column ;
2020-11-12 15:56:17 +00:00
auto res = callOnIndexAndDataType < ToDataType > ( from_type_index , [ & ] ( const auto & types ) - > bool {
2020-11-05 19:09:17 +00:00
using Types = std : : decay_t < decltype ( types ) > ;
using LeftDataType = typename Types : : LeftType ;
using RightDataType = typename Types : : RightType ;
2020-11-10 13:18:58 +00:00
if constexpr ( IsDataTypeNumber < LeftDataType > & & IsDataTypeNumber < RightDataType > )
2020-11-05 19:09:17 +00:00
{
2020-12-05 13:18:56 +00:00
if ( wrapper_cast_type = = CastType : : accurate )
{
result_column = ConvertImpl < LeftDataType , RightDataType , NameCast > : : execute (
arguments , result_type , input_rows_count , AccurateConvertStrategyAdditions ( ) ) ;
}
else
{
result_column = ConvertImpl < LeftDataType , RightDataType , NameCast > : : execute (
arguments , result_type , input_rows_count , AccurateOrNullConvertStrategyAdditions ( ) ) ;
}
2020-11-05 19:09:17 +00:00
return true ;
}
return false ;
} ) ;
/// Additionally check if callOnIndexAndDataType wasn't called at all.
if ( ! res )
{
2020-12-05 13:18:56 +00:00
if ( wrapper_cast_type = = CastType : : accurateOrNull )
{
auto nullable_column_wrapper = FunctionCast : : createToNullableColumnWrapper ( ) ;
2020-11-12 11:27:02 +00:00
return nullable_column_wrapper ( arguments , result_type , column_nullable , input_rows_count ) ;
2020-12-05 13:18:56 +00:00
}
2020-11-12 11:27:02 +00:00
else
{
2020-11-12 15:56:17 +00:00
throw Exception { " Conversion from " + std : : string ( getTypeName ( from_type_index ) ) + " to " + to_type - > getName ( ) + " is not supported " ,
2020-11-12 11:27:02 +00:00
ErrorCodes : : CANNOT_CONVERT_TYPE } ;
}
2020-11-05 19:09:17 +00:00
}
return result_column ;
2017-04-01 07:20:54 +00:00
} ;
}
2021-04-10 23:33:54 +00:00
static WrapperType createStringWrapper ( const DataTypePtr & from_type )
2018-09-07 16:38:40 +00:00
{
2020-11-05 19:09:17 +00:00
FunctionPtr function = FunctionToString : : create ( ) ;
return createFunctionAdaptor ( function , from_type ) ;
}
2018-09-07 16:38:40 +00:00
2020-11-05 19:09:17 +00:00
WrapperType createFixedStringWrapper ( const DataTypePtr & from_type , const size_t N ) const
{
if ( ! isStringOrFixedString ( from_type ) )
throw Exception { " CAST AS FixedString is only implemented for types String and FixedString " , ErrorCodes : : NOT_IMPLEMENTED } ;
2018-09-07 16:38:40 +00:00
2020-12-05 13:18:56 +00:00
bool exception_mode_null = cast_type = = CastType : : accurateOrNull ;
2020-11-05 19:09:17 +00:00
return [ exception_mode_null , N ] ( ColumnsWithTypeAndName & arguments , const DataTypePtr & , const ColumnNullable * , size_t /*input_rows_count*/ )
2018-09-07 16:38:40 +00:00
{
2020-11-05 19:09:17 +00:00
if ( exception_mode_null )
return FunctionToFixedString : : executeForN < ConvertToFixedStringExceptionMode : : Null > ( arguments , N ) ;
else
return FunctionToFixedString : : executeForN < ConvertToFixedStringExceptionMode : : Throw > ( arguments , N ) ;
2018-09-07 16:38:40 +00:00
} ;
}
2019-10-24 10:44:38 +00:00
template < typename ToDataType >
std : : enable_if_t < IsDataTypeDecimal < ToDataType > , WrapperType >
2021-01-04 12:40:33 +00:00
createDecimalWrapper ( const DataTypePtr & from_type , const ToDataType * to_type , bool requested_result_is_nullable ) const
2018-08-31 08:59:21 +00:00
{
TypeIndex type_index = from_type - > getTypeId ( ) ;
UInt32 scale = to_type - > getScale ( ) ;
2019-08-02 14:31:15 +00:00
WhichDataType which ( type_index ) ;
2021-07-15 11:40:45 +00:00
bool ok = which . isNativeInt ( ) | | which . isNativeUInt ( ) | | which . isDecimal ( ) | | which . isFloat ( ) | | which . isDateOrDate32 ( ) | | which . isDateTime ( ) | | which . isDateTime64 ( )
2020-11-12 15:56:17 +00:00
| | which . isStringOrFixedString ( ) ;
2019-08-02 14:31:15 +00:00
if ( ! ok )
2020-11-10 13:18:58 +00:00
{
2020-12-05 13:18:56 +00:00
if ( cast_type = = CastType : : accurateOrNull )
2020-11-05 19:09:17 +00:00
return createToNullableColumnWrapper ( ) ;
2020-11-10 13:18:58 +00:00
else
2020-11-12 15:56:17 +00:00
throw Exception { " Conversion from " + from_type - > getName ( ) + " to " + to_type - > getName ( ) + " is not supported " ,
2020-11-05 19:09:17 +00:00
ErrorCodes : : CANNOT_CONVERT_TYPE } ;
}
2019-08-02 14:31:15 +00:00
2020-12-05 13:18:56 +00:00
auto wrapper_cast_type = cast_type ;
2019-08-02 14:31:15 +00:00
2021-01-04 14:52:07 +00:00
return [ wrapper_cast_type , type_index , scale , to_type , requested_result_is_nullable ]
2020-11-12 11:27:02 +00:00
( ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , const ColumnNullable * column_nullable , size_t input_rows_count )
2018-08-31 08:59:21 +00:00
{
2020-10-17 14:23:37 +00:00
ColumnPtr result_column ;
2021-01-04 14:52:07 +00:00
auto res = callOnIndexAndDataType < ToDataType > ( type_index , [ & ] ( const auto & types ) - > bool
2021-01-04 12:40:33 +00:00
{
2018-08-31 08:59:21 +00:00
using Types = std : : decay_t < decltype ( types ) > ;
using LeftDataType = typename Types : : LeftType ;
using RightDataType = typename Types : : RightType ;
Fix DateTime64 initialization (to match DateTime behaviour)
There was no specializations for toDateTime64(<numeric>), and because of
this default decimal conversion was used, however this is not enough for
DateTime/DateTime64 types, since the date may overflow and the proper
check is required (like DateTime has), and this what UBsan found [1]:
../src/IO/WriteHelpers.h:812:33: runtime error: index 508 out of bounds for type 'const char [201]' Received signal -3 Received signal Unknown signal (-3)
Backtrace:
(gdb) bt
0 LocalDateTime::LocalDateTime (this=0x7fffffff8418, year_=1970, month_=1 '\001', day_=1 '\001', hour_=2 '\002', minute_=0 '\000', second_=254 '\376') at LocalDateTime.h:83
1 0x00000000138a5edb in DB::writeDateTimeText<(char)45, (char)58, (char)32, (char)46> (datetime64=..., scale=7, buf=..., date_lut=...) at WriteHelpers.h:852
2 0x0000000019c379b4 in DB::DataTypeDateTime64::serializeText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at DataTypeDateTime64.cpp:66
3 0x0000000019d297e4 in DB::IDataType::serializeAsText (this=0x7ffff5c4b0d8, column=..., row_num=0, ostr=..., settings=...) at IDataType.cpp:387
[1]: https://clickhouse-test-reports.s3.yandex.net/19527/cea8ae162ffbf92e5ed29304ab010704c5d611c8/fuzzer_ubsan/report.html#fail1
Also fix CAST for DateTime64
2021-02-05 19:06:23 +00:00
if constexpr ( IsDataTypeDecimalOrNumber < LeftDataType > & & IsDataTypeDecimalOrNumber < RightDataType > & & ! std : : is_same_v < DataTypeDateTime64 , RightDataType > )
2020-11-12 11:27:02 +00:00
{
2020-12-05 13:18:56 +00:00
if ( wrapper_cast_type = = CastType : : accurate )
2020-11-12 11:27:02 +00:00
{
2020-12-05 13:18:56 +00:00
AccurateConvertStrategyAdditions additions ;
2020-11-12 11:27:02 +00:00
additions . scale = scale ;
2020-12-05 13:18:56 +00:00
result_column = ConvertImpl < LeftDataType , RightDataType , NameCast > : : execute (
arguments , result_type , input_rows_count , additions ) ;
return true ;
}
else if ( wrapper_cast_type = = CastType : : accurateOrNull )
{
AccurateOrNullConvertStrategyAdditions additions ;
additions . scale = scale ;
result_column = ConvertImpl < LeftDataType , RightDataType , NameCast > : : execute (
arguments , result_type , input_rows_count , additions ) ;
2020-11-12 11:27:02 +00:00
return true ;
}
}
2021-01-04 14:52:07 +00:00
else if constexpr ( std : : is_same_v < LeftDataType , DataTypeString > )
2020-12-05 13:18:56 +00:00
{
2021-01-04 14:52:07 +00:00
if ( requested_result_is_nullable )
{
/// Consistent with CAST(Nullable(String) AS Nullable(Numbers))
/// In case when converting to Nullable type, we apply different parsing rule,
/// that will not throw an exception but return NULL in case of malformed input.
result_column = ConvertImpl < LeftDataType , RightDataType , NameCast , ConvertReturnNullOnErrorTag > : : execute (
arguments , result_type , input_rows_count , scale ) ;
return true ;
}
2020-12-05 13:18:56 +00:00
}
2020-11-12 11:27:02 +00:00
2020-10-17 14:23:37 +00:00
result_column = ConvertImpl < LeftDataType , RightDataType , NameCast > : : execute ( arguments , result_type , input_rows_count , scale ) ;
2020-11-12 11:27:02 +00:00
2018-08-31 08:59:21 +00:00
return true ;
} ) ;
2019-08-05 15:23:32 +00:00
/// Additionally check if callOnIndexAndDataType wasn't called at all.
if ( ! res )
{
2020-12-05 13:18:56 +00:00
if ( wrapper_cast_type = = CastType : : accurateOrNull )
{
auto nullable_column_wrapper = FunctionCast : : createToNullableColumnWrapper ( ) ;
2020-11-12 11:27:02 +00:00
return nullable_column_wrapper ( arguments , result_type , column_nullable , input_rows_count ) ;
2020-12-05 13:18:56 +00:00
}
2020-11-10 13:18:58 +00:00
else
2020-11-12 15:56:17 +00:00
throw Exception { " Conversion from " + std : : string ( getTypeName ( type_index ) ) + " to " + to_type - > getName ( ) + " is not supported " ,
2020-11-10 13:18:58 +00:00
ErrorCodes : : CANNOT_CONVERT_TYPE } ;
2019-08-05 15:23:32 +00:00
}
2020-10-17 14:23:37 +00:00
return result_column ;
2018-08-31 08:59:21 +00:00
} ;
}
2019-09-03 14:36:17 +00:00
2020-11-05 19:09:17 +00:00
WrapperType createAggregateFunctionWrapper ( const DataTypePtr & from_type_untyped , const DataTypeAggregateFunction * to_type ) const
2019-09-02 09:16:49 +00:00
{
/// Conversion from String through parsing.
if ( checkAndGetDataType < DataTypeString > ( from_type_untyped . get ( ) ) )
{
2020-10-20 13:11:57 +00:00
return [ ] ( ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , const ColumnNullable * , size_t /*input_rows_count*/ )
2019-09-02 09:16:49 +00:00
{
2020-10-17 14:23:37 +00:00
return ConvertImplGenericFromString : : execute ( arguments , result_type ) ;
2019-09-02 09:16:49 +00:00
} ;
}
else
2020-11-05 19:09:17 +00:00
{
2020-12-05 13:18:56 +00:00
if ( cast_type = = CastType : : accurateOrNull )
2020-11-05 19:09:17 +00:00
return createToNullableColumnWrapper ( ) ;
2020-11-15 19:24:15 +00:00
else
2020-11-05 19:09:17 +00:00
throw Exception { " Conversion from " + from_type_untyped - > getName ( ) + " to " + to_type - > getName ( ) +
" is not supported " , ErrorCodes : : CANNOT_CONVERT_TYPE } ;
}
2019-09-02 09:16:49 +00:00
}
2018-08-31 08:59:21 +00:00
2021-04-21 21:19:01 +00:00
WrapperType createArrayWrapper ( const DataTypePtr & from_type_untyped , const DataTypeArray & to_type ) const
2017-04-01 07:20:54 +00:00
{
/// Conversion from String through parsing.
2017-07-21 06:35:58 +00:00
if ( checkAndGetDataType < DataTypeString > ( from_type_untyped . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
2020-10-20 13:11:57 +00:00
return [ ] ( ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , const ColumnNullable * , size_t /*input_rows_count*/ )
2017-04-01 07:20:54 +00:00
{
2020-10-17 14:23:37 +00:00
return ConvertImplGenericFromString : : execute ( arguments , result_type ) ;
2017-04-01 07:20:54 +00:00
} ;
}
2020-10-17 14:23:37 +00:00
const auto * from_type = checkAndGetDataType < DataTypeArray > ( from_type_untyped . get ( ) ) ;
2021-04-21 21:19:01 +00:00
if ( ! from_type )
2017-04-01 07:20:54 +00:00
{
2021-04-21 21:19:01 +00:00
throw Exception ( ErrorCodes : : TYPE_MISMATCH ,
2021-05-24 23:42:13 +00:00
" CAST AS Array can only be performed between same-dimensional Array or String types " ) ;
2021-04-21 21:19:01 +00:00
}
2017-04-01 07:20:54 +00:00
2021-04-21 21:19:01 +00:00
DataTypePtr from_nested_type = from_type - > getNestedType ( ) ;
2017-04-01 07:20:54 +00:00
2021-04-21 21:19:01 +00:00
/// In query SELECT CAST([] AS Array(Array(String))) from type is Array(Nothing)
bool from_empty_array = isNothing ( from_nested_type ) ;
if ( from_type - > getNumberOfDimensions ( ) ! = to_type . getNumberOfDimensions ( ) & & ! from_empty_array )
throw Exception ( ErrorCodes : : TYPE_MISMATCH ,
2021-05-24 23:42:13 +00:00
" CAST AS Array can only be performed between same-dimensional array types " ) ;
2021-04-21 21:19:01 +00:00
const DataTypePtr & to_nested_type = to_type . getNestedType ( ) ;
2017-04-01 07:20:54 +00:00
/// Prepare nested type conversion
2018-06-07 10:37:06 +00:00
const auto nested_function = prepareUnpackDictionaries ( from_nested_type , to_nested_type ) ;
2017-04-01 07:20:54 +00:00
2017-12-08 03:52:33 +00:00
return [ nested_function , from_nested_type , to_nested_type ] (
2020-10-20 13:11:57 +00:00
ColumnsWithTypeAndName & arguments , const DataTypePtr & , const ColumnNullable * nullable_source , size_t /*input_rows_count*/ ) - > ColumnPtr
2017-04-01 07:20:54 +00:00
{
2020-10-17 14:23:37 +00:00
const auto & array_arg = arguments . front ( ) ;
2017-04-01 07:20:54 +00:00
2017-12-16 04:59:32 +00:00
if ( const ColumnArray * col_array = checkAndGetColumn < ColumnArray > ( array_arg . column . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
2020-10-14 14:04:50 +00:00
/// create columns for converting nested column containing original and result columns
2020-10-17 14:23:37 +00:00
ColumnsWithTypeAndName nested_columns { { col_array - > getDataPtr ( ) , from_nested_type , " " } } ;
2017-04-01 07:20:54 +00:00
/// convert nested column
2020-10-20 13:11:57 +00:00
auto result_column = nested_function ( nested_columns , to_nested_type , nullable_source , nested_columns . front ( ) . column - > size ( ) ) ;
2017-04-01 07:20:54 +00:00
/// set converted nested column to result
2020-10-17 14:23:37 +00:00
return ColumnArray : : create ( result_column , col_array - > getOffsetsPtr ( ) ) ;
2017-04-01 07:20:54 +00:00
}
else
2018-02-18 02:22:32 +00:00
throw Exception { " Illegal column " + array_arg . column - > getName ( ) + " for function CAST AS Array " , ErrorCodes : : LOGICAL_ERROR } ;
2017-04-01 07:20:54 +00:00
} ;
}
2020-12-14 22:16:04 +00:00
using ElementWrappers = std : : vector < WrapperType > ;
ElementWrappers getElementWrappers ( const DataTypes & from_element_types , const DataTypes & to_element_types ) const
{
ElementWrappers element_wrappers ;
element_wrappers . reserve ( from_element_types . size ( ) ) ;
/// Create conversion wrapper for each element in tuple
2021-06-15 19:55:21 +00:00
for ( size_t i = 0 ; i < from_element_types . size ( ) ; + + i )
{
const DataTypePtr & from_element_type = from_element_types [ i ] ;
const DataTypePtr & to_element_type = to_element_types [ i ] ;
element_wrappers . push_back ( prepareUnpackDictionaries ( from_element_type , to_element_type ) ) ;
}
2020-12-14 22:16:04 +00:00
return element_wrappers ;
}
2018-02-02 08:33:36 +00:00
WrapperType createTupleWrapper ( const DataTypePtr & from_type_untyped , const DataTypeTuple * to_type ) const
2017-04-01 07:20:54 +00:00
{
/// Conversion from String through parsing.
2017-07-21 06:35:58 +00:00
if ( checkAndGetDataType < DataTypeString > ( from_type_untyped . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
2020-10-20 13:11:57 +00:00
return [ ] ( ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , const ColumnNullable * , size_t /*input_rows_count*/ )
2017-04-01 07:20:54 +00:00
{
2020-10-17 14:23:37 +00:00
return ConvertImplGenericFromString : : execute ( arguments , result_type ) ;
2017-04-01 07:20:54 +00:00
} ;
}
2020-10-17 14:23:37 +00:00
const auto * from_type = checkAndGetDataType < DataTypeTuple > ( from_type_untyped . get ( ) ) ;
2017-04-01 07:20:54 +00:00
if ( ! from_type )
2018-08-31 08:59:21 +00:00
throw Exception { " CAST AS Tuple can only be performed between tuple types or from String. \n Left type: "
+ from_type_untyped - > getName ( ) + " , right type: " + to_type - > getName ( ) , ErrorCodes : : TYPE_MISMATCH } ;
2017-04-01 07:20:54 +00:00
if ( from_type - > getElements ( ) . size ( ) ! = to_type - > getElements ( ) . size ( ) )
2018-02-18 02:22:32 +00:00
throw Exception { " CAST AS Tuple can only be performed between tuple types with the same number of elements or from String. \n "
" Left type: " + from_type - > getName ( ) + " , right type: " + to_type - > getName ( ) , ErrorCodes : : TYPE_MISMATCH } ;
2017-04-01 07:20:54 +00:00
const auto & from_element_types = from_type - > getElements ( ) ;
const auto & to_element_types = to_type - > getElements ( ) ;
2020-12-14 22:16:04 +00:00
auto element_wrappers = getElementWrappers ( from_element_types , to_element_types ) ;
2017-04-01 07:20:54 +00:00
2017-12-08 00:50:25 +00:00
return [ element_wrappers , from_element_types , to_element_types ]
2020-10-20 13:11:57 +00:00
( ColumnsWithTypeAndName & arguments , const DataTypePtr & , const ColumnNullable * nullable_source , size_t input_rows_count ) - > ColumnPtr
2017-04-01 07:20:54 +00:00
{
2020-10-17 14:23:37 +00:00
const auto * col = arguments . front ( ) . column . get ( ) ;
2017-04-01 07:20:54 +00:00
2017-12-08 00:50:25 +00:00
size_t tuple_size = from_element_types . size ( ) ;
const ColumnTuple & column_tuple = typeid_cast < const ColumnTuple & > ( * col ) ;
2020-10-17 14:23:37 +00:00
Columns converted_columns ( tuple_size ) ;
2020-10-09 07:41:28 +00:00
2017-04-01 07:20:54 +00:00
/// invoke conversion for each element
2017-12-08 00:50:25 +00:00
for ( size_t i = 0 ; i < tuple_size ; + + i )
2020-10-17 14:23:37 +00:00
{
ColumnsWithTypeAndName element = { { column_tuple . getColumns ( ) [ i ] , from_element_types [ i ] , " " } } ;
2020-10-20 13:11:57 +00:00
converted_columns [ i ] = element_wrappers [ i ] ( element , to_element_types [ i ] , nullable_source , input_rows_count ) ;
2020-10-17 14:23:37 +00:00
}
2017-04-01 07:20:54 +00:00
2020-10-17 14:23:37 +00:00
return ColumnTuple : : create ( converted_columns ) ;
2017-04-01 07:20:54 +00:00
} ;
}
2020-12-15 22:22:21 +00:00
/// The case of: tuple([key1, key2, ..., key_n], [value1, value2, ..., value_n])
2020-12-14 22:16:04 +00:00
WrapperType createTupleToMapWrapper ( const DataTypes & from_kv_types , const DataTypes & to_kv_types ) const
{
return [ element_wrappers = getElementWrappers ( from_kv_types , to_kv_types ) , from_kv_types , to_kv_types ]
2021-02-23 10:42:33 +00:00
( ColumnsWithTypeAndName & arguments , const DataTypePtr & , const ColumnNullable * nullable_source , size_t /*input_rows_count*/ ) - > ColumnPtr
2020-12-14 22:16:04 +00:00
{
const auto * col = arguments . front ( ) . column . get ( ) ;
const auto & column_tuple = assert_cast < const ColumnTuple & > ( * col ) ;
2020-12-25 17:12:47 +00:00
Columns offsets ( 2 ) ;
2020-12-14 22:16:04 +00:00
Columns converted_columns ( 2 ) ;
for ( size_t i = 0 ; i < 2 ; + + i )
{
const auto & column_array = assert_cast < const ColumnArray & > ( column_tuple . getColumn ( i ) ) ;
ColumnsWithTypeAndName element = { { column_array . getDataPtr ( ) , from_kv_types [ i ] , " " } } ;
2021-02-23 10:42:33 +00:00
converted_columns [ i ] = element_wrappers [ i ] ( element , to_kv_types [ i ] , nullable_source , ( element [ 0 ] . column ) - > size ( ) ) ;
2020-12-25 17:12:47 +00:00
offsets [ i ] = column_array . getOffsetsPtr ( ) ;
2020-12-14 22:16:04 +00:00
}
2020-12-25 17:12:47 +00:00
const auto & keys_offsets = assert_cast < const ColumnArray : : ColumnOffsets & > ( * offsets [ 0 ] ) . getData ( ) ;
const auto & values_offsets = assert_cast < const ColumnArray : : ColumnOffsets & > ( * offsets [ 1 ] ) . getData ( ) ;
if ( keys_offsets ! = values_offsets )
throw Exception ( ErrorCodes : : TYPE_MISMATCH ,
" CAST AS Map can only be performed from tuple of arrays with equal sizes. " ) ;
return ColumnMap : : create ( converted_columns [ 0 ] , converted_columns [ 1 ] , offsets [ 0 ] ) ;
2020-12-14 22:16:04 +00:00
} ;
}
WrapperType createMapToMapWrrapper ( const DataTypes & from_kv_types , const DataTypes & to_kv_types ) const
{
return [ element_wrappers = getElementWrappers ( from_kv_types , to_kv_types ) , from_kv_types , to_kv_types ]
2021-04-09 17:50:10 +00:00
( ColumnsWithTypeAndName & arguments , const DataTypePtr & , const ColumnNullable * nullable_source , size_t /*input_rows_count*/ ) - > ColumnPtr
2020-12-14 22:16:04 +00:00
{
const auto * col = arguments . front ( ) . column . get ( ) ;
const auto & column_map = typeid_cast < const ColumnMap & > ( * col ) ;
const auto & nested_data = column_map . getNestedData ( ) ;
Columns converted_columns ( 2 ) ;
for ( size_t i = 0 ; i < 2 ; + + i )
{
ColumnsWithTypeAndName element = { { nested_data . getColumnPtr ( i ) , from_kv_types [ i ] , " " } } ;
2021-04-09 17:50:10 +00:00
converted_columns [ i ] = element_wrappers [ i ] ( element , to_kv_types [ i ] , nullable_source , ( element [ 0 ] . column ) - > size ( ) ) ;
2020-12-14 22:16:04 +00:00
}
return ColumnMap : : create ( converted_columns [ 0 ] , converted_columns [ 1 ] , column_map . getNestedColumn ( ) . getOffsetsPtr ( ) ) ;
} ;
}
2020-12-15 22:22:21 +00:00
/// The case of: [(key1, value1), (key2, value2), ...]
2020-12-14 22:16:04 +00:00
WrapperType createArrayToMapWrrapper ( const DataTypes & from_kv_types , const DataTypes & to_kv_types ) const
{
return [ element_wrappers = getElementWrappers ( from_kv_types , to_kv_types ) , from_kv_types , to_kv_types ]
2021-04-09 17:50:10 +00:00
( ColumnsWithTypeAndName & arguments , const DataTypePtr & , const ColumnNullable * nullable_source , size_t /*input_rows_count*/ ) - > ColumnPtr
2020-12-14 22:16:04 +00:00
{
const auto * col = arguments . front ( ) . column . get ( ) ;
const auto & column_array = typeid_cast < const ColumnArray & > ( * col ) ;
const auto & nested_data = typeid_cast < const ColumnTuple & > ( column_array . getData ( ) ) ;
Columns converted_columns ( 2 ) ;
for ( size_t i = 0 ; i < 2 ; + + i )
{
ColumnsWithTypeAndName element = { { nested_data . getColumnPtr ( i ) , from_kv_types [ i ] , " " } } ;
2021-04-09 17:50:10 +00:00
converted_columns [ i ] = element_wrappers [ i ] ( element , to_kv_types [ i ] , nullable_source , ( element [ 0 ] . column ) - > size ( ) ) ;
2020-12-14 22:16:04 +00:00
}
return ColumnMap : : create ( converted_columns [ 0 ] , converted_columns [ 1 ] , column_array . getOffsetsPtr ( ) ) ;
} ;
}
2020-10-10 06:49:03 +00:00
WrapperType createMapWrapper ( const DataTypePtr & from_type_untyped , const DataTypeMap * to_type ) const
{
2020-12-03 03:52:41 +00:00
if ( const auto * from_tuple = checkAndGetDataType < DataTypeTuple > ( from_type_untyped . get ( ) ) )
2020-10-26 18:20:45 +00:00
{
2020-12-03 03:52:41 +00:00
if ( from_tuple - > getElements ( ) . size ( ) ! = 2 )
throw Exception { " CAST AS Map from tuple requeires 2 elements. \n "
2020-10-26 18:20:45 +00:00
" Left type: " + from_tuple - > getName ( ) + " , right type: " + to_type - > getName ( ) , ErrorCodes : : TYPE_MISMATCH } ;
2020-12-03 03:52:41 +00:00
DataTypes from_kv_types ;
const auto & to_kv_types = to_type - > getKeyValueTypes ( ) ;
for ( const auto & elem : from_tuple - > getElements ( ) )
{
const auto * type_array = checkAndGetDataType < DataTypeArray > ( elem . get ( ) ) ;
if ( ! type_array )
throw Exception ( ErrorCodes : : TYPE_MISMATCH ,
" CAST AS Map can only be performed from tuples of array. Got: {} " , from_tuple - > getName ( ) ) ;
from_kv_types . push_back ( type_array - > getNestedType ( ) ) ;
}
2020-12-14 22:16:04 +00:00
return createTupleToMapWrapper ( from_kv_types , to_kv_types ) ;
}
else if ( const auto * from_array = typeid_cast < const DataTypeArray * > ( from_type_untyped . get ( ) ) )
{
const auto * nested_tuple = typeid_cast < const DataTypeTuple * > ( from_array - > getNestedType ( ) . get ( ) ) ;
if ( ! nested_tuple | | nested_tuple - > getElements ( ) . size ( ) ! = 2 )
throw Exception { " CAST AS Map from array requeires nested tuple of 2 elements. \n "
2020-12-15 14:34:37 +00:00
" Left type: " + from_array - > getName ( ) + " , right type: " + to_type - > getName ( ) , ErrorCodes : : TYPE_MISMATCH } ;
2020-12-03 03:52:41 +00:00
2020-12-14 22:16:04 +00:00
return createArrayToMapWrrapper ( nested_tuple - > getElements ( ) , to_type - > getKeyValueTypes ( ) ) ;
2020-10-26 18:20:45 +00:00
}
2020-12-03 03:52:41 +00:00
else if ( const auto * from_type = checkAndGetDataType < DataTypeMap > ( from_type_untyped . get ( ) ) )
2020-10-10 06:49:03 +00:00
{
2020-12-14 22:16:04 +00:00
return createMapToMapWrrapper ( from_type - > getKeyValueTypes ( ) , to_type - > getKeyValueTypes ( ) ) ;
2020-11-02 09:23:02 +00:00
}
else
{
2020-12-03 03:52:41 +00:00
throw Exception { " Unsupported types to CAST AS Map \n "
2020-11-02 09:23:02 +00:00
" Left type: " + from_type_untyped - > getName ( ) + " , right type: " + to_type - > getName ( ) , ErrorCodes : : TYPE_MISMATCH } ;
}
2020-10-10 06:49:03 +00:00
}
2017-04-01 07:20:54 +00:00
template < typename FieldType >
2020-10-20 13:11:57 +00:00
WrapperType createEnumWrapper ( const DataTypePtr & from_type , const DataTypeEnum < FieldType > * to_type ) const
2017-04-01 07:20:54 +00:00
{
using EnumType = DataTypeEnum < FieldType > ;
using Function = typename FunctionTo < EnumType > : : Type ;
2020-10-17 14:23:37 +00:00
if ( const auto * from_enum8 = checkAndGetDataType < DataTypeEnum8 > ( from_type . get ( ) ) )
2017-04-01 07:20:54 +00:00
checkEnumToEnumConversion ( from_enum8 , to_type ) ;
2020-10-17 14:23:37 +00:00
else if ( const auto * from_enum16 = checkAndGetDataType < DataTypeEnum16 > ( from_type . get ( ) ) )
2017-04-01 07:20:54 +00:00
checkEnumToEnumConversion ( from_enum16 , to_type ) ;
2017-07-21 06:35:58 +00:00
if ( checkAndGetDataType < DataTypeString > ( from_type . get ( ) ) )
2020-10-20 13:11:57 +00:00
return createStringToEnumWrapper < ColumnString , EnumType > ( ) ;
2017-07-21 06:35:58 +00:00
else if ( checkAndGetDataType < DataTypeFixedString > ( from_type . get ( ) ) )
2020-10-20 13:11:57 +00:00
return createStringToEnumWrapper < ColumnFixedString , EnumType > ( ) ;
2019-05-24 12:11:03 +00:00
else if ( isNativeNumber ( from_type ) | | isEnum ( from_type ) )
2017-04-01 07:20:54 +00:00
{
2019-12-25 19:17:41 +00:00
auto function = Function : : create ( ) ;
2020-11-05 19:09:17 +00:00
return createFunctionAdaptor ( function , from_type ) ;
2017-04-01 07:20:54 +00:00
}
else
2020-11-05 19:09:17 +00:00
{
2020-12-05 13:18:56 +00:00
if ( cast_type = = CastType : : accurateOrNull )
2020-11-05 19:09:17 +00:00
return createToNullableColumnWrapper ( ) ;
2020-11-12 15:56:17 +00:00
else
throw Exception { " Conversion from " + from_type - > getName ( ) + " to " + to_type - > getName ( ) + " is not supported " ,
ErrorCodes : : CANNOT_CONVERT_TYPE } ;
2020-11-05 19:09:17 +00:00
}
2017-04-01 07:20:54 +00:00
}
template < typename EnumTypeFrom , typename EnumTypeTo >
2018-02-02 08:33:36 +00:00
void checkEnumToEnumConversion ( const EnumTypeFrom * from_type , const EnumTypeTo * to_type ) const
2017-04-01 07:20:54 +00:00
{
const auto & from_values = from_type - > getValues ( ) ;
const auto & to_values = to_type - > getValues ( ) ;
using ValueType = std : : common_type_t < typename EnumTypeFrom : : FieldType , typename EnumTypeTo : : FieldType > ;
using NameValuePair = std : : pair < std : : string , ValueType > ;
using EnumValues = std : : vector < NameValuePair > ;
EnumValues name_intersection ;
std : : set_intersection ( std : : begin ( from_values ) , std : : end ( from_values ) ,
std : : begin ( to_values ) , std : : end ( to_values ) , std : : back_inserter ( name_intersection ) ,
[ ] ( auto & & from , auto & & to ) { return from . first < to . first ; } ) ;
for ( const auto & name_value : name_intersection )
{
const auto & old_value = name_value . second ;
const auto & new_value = to_type - > getValue ( name_value . first ) ;
if ( old_value ! = new_value )
2018-02-18 02:22:32 +00:00
throw Exception { " Enum conversion changes value for element ' " + name_value . first +
" ' from " + toString ( old_value ) + " to " + toString ( new_value ) , ErrorCodes : : CANNOT_CONVERT_TYPE } ;
2017-04-01 07:20:54 +00:00
}
2018-08-26 00:50:48 +00:00
}
2017-04-01 07:20:54 +00:00
template < typename ColumnStringType , typename EnumType >
2020-10-20 13:11:57 +00:00
WrapperType createStringToEnumWrapper ( ) const
2017-04-01 07:20:54 +00:00
{
2018-02-06 19:34:53 +00:00
const char * function_name = name ;
2020-10-20 13:11:57 +00:00
return [ function_name ] (
ColumnsWithTypeAndName & arguments , const DataTypePtr & res_type , const ColumnNullable * nullable_col , size_t /*input_rows_count*/ )
2017-04-01 07:20:54 +00:00
{
2020-10-17 14:23:37 +00:00
const auto & first_col = arguments . front ( ) . column . get ( ) ;
const auto & result_type = typeid_cast < const EnumType & > ( * res_type ) ;
2017-04-01 07:20:54 +00:00
2020-07-23 22:10:14 +00:00
const ColumnStringType * col = typeid_cast < const ColumnStringType * > ( first_col ) ;
2020-10-20 13:11:57 +00:00
if ( col & & nullable_col & & nullable_col - > size ( ) ! = col - > size ( ) )
throw Exception ( " ColumnNullable is not compatible with original " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-07-23 22:10:14 +00:00
if ( col )
2017-04-01 07:20:54 +00:00
{
const auto size = col - > size ( ) ;
auto res = result_type . createColumn ( ) ;
auto & out_data = static_cast < typename EnumType : : ColumnType & > ( * res ) . getData ( ) ;
out_data . resize ( size ) ;
2021-03-23 11:58:00 +00:00
auto default_enum_value = result_type . getValues ( ) . front ( ) . second ;
2020-07-23 22:10:14 +00:00
if ( nullable_col )
{
2021-06-15 19:55:21 +00:00
for ( size_t i = 0 ; i < size ; + + i )
2020-07-23 22:10:14 +00:00
{
if ( ! nullable_col - > isNullAt ( i ) )
out_data [ i ] = result_type . getValue ( col - > getDataAt ( i ) ) ;
2021-03-23 11:58:00 +00:00
else
out_data [ i ] = default_enum_value ;
2020-07-23 22:10:14 +00:00
}
}
else
{
2021-06-15 19:55:21 +00:00
for ( size_t i = 0 ; i < size ; + + i )
2020-07-23 22:10:14 +00:00
out_data [ i ] = result_type . getValue ( col - > getDataAt ( i ) ) ;
}
2017-04-01 07:20:54 +00:00
2020-10-17 14:23:37 +00:00
return res ;
2017-04-01 07:20:54 +00:00
}
else
2018-02-18 02:22:32 +00:00
throw Exception { " Unexpected column " + first_col - > getName ( ) + " as first argument of function " + function_name ,
2017-07-23 08:40:43 +00:00
ErrorCodes : : LOGICAL_ERROR } ;
2017-04-01 07:20:54 +00:00
} ;
}
2020-10-17 14:23:37 +00:00
static WrapperType createIdentityWrapper ( const DataTypePtr & )
2017-04-01 07:20:54 +00:00
{
2020-10-20 13:11:57 +00:00
return [ ] ( ColumnsWithTypeAndName & arguments , const DataTypePtr & , const ColumnNullable * , size_t /*input_rows_count*/ )
2017-04-01 07:20:54 +00:00
{
2020-10-17 14:23:37 +00:00
return arguments . front ( ) . column ;
2017-04-01 07:20:54 +00:00
} ;
}
2020-10-17 14:23:37 +00:00
static WrapperType createNothingWrapper ( const IDataType * to_type )
2017-12-09 20:56:53 +00:00
{
2017-12-18 04:07:26 +00:00
ColumnPtr res = to_type - > createColumnConstWithDefaultValue ( 1 ) ;
2020-10-20 13:11:57 +00:00
return [ res ] ( ColumnsWithTypeAndName & , const DataTypePtr & , const ColumnNullable * , size_t input_rows_count )
2017-12-09 20:56:53 +00:00
{
/// Column of Nothing type is trivially convertible to any other column
2020-10-17 14:23:37 +00:00
return res - > cloneResized ( input_rows_count ) - > convertToFullColumnIfConst ( ) ;
2017-12-09 20:56:53 +00:00
} ;
}
2018-06-07 10:37:06 +00:00
WrapperType prepareUnpackDictionaries ( const DataTypePtr & from_type , const DataTypePtr & to_type ) const
2017-04-01 07:20:54 +00:00
{
2018-09-27 15:55:22 +00:00
const auto * from_low_cardinality = typeid_cast < const DataTypeLowCardinality * > ( from_type . get ( ) ) ;
const auto * to_low_cardinality = typeid_cast < const DataTypeLowCardinality * > ( to_type . get ( ) ) ;
const auto & from_nested = from_low_cardinality ? from_low_cardinality - > getDictionaryType ( ) : from_type ;
const auto & to_nested = to_low_cardinality ? to_low_cardinality - > getDictionaryType ( ) : to_type ;
2018-09-19 11:11:52 +00:00
2017-12-09 06:32:22 +00:00
if ( from_type - > onlyNull ( ) )
2017-12-08 03:52:33 +00:00
{
2018-09-19 11:11:52 +00:00
if ( ! to_nested - > isNullable ( ) )
2020-11-05 19:09:17 +00:00
{
2020-12-05 13:18:56 +00:00
if ( cast_type = = CastType : : accurateOrNull )
2020-11-05 19:09:17 +00:00
{
return createToNullableColumnWrapper ( ) ;
}
else
{
throw Exception { " Cannot convert NULL to a non-nullable type " , ErrorCodes : : CANNOT_CONVERT_TYPE } ;
}
}
2018-02-22 21:20:02 +00:00
2020-10-20 13:11:57 +00:00
return [ ] ( ColumnsWithTypeAndName & , const DataTypePtr & result_type , const ColumnNullable * , size_t input_rows_count )
2017-12-08 03:52:33 +00:00
{
2020-10-17 14:23:37 +00:00
return result_type - > createColumnConstWithDefaultValue ( input_rows_count ) - > convertToFullColumnIfConst ( ) ;
2017-12-08 03:52:33 +00:00
} ;
}
2019-07-22 15:41:52 +00:00
bool skip_not_null_check = false ;
if ( from_low_cardinality & & from_nested - > isNullable ( ) & & ! to_nested - > isNullable ( ) )
/// Disable check for dictionary. Will check that column doesn't contain NULL in wrapper below.
skip_not_null_check = true ;
auto wrapper = prepareRemoveNullable ( from_nested , to_nested , skip_not_null_check ) ;
2018-09-27 15:55:22 +00:00
if ( ! from_low_cardinality & & ! to_low_cardinality )
2018-06-07 10:37:06 +00:00
return wrapper ;
2019-07-22 15:41:52 +00:00
return [ wrapper , from_low_cardinality , to_low_cardinality , skip_not_null_check ]
2020-10-20 13:11:57 +00:00
( ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , const ColumnNullable * nullable_source , size_t input_rows_count ) - > ColumnPtr
2018-06-07 10:37:06 +00:00
{
2020-10-17 14:23:37 +00:00
ColumnsWithTypeAndName args = { arguments [ 0 ] } ;
auto & arg = args . front ( ) ;
auto res_type = result_type ;
ColumnPtr converted_column ;
2018-06-07 10:37:06 +00:00
ColumnPtr res_indexes ;
2018-08-08 16:05:50 +00:00
/// For some types default can't be casted (for example, String to Int). In that case convert column to full.
bool src_converted_to_full_column = false ;
2018-06-07 10:37:06 +00:00
{
auto tmp_rows_count = input_rows_count ;
2018-09-27 15:55:22 +00:00
if ( to_low_cardinality )
2020-10-17 14:23:37 +00:00
res_type = to_low_cardinality - > getDictionaryType ( ) ;
2018-08-08 16:05:50 +00:00
2018-09-27 15:55:22 +00:00
if ( from_low_cardinality )
2018-06-07 10:37:06 +00:00
{
2020-10-17 14:23:37 +00:00
const auto * col_low_cardinality = typeid_cast < const ColumnLowCardinality * > ( arguments [ 0 ] . column . get ( ) ) ;
2019-07-22 15:41:52 +00:00
if ( skip_not_null_check & & col_low_cardinality - > containsNull ( ) )
throw Exception { " Cannot convert NULL value to non-Nullable type " ,
ErrorCodes : : CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN } ;
2018-09-27 15:55:22 +00:00
arg . column = col_low_cardinality - > getDictionary ( ) . getNestedColumn ( ) ;
arg . type = from_low_cardinality - > getDictionaryType ( ) ;
2018-06-07 10:37:06 +00:00
2018-08-08 16:05:50 +00:00
/// TODO: Make map with defaults conversion.
2020-10-17 14:23:37 +00:00
src_converted_to_full_column = ! removeNullable ( arg . type ) - > equals ( * removeNullable ( res_type ) ) ;
2018-08-08 16:05:50 +00:00
if ( src_converted_to_full_column )
2018-09-27 15:55:22 +00:00
arg . column = arg . column - > index ( col_low_cardinality - > getIndexes ( ) , 0 ) ;
2018-08-08 16:05:50 +00:00
else
2018-09-27 15:55:22 +00:00
res_indexes = col_low_cardinality - > getIndexesPtr ( ) ;
2018-08-08 16:05:50 +00:00
2018-06-07 10:37:06 +00:00
tmp_rows_count = arg . column - > size ( ) ;
}
/// Perform the requested conversion.
2020-10-20 13:11:57 +00:00
converted_column = wrapper ( args , res_type , nullable_source , tmp_rows_count ) ;
2018-06-07 10:37:06 +00:00
}
2018-09-27 15:55:22 +00:00
if ( to_low_cardinality )
2018-06-07 10:37:06 +00:00
{
2018-09-27 15:55:22 +00:00
auto res_column = to_low_cardinality - > createColumn ( ) ;
auto * col_low_cardinality = typeid_cast < ColumnLowCardinality * > ( res_column . get ( ) ) ;
2018-06-07 10:37:06 +00:00
2018-09-27 15:55:22 +00:00
if ( from_low_cardinality & & ! src_converted_to_full_column )
2018-06-07 10:37:06 +00:00
{
2020-10-17 14:23:37 +00:00
col_low_cardinality - > insertRangeFromDictionaryEncodedColumn ( * converted_column , * res_indexes ) ;
2018-06-07 10:37:06 +00:00
}
else
2020-10-17 14:23:37 +00:00
col_low_cardinality - > insertRangeFromFullColumn ( * converted_column , 0 , converted_column - > size ( ) ) ;
2018-06-07 10:37:06 +00:00
2020-10-17 14:23:37 +00:00
return res_column ;
2018-06-07 10:37:06 +00:00
}
2018-08-08 16:05:50 +00:00
else if ( ! src_converted_to_full_column )
2020-10-17 14:23:37 +00:00
return converted_column - > index ( * res_indexes , 0 ) ;
else
return converted_column ;
2018-06-07 10:37:06 +00:00
} ;
}
2019-07-22 15:41:52 +00:00
WrapperType prepareRemoveNullable ( const DataTypePtr & from_type , const DataTypePtr & to_type , bool skip_not_null_check ) const
2018-06-07 10:37:06 +00:00
{
/// Determine whether pre-processing and/or post-processing must take place during conversion.
2018-07-20 11:08:54 +00:00
bool source_is_nullable = from_type - > isNullable ( ) ;
bool result_is_nullable = to_type - > isNullable ( ) ;
2017-12-08 03:52:33 +00:00
2020-10-20 13:11:57 +00:00
auto wrapper = prepareImpl ( removeNullable ( from_type ) , removeNullable ( to_type ) , result_is_nullable ) ;
2017-04-01 07:20:54 +00:00
2018-07-11 19:51:18 +00:00
if ( result_is_nullable )
2017-04-01 07:20:54 +00:00
{
2018-07-11 19:51:18 +00:00
return [ wrapper , source_is_nullable ]
2020-10-20 13:11:57 +00:00
( ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , const ColumnNullable * , size_t input_rows_count ) - > ColumnPtr
2017-04-01 07:20:54 +00:00
{
2020-10-14 14:04:50 +00:00
/// Create a temporary columns on which to perform the operation.
2020-10-17 14:23:37 +00:00
const auto & nullable_type = static_cast < const DataTypeNullable & > ( * result_type ) ;
2017-04-01 07:20:54 +00:00
const auto & nested_type = nullable_type . getNestedType ( ) ;
2020-10-17 14:23:37 +00:00
ColumnsWithTypeAndName tmp_args ;
2018-07-11 19:51:18 +00:00
if ( source_is_nullable )
2020-10-17 14:23:37 +00:00
tmp_args = createBlockWithNestedColumns ( arguments ) ;
2017-04-01 07:20:54 +00:00
else
2020-10-17 14:23:37 +00:00
tmp_args = arguments ;
2017-04-01 07:20:54 +00:00
2020-10-20 13:11:57 +00:00
const ColumnNullable * nullable_source = nullptr ;
2020-07-23 22:10:14 +00:00
/// Add original ColumnNullable for createStringToEnumWrapper()
if ( source_is_nullable )
{
if ( arguments . size ( ) ! = 1 )
throw Exception ( " Invalid number of arguments " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-10-20 13:11:57 +00:00
nullable_source = typeid_cast < const ColumnNullable * > ( arguments . front ( ) . column . get ( ) ) ;
2020-07-23 22:10:14 +00:00
}
2017-04-01 07:20:54 +00:00
/// Perform the requested conversion.
2020-10-20 13:11:57 +00:00
auto tmp_res = wrapper ( tmp_args , nested_type , nullable_source , input_rows_count ) ;
2018-07-11 21:43:09 +00:00
2019-08-05 15:23:32 +00:00
/// May happen in fuzzy tests. For debug purpose.
2020-10-17 14:23:37 +00:00
if ( ! tmp_res )
throw Exception ( " Couldn't convert " + arguments [ 0 ] . type - > getName ( ) + " to "
2019-08-05 15:23:32 +00:00
+ nested_type - > getName ( ) + " in " + " prepareRemoveNullable wrapper. " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-10-17 14:23:37 +00:00
return wrapInNullable ( tmp_res , arguments , nested_type , input_rows_count ) ;
2017-04-01 07:20:54 +00:00
} ;
}
2018-07-11 19:51:18 +00:00
else if ( source_is_nullable )
2018-02-22 21:20:02 +00:00
{
/// Conversion from Nullable to non-Nullable.
2020-10-17 14:23:37 +00:00
return [ wrapper , skip_not_null_check ]
2020-10-20 13:11:57 +00:00
( ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , const ColumnNullable * , size_t input_rows_count ) - > ColumnPtr
2018-02-22 21:20:02 +00:00
{
2020-10-17 14:23:37 +00:00
auto tmp_args = createBlockWithNestedColumns ( arguments ) ;
2020-10-20 13:11:57 +00:00
auto nested_type = removeNullable ( result_type ) ;
2018-02-22 21:20:02 +00:00
/// Check that all values are not-NULL.
2019-07-22 15:41:52 +00:00
/// Check can be skipped in case if LowCardinality dictionary is transformed.
/// In that case, correctness will be checked beforehand.
if ( ! skip_not_null_check )
{
2020-10-17 14:23:37 +00:00
const auto & col = arguments [ 0 ] . column ;
2019-08-21 02:28:04 +00:00
const auto & nullable_col = assert_cast < const ColumnNullable & > ( * col ) ;
2019-07-22 15:41:52 +00:00
const auto & null_map = nullable_col . getNullMapData ( ) ;
2018-02-22 21:20:02 +00:00
2019-07-22 15:41:52 +00:00
if ( ! memoryIsZero ( null_map . data ( ) , null_map . size ( ) ) )
throw Exception { " Cannot convert NULL value to non-Nullable type " ,
ErrorCodes : : CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN } ;
}
2020-10-20 13:11:57 +00:00
const ColumnNullable * nullable_source = typeid_cast < const ColumnNullable * > ( arguments . front ( ) . column . get ( ) ) ;
return wrapper ( tmp_args , nested_type , nullable_source , input_rows_count ) ;
2018-02-22 21:20:02 +00:00
} ;
}
2017-04-01 07:20:54 +00:00
else
return wrapper ;
}
2018-08-31 08:59:21 +00:00
/// 'from_type' and 'to_type' are nested types in case of Nullable.
/// 'requested_result_is_nullable' is true if CAST to Nullable type is requested.
2020-10-20 13:11:57 +00:00
WrapperType prepareImpl ( const DataTypePtr & from_type , const DataTypePtr & to_type , bool requested_result_is_nullable ) const
2017-04-01 07:20:54 +00:00
{
if ( from_type - > equals ( * to_type ) )
return createIdentityWrapper ( from_type ) ;
2018-09-07 14:37:26 +00:00
else if ( WhichDataType ( from_type ) . isNothing ( ) )
2018-02-22 21:20:02 +00:00
return createNothingWrapper ( to_type . get ( ) ) ;
2018-08-31 08:59:21 +00:00
WrapperType ret ;
auto make_default_wrapper = [ & ] ( const auto & types ) - > bool
{
using Types = std : : decay_t < decltype ( types ) > ;
using ToDataType = typename Types : : LeftType ;
if constexpr (
std : : is_same_v < ToDataType , DataTypeUInt8 > | |
std : : is_same_v < ToDataType , DataTypeUInt16 > | |
std : : is_same_v < ToDataType , DataTypeUInt32 > | |
std : : is_same_v < ToDataType , DataTypeUInt64 > | |
2021-05-03 15:41:37 +00:00
std : : is_same_v < ToDataType , DataTypeUInt128 > | |
2020-08-19 11:52:17 +00:00
std : : is_same_v < ToDataType , DataTypeUInt256 > | |
2018-08-31 08:59:21 +00:00
std : : is_same_v < ToDataType , DataTypeInt8 > | |
std : : is_same_v < ToDataType , DataTypeInt16 > | |
std : : is_same_v < ToDataType , DataTypeInt32 > | |
std : : is_same_v < ToDataType , DataTypeInt64 > | |
2020-08-19 11:52:17 +00:00
std : : is_same_v < ToDataType , DataTypeInt128 > | |
std : : is_same_v < ToDataType , DataTypeInt256 > | |
2018-08-31 08:59:21 +00:00
std : : is_same_v < ToDataType , DataTypeFloat32 > | |
std : : is_same_v < ToDataType , DataTypeFloat64 > | |
std : : is_same_v < ToDataType , DataTypeDate > | |
2021-07-15 11:40:45 +00:00
std : : is_same_v < ToDataType , DataTypeDate32 > | |
2020-11-05 19:09:17 +00:00
std : : is_same_v < ToDataType , DataTypeDateTime > | |
std : : is_same_v < ToDataType , DataTypeUUID > )
2018-08-31 08:59:21 +00:00
{
ret = createWrapper ( from_type , checkAndGetDataType < ToDataType > ( to_type . get ( ) ) , requested_result_is_nullable ) ;
return true ;
}
if constexpr (
std : : is_same_v < ToDataType , DataTypeEnum8 > | |
std : : is_same_v < ToDataType , DataTypeEnum16 > )
{
2020-10-20 13:11:57 +00:00
ret = createEnumWrapper ( from_type , checkAndGetDataType < ToDataType > ( to_type . get ( ) ) ) ;
2018-08-31 08:59:21 +00:00
return true ;
}
if constexpr (
std : : is_same_v < ToDataType , DataTypeDecimal < Decimal32 > > | |
std : : is_same_v < ToDataType , DataTypeDecimal < Decimal64 > > | |
2019-10-30 13:59:38 +00:00
std : : is_same_v < ToDataType , DataTypeDecimal < Decimal128 > > | |
2020-08-19 11:52:17 +00:00
std : : is_same_v < ToDataType , DataTypeDecimal < Decimal256 > > | |
2019-10-24 10:44:38 +00:00
std : : is_same_v < ToDataType , DataTypeDateTime64 > )
2018-08-31 08:59:21 +00:00
{
2021-01-04 12:40:33 +00:00
ret = createDecimalWrapper ( from_type , checkAndGetDataType < ToDataType > ( to_type . get ( ) ) , requested_result_is_nullable ) ;
2018-08-31 08:59:21 +00:00
return true ;
}
return false ;
} ;
if ( callOnIndexAndDataType < void > ( to_type - > getTypeId ( ) , make_default_wrapper ) )
return ret ;
switch ( to_type - > getTypeId ( ) )
{
case TypeIndex : : String :
return createStringWrapper ( from_type ) ;
case TypeIndex : : FixedString :
return createFixedStringWrapper ( from_type , checkAndGetDataType < DataTypeFixedString > ( to_type . get ( ) ) - > getN ( ) ) ;
case TypeIndex : : Array :
2021-04-21 21:19:01 +00:00
return createArrayWrapper ( from_type , static_cast < const DataTypeArray & > ( * to_type ) ) ;
2018-08-31 08:59:21 +00:00
case TypeIndex : : Tuple :
return createTupleWrapper ( from_type , checkAndGetDataType < DataTypeTuple > ( to_type . get ( ) ) ) ;
2020-10-10 06:49:03 +00:00
case TypeIndex : : Map :
return createMapWrapper ( from_type , checkAndGetDataType < DataTypeMap > ( to_type . get ( ) ) ) ;
2019-09-02 09:16:49 +00:00
case TypeIndex : : AggregateFunction :
return createAggregateFunctionWrapper ( from_type , checkAndGetDataType < DataTypeAggregateFunction > ( to_type . get ( ) ) ) ;
2018-08-31 08:59:21 +00:00
default :
break ;
}
2017-04-01 07:20:54 +00:00
2020-12-05 13:18:56 +00:00
if ( cast_type = = CastType : : accurateOrNull )
2020-11-05 19:09:17 +00:00
return createToNullableColumnWrapper ( ) ;
else
throw Exception { " Conversion from " + from_type - > getName ( ) + " to " + to_type - > getName ( ) + " is not supported " ,
ErrorCodes : : CANNOT_CONVERT_TYPE } ;
2017-04-01 07:20:54 +00:00
}
2018-02-02 08:33:36 +00:00
} ;
2020-11-05 19:09:17 +00:00
class MonotonicityHelper
{
public :
using MonotonicityForRange = FunctionCast : : MonotonicityForRange ;
2020-12-17 18:32:25 +00:00
template < typename DataType >
2020-11-05 19:09:17 +00:00
static auto monotonicityForType ( const DataType * const )
{
return FunctionTo < DataType > : : Type : : Monotonic : : get ;
}
static MonotonicityForRange getMonotonicityInformation ( const DataTypePtr & from_type , const IDataType * to_type )
{
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeUInt8 > ( to_type ) )
return monotonicityForType ( type ) ;
if ( const auto * type = checkAndGetDataType < DataTypeUInt16 > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeUInt32 > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeUInt64 > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeUInt128 > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeUInt256 > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeInt8 > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeInt16 > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeInt32 > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeInt64 > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeInt128 > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeInt256 > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeFloat32 > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeFloat64 > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeDate > ( to_type ) )
2021-07-15 11:40:45 +00:00
return monotonicityForType ( type ) ;
if ( const auto * type = checkAndGetDataType < DataTypeDate32 > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeDateTime > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeString > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
if ( isEnum ( from_type ) )
{
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeEnum8 > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
2021-05-03 15:41:37 +00:00
if ( const auto * type = checkAndGetDataType < DataTypeEnum16 > ( to_type ) )
2020-11-05 19:09:17 +00:00
return monotonicityForType ( type ) ;
}
/// other types like Null, FixedString, Array and Tuple have no monotonicity defined
return { } ;
2017-04-01 07:20:54 +00:00
}
2018-02-02 08:33:36 +00:00
} ;
2020-12-05 13:18:56 +00:00
template < CastType cast_type >
2021-05-15 17:33:15 +00:00
class CastOverloadResolver : public IFunctionOverloadResolver
2018-02-02 08:33:36 +00:00
{
public :
using MonotonicityForRange = FunctionCast : : MonotonicityForRange ;
2020-11-18 09:35:32 +00:00
using Diagnostic = FunctionCast : : Diagnostic ;
2018-02-02 08:33:36 +00:00
2020-12-05 13:18:56 +00:00
static constexpr auto accurate_cast_name = " accurateCast " ;
static constexpr auto accurate_cast_or_null_name = " accurateCastOrNull " ;
static constexpr auto cast_name = " CAST " ;
2018-02-02 08:33:36 +00:00
2020-12-05 14:55:37 +00:00
static constexpr auto name = cast_type = = CastType : : accurate
? accurate_cast_name
: ( cast_type = = CastType : : accurateOrNull ? accurate_cast_or_null_name : cast_name ) ;
2020-12-05 13:18:56 +00:00
2021-05-15 17:33:15 +00:00
static FunctionOverloadResolverPtr create ( ContextPtr context )
2020-12-05 13:18:56 +00:00
{
2021-04-10 23:33:54 +00:00
return createImpl ( context - > getSettingsRef ( ) . cast_keep_nullable ) ;
2020-12-05 13:18:56 +00:00
}
2018-02-02 08:33:36 +00:00
2021-05-15 17:33:15 +00:00
static FunctionOverloadResolverPtr createImpl ( bool keep_nullable , std : : optional < Diagnostic > diagnostic = { } )
2020-11-18 09:35:32 +00:00
{
return std : : make_unique < CastOverloadResolver > ( keep_nullable , std : : move ( diagnostic ) ) ;
}
2020-06-18 10:18:28 +00:00
2020-11-18 09:35:32 +00:00
explicit CastOverloadResolver ( bool keep_nullable_ , std : : optional < Diagnostic > diagnostic_ = { } )
: keep_nullable ( keep_nullable_ ) , diagnostic ( std : : move ( diagnostic_ ) )
2020-06-18 10:18:28 +00:00
{ }
2018-02-02 08:33:36 +00:00
2018-02-09 19:32:12 +00:00
String getName ( ) const override { return name ; }
2018-02-02 08:33:36 +00:00
size_t getNumberOfArguments ( ) const override { return 2 ; }
2019-10-02 17:51:00 +00:00
ColumnNumbers getArgumentsThatAreAlwaysConstant ( ) const override { return { 1 } ; }
2017-04-01 07:20:54 +00:00
2021-04-29 14:48:26 +00:00
bool isSuitableForShortCircuitArgumentsExecution ( ) const override { return false ; }
2018-02-02 08:33:36 +00:00
protected :
2021-05-15 17:33:15 +00:00
FunctionBasePtr buildImpl ( const ColumnsWithTypeAndName & arguments , const DataTypePtr & return_type ) const override
2018-02-02 08:33:36 +00:00
{
DataTypes data_types ( arguments . size ( ) ) ;
for ( size_t i = 0 ; i < arguments . size ( ) ; + + i )
data_types [ i ] = arguments [ i ] . type ;
2020-11-05 19:09:17 +00:00
auto monotonicity = MonotonicityHelper : : getMonotonicityInformation ( arguments . front ( ) . type , return_type . get ( ) ) ;
2020-12-05 13:18:56 +00:00
return std : : make_unique < FunctionCast > ( name , std : : move ( monotonicity ) , data_types , return_type , diagnostic , cast_type ) ;
2018-02-02 08:33:36 +00:00
}
2021-05-15 17:33:15 +00:00
DataTypePtr getReturnTypeImpl ( const ColumnsWithTypeAndName & arguments ) const override
2018-02-02 08:33:36 +00:00
{
2020-04-12 02:51:56 +00:00
const auto & column = arguments . back ( ) . column ;
if ( ! column )
throw Exception ( " Second argument to " + getName ( ) + " must be a constant string describing type. "
" Instead there is non-constant column of type " + arguments . back ( ) . type - > getName ( ) ,
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
2020-10-17 14:23:37 +00:00
const auto * type_col = checkAndGetColumnConst < ColumnString > ( column . get ( ) ) ;
2018-02-02 08:33:36 +00:00
if ( ! type_col )
2020-04-12 02:51:56 +00:00
throw Exception ( " Second argument to " + getName ( ) + " must be a constant string describing type. "
" Instead there is a column with the following structure: " + column - > dumpStructure ( ) ,
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
2018-02-02 08:33:36 +00:00
2020-06-18 10:18:28 +00:00
DataTypePtr type = DataTypeFactory : : instance ( ) . get ( type_col - > getValue < String > ( ) ) ;
2020-12-05 13:18:56 +00:00
if constexpr ( cast_type = = CastType : : accurateOrNull )
{
2020-06-18 10:18:28 +00:00
return makeNullable ( type ) ;
2020-12-05 13:18:56 +00:00
}
else
{
if ( keep_nullable & & arguments . front ( ) . type - > isNullable ( ) )
return makeNullable ( type ) ;
return type ;
}
2018-02-02 08:33:36 +00:00
}
bool useDefaultImplementationForNulls ( ) const override { return false ; }
2018-09-27 15:55:22 +00:00
bool useDefaultImplementationForLowCardinalityColumns ( ) const override { return false ; }
2018-02-02 08:33:36 +00:00
private :
2020-06-18 10:18:28 +00:00
bool keep_nullable ;
2020-11-18 09:35:32 +00:00
std : : optional < Diagnostic > diagnostic ;
2015-12-23 08:11:11 +00:00
} ;
2011-10-15 23:40:56 +00:00
}