2011-10-15 23:40:56 +00:00
# pragma once
2021-09-29 16:42:41 +00:00
# include <cstddef>
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>
2021-08-18 20:34:07 +00:00
# include <Columns/ColumnStringHelpers.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-12-23 13:46:16 +00:00
bool result_is_bool = isBool ( result_type ) ;
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
{
2022-01-18 11:52:27 +00:00
if constexpr ( std : : is_same_v < ToDataType , DataTypeUInt8 > )
{
if ( result_is_bool )
{
vec_to [ i ] = vec_from [ i ] ! = FromFieldType ( 0 ) ;
continue ;
}
}
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-10-20 03:36:07 +00:00
static inline Int64 execute ( Int32 d , const DateLUTImpl & time_zone )
2021-07-15 11:40:45 +00:00
{
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-09-30 07:06:08 +00:00
template < typename Name > struct ConvertImpl < DataTypeDateTime64 , DataTypeDate32 , Name , ConvertDefaultBehaviorTag >
: DateTimeTransformImpl < DataTypeDateTime64 , DataTypeDate32 , TransformDateTime64 < ToDate32Impl > > { } ;
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 ) ;
}
2021-09-30 07:06:08 +00:00
inline DateTime64 : : NativeType execute ( Int32 d , const DateLUTImpl & time_zone ) const
{
const auto dt = ToDateTimeImpl : : execute ( d , time_zone ) ;
2021-10-20 03:36:07 +00:00
return DecimalUtils : : decimalFromComponentsWithMultiplier < DateTime64 > ( dt , 0 , scale_multiplier ) ;
2021-09-30 07:06:08 +00:00
}
2020-04-17 13:26:44 +00:00
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 > { } ;
2021-09-30 07:06:08 +00:00
template < typename Name > struct ConvertImpl < DataTypeDate32 , DataTypeDateTime64 , Name , ConvertDefaultBehaviorTag >
: DateTimeTransformImpl < DataTypeDate32 , DataTypeDateTime64 , ToDateTime64Transform > { } ;
2020-04-17 13:26:44 +00:00
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
{
2021-08-16 08:03:23 +00:00
writeText ( x , type - > getScale ( ) , wb , false ) ;
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 ;
2021-09-10 11:49:22 +00:00
using ColVecType = ColumnVectorOrDecimal < 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
2021-08-18 20:34:07 +00:00
/// Generic conversion of any type to String or FixedString via serialization to text.
template < typename StringColumnType >
2016-07-30 04:39:51 +00:00
struct ConvertImplGenericToString
{
2021-09-29 16:42:41 +00:00
static ColumnPtr execute ( const ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , size_t /*input_rows_count*/ )
2017-04-01 07:20:54 +00:00
{
2021-08-18 20:34:07 +00:00
static_assert ( std : : is_same_v < StringColumnType , ColumnString > | | std : : is_same_v < StringColumnType , ColumnFixedString > ,
" Can be used only to serialize to ColumnString or ColumnFixedString " ) ;
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 ;
2021-09-29 16:42:41 +00:00
size_t size = col_from . size ( ) ;
2021-08-18 20:34:07 +00:00
auto col_to = result_type - > createColumn ( ) ;
2017-04-01 07:20:54 +00:00
2021-08-18 20:34:07 +00:00
{
ColumnStringHelpers : : WriteHelper write_helper (
assert_cast < StringColumnType & > ( * col_to ) ,
2021-09-29 16:42:41 +00:00
size ) ;
2017-04-01 07:20:54 +00:00
2021-08-18 20:34:07 +00:00
auto & write_buffer = write_helper . getWriteBuffer ( ) ;
2017-04-01 07:20:54 +00:00
2021-08-18 20:34:07 +00:00
FormatSettings format_settings ;
auto serialization = type . getDefaultSerialization ( ) ;
2021-09-29 16:42:41 +00:00
for ( size_t i = 0 ; i < size ; + + i )
2021-08-18 20:34:07 +00:00
{
serialization - > serializeText ( col_from , i , write_buffer , format_settings ) ;
write_helper . rowWritten ( ) ;
}
2021-09-29 16:42:41 +00:00
write_helper . finalize ( ) ;
2017-04-01 07:20:54 +00:00
}
2017-12-09 10:14:45 +00:00
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 " ;
2021-08-18 20:34:07 +00:00
// Currently there are no functions toIPv{4,6}Or{Null,Zero}
if ( isNativeNumber ( to_type ) & & ! ( to_type . getName ( ) = = " IPv4 " | | to_type . getName ( ) = = " IPv6 " ) )
2019-12-25 19:17:41 +00:00
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 > { } ;
2021-08-18 20:34:07 +00:00
/// Generic conversion of any type from String. Used for complex types: Array and Tuple or types with custom serialization.
template < typename StringColumnType >
2016-08-02 03:30:34 +00:00
struct ConvertImplGenericFromString
{
2021-08-18 20:34:07 +00:00
static ColumnPtr execute ( ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , const ColumnNullable * , size_t input_rows_count )
2017-04-01 07:20:54 +00:00
{
2021-08-18 20:34:07 +00:00
static_assert ( std : : is_same_v < StringColumnType , ColumnString > | | std : : is_same_v < StringColumnType , ColumnFixedString > ,
" Can be used only to parse from ColumnString or ColumnFixedString " ) ;
2016-08-02 03:30:34 +00:00
2021-08-18 20:34:07 +00:00
const IColumn & col_from = * arguments [ 0 ] . column ;
2020-10-17 14:23:37 +00:00
const IDataType & data_type_to = * result_type ;
2021-08-18 20:34:07 +00:00
if ( const StringColumnType * col_from_string = checkAndGetColumn < StringColumnType > ( & 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 ;
2021-08-18 20:34:07 +00:00
column_to . reserve ( input_rows_count ) ;
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 ( ) ;
2021-08-18 20:34:07 +00:00
for ( size_t i = 0 ; i < input_rows_count ; + + i )
2017-04-01 07:20:54 +00:00
{
2021-08-18 20:34:07 +00:00
const auto & val = col_from_string - > getDataAt ( i ) ;
ReadBufferFromMemory read_buffer ( val . data , val . size ) ;
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 ) ;
2017-04-01 07:20:54 +00:00
}
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
} ;
2021-12-20 23:59:08 +00:00
template < typename Name >
struct ConvertImpl < DataTypeUInt8 , DataTypeUInt8 , Name , ConvertDefaultBehaviorTag >
{
template < typename Additions = void * >
static ColumnPtr execute ( const ColumnsWithTypeAndName & arguments , const DataTypePtr & , size_t /*input_rows_count*/ ,
Additions additions [[maybe_unused]] = Additions ( ) )
{
return arguments [ 0 ] . column ;
}
} ;
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-08-16 11:30:56 +00:00
static constexpr bool to_date_or_datetime = std : : is_same_v < ToDataType , DataTypeDate > | |
std : : is_same_v < ToDataType , DataTypeDate32 > | |
std : : is_same_v < ToDataType , DataTypeDateTime > ;
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-08-16 11:30:56 +00:00
bool isSuitableForShortCircuitArgumentsExecution ( const DataTypesWithConstInfo & arguments ) const override
{
/// TODO: We can make more optimizations here.
return ! ( to_date_or_datetime & & isNumber ( * arguments [ 0 ] . type ) ) ;
}
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
{
2021-09-30 11:35:24 +00:00
mandatory_args . push_back ( { " scale " , & isNativeInteger < IDataType > , & 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
{
2021-09-30 11:35:24 +00:00
mandatory_args . push_back ( { " scale " , & isNativeInteger < IDataType > , & 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
{
2021-09-30 11:35:24 +00:00
optional_args . push_back ( { " timezone " , & isString < IDataType > , & 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 =
2021-09-10 11:49:22 +00:00
is_decimal < LeftT > | | std : : is_floating_point_v < LeftT > | | is_big_int_v < LeftT > | | is_signed_v < LeftT > ;
2020-08-19 11:52:17 +00:00
static constexpr bool bad_right =
2021-09-10 11:49:22 +00:00
is_decimal < RightT > | | std : : is_floating_point_v < RightT > | | is_big_int_v < RightT > | | is_signed_v < RightT > ;
2020-08-19 11:52:17 +00:00
/// 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
2021-12-20 23:59:08 +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-08-18 20:34:07 +00:00
return ConvertImplGenericToString < ColumnString > : : execute ( arguments , result_type , input_rows_count ) ;
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-06-28 10:23:07 +00:00
bool isSuitableForShortCircuitArgumentsExecution ( const DataTypesWithConstInfo & /*arguments*/ ) 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 ; }
2021-12-29 10:34:54 +00:00
bool canBeExecutedOnDefaultArguments ( ) const override { return false ; }
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 ,
2021-09-30 11:35:24 +00:00
FunctionArgumentDescriptors { { " string " , & isStringOrFixedString < IDataType > , nullptr , " String or FixedString " } } ,
2020-05-16 12:11:17 +00:00
// optional
FunctionArgumentDescriptors {
2021-09-30 11:35:24 +00:00
{ " precision " , & isUInt8 < IDataType > , isColumnConst , " const UInt8 " } ,
{ " timezone " , & isStringOrFixedString < IDataType > , isColumnConst , " const String or FixedString " } ,
2020-05-16 12:11:17 +00:00
} ) ;
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
{
2021-09-29 16:01:26 +00:00
return { . is_monotonic = true } ;
2017-04-01 07:20:54 +00:00
}
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 & )
{
2021-09-29 16:01:26 +00:00
return { } ;
2018-08-22 17:26:11 +00:00
}
} ;
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 ) )
2021-09-29 16:01:26 +00:00
return { . is_monotonic = true , . is_always_monotonic = 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.
2021-09-29 16:01:26 +00:00
if constexpr ( std : : is_floating_point_v < T > )
return { . is_monotonic = true , . is_always_monotonic = true } ;
2018-12-24 15:46:55 +00:00
/// 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 ( ) ) )
2021-09-29 16:01:26 +00:00
return { . is_monotonic = true } ;
2017-04-01 07:20:54 +00:00
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 )
2021-09-29 16:01:26 +00:00
return { . is_monotonic = true , . is_always_monotonic = true } ;
2018-12-24 15:46:55 +00:00
if ( left_in_first_half = = right_in_first_half )
2021-09-29 16:01:26 +00:00
return { . is_monotonic = 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 )
2021-09-29 16:01:26 +00:00
return { . is_monotonic = true , . is_always_monotonic = true } ;
2018-12-24 15:46:55 +00:00
if ( ! to_is_unsigned )
2021-09-29 16:01:26 +00:00
return { . is_monotonic = true , . is_always_monotonic = true } ;
2018-12-24 15:46:55 +00:00
/// signed -> unsigned. If arguments from the same half, then function is monotonic.
if ( left_in_first_half = = right_in_first_half )
2021-09-29 16:01:26 +00:00
return { . is_monotonic = 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 )
2021-09-29 16:01:26 +00:00
return { . is_monotonic = true } ;
2020-09-14 03:34:14 +00:00
else
2021-09-29 16:01:26 +00:00
{
2020-09-14 03:34:14 +00:00
// If To is signed, it's possible that the signedness is different after conversion. So we check it explicitly.
2021-09-29 16:01:26 +00:00
const bool is_monotonic = ( T ( left . get < UInt64 > ( ) ) > = 0 ) = = ( T ( right . get < UInt64 > ( ) ) > = 0 ) ;
return { . is_monotonic = is_monotonic } ;
}
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 ( ) )
2021-09-29 16:01:26 +00:00
return { . is_monotonic = true , . is_always_monotonic = true } ;
2020-08-07 17:38:42 +00:00
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
2021-09-29 16:01:26 +00:00
return { . is_monotonic = true , . is_always_monotonic = true } ;
2020-08-07 17:38:42 +00:00
}
} ;
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 ( ) )
2021-09-29 16:01:26 +00:00
return { . is_monotonic = true , . is_always_monotonic = true } ;
2020-08-08 06:30:50 +00:00
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 )
{
2021-09-29 16:01:26 +00:00
IFunction : : Monotonicity positive { . is_monotonic = true } ;
2017-04-01 07:20:54 +00:00
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
} ;
2021-08-07 08:11:40 +00:00
struct CastName { static constexpr auto name = " CAST " ; } ;
struct CastInternalName { static constexpr auto name = " _CAST " ; } ;
2018-07-11 19:51:18 +00:00
2020-12-05 14:55:37 +00:00
enum class CastType
{
2020-12-05 13:18:56 +00:00
nonAccurate ,
accurate ,
accurateOrNull
} ;
2021-08-07 08:11:40 +00:00
class FunctionCastBase : public IFunctionBase
2018-02-02 08:33:36 +00:00
{
public :
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 ;
2021-08-07 08:11:40 +00:00
} ;
2018-02-02 08:33:36 +00:00
2021-08-07 08:11:40 +00:00
template < typename FunctionName >
class FunctionCast final : public FunctionCastBase
{
public :
using WrapperType = std : : function < ColumnPtr ( ColumnsWithTypeAndName & , const DataTypePtr & , const ColumnNullable * , size_t ) > ;
FunctionCast ( const char * cast_name_
, MonotonicityForRange & & monotonicity_for_range_
, const DataTypes & argument_types_
, const DataTypePtr & return_type_
, std : : optional < Diagnostic > diagnostic_
, CastType cast_type_ )
: cast_name ( cast_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 > (
2021-08-07 08:11:40 +00:00
prepareUnpackDictionaries ( getArgumentTypes ( ) [ 0 ] , getResultType ( ) ) , cast_name , diagnostic ) ;
2020-11-18 09:45:46 +00:00
}
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
}
2021-08-07 08:11:40 +00:00
String getName ( ) const override { return cast_name ; }
2018-02-02 08:33:36 +00:00
2019-10-10 14:38:08 +00:00
bool isDeterministic ( ) const override { return true ; }
bool isDeterministicInScopeOfQuery ( ) const override { return true ; }
2021-06-28 10:23:07 +00:00
bool isSuitableForShortCircuitArgumentsExecution ( const DataTypesWithConstInfo & /*arguments*/ ) const override { return true ; }
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 :
2021-08-07 08:11:40 +00:00
const char * cast_name ;
2018-02-02 08:33:36 +00:00
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.
2021-12-20 23:59:08 +00:00
2021-08-07 08:11:40 +00:00
FunctionPtr function = FunctionConvertFromString < ToDataType , FunctionName , ConvertFromStringExceptionMode : : Null > : : create ( ) ;
2020-11-05 19:09:17 +00:00
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 )
{
2021-08-07 08:11:40 +00:00
result_column = ConvertImpl < LeftDataType , RightDataType , FunctionName > : : execute (
2020-12-05 13:18:56 +00:00
arguments , result_type , input_rows_count , AccurateConvertStrategyAdditions ( ) ) ;
}
else
{
2021-08-07 08:11:40 +00:00
result_column = ConvertImpl < LeftDataType , RightDataType , FunctionName > : : execute (
2020-12-05 13:18:56 +00:00
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 )
{
2021-08-07 08:11:40 +00:00
auto nullable_column_wrapper = FunctionCast < FunctionName > : : 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
{
2021-09-06 15:59:46 +00:00
throw Exception ( ErrorCodes : : CANNOT_CONVERT_TYPE ,
" Conversion from {} to {} is not supported " ,
from_type_index , to_type - > getName ( ) ) ;
2020-11-12 11:27:02 +00:00
}
2020-11-05 19:09:17 +00:00
}
return result_column ;
2017-04-01 07:20:54 +00:00
} ;
}
2021-12-20 20:24:16 +00:00
template < typename ToDataType >
WrapperType createBoolWrapper ( const DataTypePtr & from_type , const ToDataType * const to_type , bool requested_result_is_nullable ) const
{
if ( checkAndGetDataType < DataTypeString > ( from_type . get ( ) ) )
{
2021-12-23 17:14:54 +00:00
return & ConvertImplGenericFromString < ColumnString > : : execute ;
2021-12-20 20:24:16 +00:00
}
return createWrapper < ToDataType > ( from_type , to_type , requested_result_is_nullable ) ;
}
2021-12-20 23:59:08 +00:00
WrapperType createUInt8ToUInt8Wrapper ( const DataTypePtr from_type , const DataTypePtr to_type ) const
{
return [ from_type , to_type ] ( ColumnsWithTypeAndName & arguments , const DataTypePtr & , const ColumnNullable * , size_t /*input_rows_count*/ ) - > ColumnPtr
{
if ( isBool ( from_type ) | | ! isBool ( to_type ) )
return arguments . front ( ) . column ;
/// Special case when we convert UInt8 column to Bool column.
/// both columns have type UInt8, but we shouldn't use identity wrapper,
/// because Bool column can contain only 0 and 1.
auto res_column = to_type - > createColumn ( ) ;
const auto & data_from = checkAndGetColumn < ColumnUInt8 > ( arguments [ 0 ] . column . get ( ) ) - > getData ( ) ;
auto & data_to = assert_cast < ColumnUInt8 * > ( res_column . get ( ) ) - > getData ( ) ;
data_to . resize ( data_from . size ( ) ) ;
for ( size_t i = 0 ; i ! = data_from . size ( ) ; + + i )
data_to [ i ] = static_cast < bool > ( data_from [ i ] ) ;
return res_column ;
} ;
}
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 ;
2021-08-07 08:11:40 +00:00
result_column = ConvertImpl < LeftDataType , RightDataType , FunctionName > : : execute (
2020-12-05 13:18:56 +00:00
arguments , result_type , input_rows_count , additions ) ;
return true ;
}
else if ( wrapper_cast_type = = CastType : : accurateOrNull )
{
AccurateOrNullConvertStrategyAdditions additions ;
additions . scale = scale ;
2021-08-07 08:11:40 +00:00
result_column = ConvertImpl < LeftDataType , RightDataType , FunctionName > : : execute (
2020-12-05 13:18:56 +00:00
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.
2021-08-07 08:11:40 +00:00
result_column = ConvertImpl < LeftDataType , RightDataType , FunctionName , ConvertReturnNullOnErrorTag > : : execute (
2021-01-04 14:52:07 +00:00
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
2021-08-07 08:11:40 +00:00
result_column = ConvertImpl < LeftDataType , RightDataType , FunctionName > : : 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 )
{
2021-08-07 08:11:40 +00:00
auto nullable_column_wrapper = FunctionCast < FunctionName > : : 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
2021-09-06 15:59:46 +00:00
throw Exception ( ErrorCodes : : CANNOT_CONVERT_TYPE ,
" Conversion from {} to {} is not supported " ,
type_index , to_type - > getName ( ) ) ;
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 ( ) ) )
{
2021-08-18 20:34:07 +00:00
return & ConvertImplGenericFromString < ColumnString > : : execute ;
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
{
2021-08-18 20:34:07 +00:00
return & ConvertImplGenericFromString < ColumnString > : : execute ;
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 ( ) ) ;
2022-01-23 00:09:29 +00:00
const auto * from_type_map = checkAndGetDataType < DataTypeMap > ( from_type_untyped . get ( ) ) ;
2022-01-21 15:14:31 +00:00
2022-01-23 00:09:29 +00:00
/// Convert from Map
if ( from_type_map )
2022-01-21 15:14:31 +00:00
from_type = checkAndGetDataType < DataTypeArray > ( from_type_map - > getNestedType ( ) . 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 ,
2022-01-21 15:14:31 +00:00
" CAST AS Array can only be performed between same-dimensional Array, Map 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-06-07 11:52:54 +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
{
2022-01-21 15:14:31 +00:00
const auto & argument_column = arguments . front ( ) ;
2017-04-01 07:20:54 +00:00
2022-01-21 15:14:31 +00:00
const ColumnArray * col_array = nullptr ;
if ( const ColumnMap * col_map = checkAndGetColumn < ColumnMap > ( argument_column . column . get ( ) ) )
col_array = & col_map - > getNestedColumn ( ) ;
else
col_array = checkAndGetColumn < ColumnArray > ( argument_column . column . get ( ) ) ;
if ( col_array )
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
2022-01-21 15:14:31 +00:00
{
throw Exception ( ErrorCodes : : LOGICAL_ERROR ,
" Illegal column {} for function CAST AS Array " ,
argument_column . column - > getName ( ) ) ;
}
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
{
2021-08-18 20:34:07 +00:00
return & ConvertImplGenericFromString < ColumnString > : : execute ;
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
{
2021-08-07 08:11:40 +00:00
const char * function_name = cast_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 ) )
2021-12-20 23:59:08 +00:00
{
if ( isUInt8 ( from_type ) )
return createUInt8ToUInt8Wrapper ( from_type , to_type ) ;
2017-04-01 07:20:54 +00:00
return createIdentityWrapper ( from_type ) ;
2021-12-20 23:59:08 +00:00
}
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 , 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 ;
}
2021-12-20 20:24:16 +00:00
if constexpr ( std : : is_same_v < ToDataType , DataTypeUInt8 > )
{
2021-12-20 23:59:08 +00:00
if ( isBool ( to_type ) )
2021-12-20 20:24:16 +00:00
ret = createBoolWrapper < ToDataType > ( from_type , checkAndGetDataType < ToDataType > ( to_type . get ( ) ) , requested_result_is_nullable ) ;
else
ret = createWrapper ( from_type , checkAndGetDataType < ToDataType > ( to_type . get ( ) ) , requested_result_is_nullable ) ;
return true ;
}
2018-08-31 08:59:21 +00:00
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 ;
} ;
2021-08-18 20:34:07 +00:00
auto make_custom_serialization_wrapper = [ & ] ( const auto & types ) - > bool
{
using Types = std : : decay_t < decltype ( types ) > ;
using ToDataType = typename Types : : RightType ;
using FromDataType = typename Types : : LeftType ;
if constexpr ( WhichDataType ( FromDataType : : type_id ) . isStringOrFixedString ( ) )
{
if ( to_type - > getCustomSerialization ( ) )
{
ret = & ConvertImplGenericFromString < typename FromDataType : : ColumnType > : : execute ;
return true ;
}
}
if constexpr ( WhichDataType ( ToDataType : : type_id ) . isStringOrFixedString ( ) )
{
if ( from_type - > getCustomSerialization ( ) )
{
ret = [ ] ( ColumnsWithTypeAndName & arguments , const DataTypePtr & result_type , const ColumnNullable * , size_t input_rows_count ) - > ColumnPtr
{
return ConvertImplGenericToString < typename ToDataType : : ColumnType > : : execute ( arguments , result_type , input_rows_count ) ;
} ;
return true ;
}
}
return false ;
} ;
if ( callOnTwoTypeIndexes ( from_type - > getTypeId ( ) , to_type - > getTypeId ( ) , make_custom_serialization_wrapper ) )
return ret ;
2018-08-31 08:59:21 +00:00
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 :
2021-08-07 08:11:40 +00:00
using MonotonicityForRange = FunctionCastBase : : MonotonicityForRange ;
2020-11-05 19:09:17 +00:00
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
} ;
2011-10-15 23:40:56 +00:00
}