2012-12-26 14:31:03 +00:00
# pragma once
2017-04-01 09:19:00 +00:00
# include <DataTypes/DataTypesNumber.h>
# include <DataTypes/DataTypeString.h>
# include <DataTypes/DataTypeFixedString.h>
# include <DataTypes/DataTypeDate.h>
# include <DataTypes/DataTypeDateTime.h>
# include <Columns/ColumnString.h>
# include <Columns/ColumnFixedString.h>
# include <Columns/ColumnConst.h>
2017-07-21 06:35:58 +00:00
# include <Columns/ColumnVector.h>
2017-07-13 20:58:19 +00:00
# include <Common/typeid_cast.h>
2019-02-10 17:40:52 +00:00
# include <Common/memcpySmall.h>
2017-04-01 09:19:00 +00:00
# include <Functions/IFunction.h>
2017-07-21 06:35:58 +00:00
# include <Functions/FunctionHelpers.h>
2012-12-26 14:31:03 +00:00
namespace DB
{
2014-06-26 00:58:14 +00:00
2017-06-13 02:06:53 +00:00
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN ;
}
2017-05-13 22:19:04 +00:00
/** Functions for transforming numbers and dates to strings that contain the same set of bytes in the machine representation, and vice versa.
2017-06-13 02:06:53 +00:00
*/
2012-12-26 14:31:03 +00:00
2017-09-15 12:16:12 +00:00
template < typename Name >
2012-12-26 14:31:03 +00:00
class FunctionReinterpretAsStringImpl : public IFunction
{
public :
2017-04-01 07:20:54 +00:00
static constexpr auto name = Name : : name ;
2018-06-03 20:39:06 +00:00
static FunctionPtr create ( const Context & ) { return std : : make_shared < FunctionReinterpretAsStringImpl > ( ) ; }
2017-04-01 07:20:54 +00:00
String getName ( ) const override
{
return name ;
}
size_t getNumberOfArguments ( ) const override { return 1 ; }
DataTypePtr getReturnTypeImpl ( const DataTypes & arguments ) const override
{
2018-01-13 22:26:49 +00:00
const IDataType & type = * arguments [ 0 ] ;
2017-04-01 07:20:54 +00:00
2018-01-13 22:26:49 +00:00
if ( type . isValueUnambiguouslyRepresentedInContiguousMemoryRegion ( ) )
return std : : make_shared < DataTypeString > ( ) ;
throw Exception ( " Cannot reinterpret " + type . getName ( ) + " as String because it is not contiguous in memory " , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
2017-04-01 07:20:54 +00:00
}
2018-01-13 22:26:49 +00:00
void executeToString ( const IColumn & src , ColumnString & dst )
{
size_t rows = src . size ( ) ;
2018-11-25 00:08:50 +00:00
ColumnString : : Chars & data_to = dst . getChars ( ) ;
2018-01-13 22:26:49 +00:00
ColumnString : : Offsets & offsets_to = dst . getOffsets ( ) ;
offsets_to . resize ( rows ) ;
2017-12-16 05:21:04 +00:00
2018-01-13 22:26:49 +00:00
ColumnString : : Offset offset = 0 ;
for ( size_t i = 0 ; i < rows ; + + i )
2017-04-01 07:20:54 +00:00
{
2018-01-13 22:26:49 +00:00
StringRef data = src . getDataAt ( i ) ;
2018-01-14 00:12:23 +00:00
/// Cut trailing zero bytes.
while ( data . size & & data . data [ data . size - 1 ] = = 0 )
- - data . size ;
2018-01-13 22:26:49 +00:00
data_to . resize ( offset + data . size + 1 ) ;
memcpySmallAllowReadWriteOverflow15 ( & data_to [ offset ] , data . data , data . size ) ;
offset + = data . size ;
data_to [ offset ] = 0 ;
+ + offset ;
offsets_to [ i ] = offset ;
2017-04-01 07:20:54 +00:00
}
}
2017-07-23 08:40:43 +00:00
bool useDefaultImplementationForConstants ( ) const override { return true ; }
2018-04-24 07:16:39 +00:00
void executeImpl ( Block & block , const ColumnNumbers & arguments , size_t result , size_t /*input_rows_count*/ ) override
2018-01-14 00:12:23 +00:00
{
const IColumn & src = * block . getByPosition ( arguments [ 0 ] ) . column ;
MutableColumnPtr dst = block . getByPosition ( result ) . type - > createColumn ( ) ;
if ( ColumnString * dst_concrete = typeid_cast < ColumnString * > ( dst . get ( ) ) )
executeToString ( src , * dst_concrete ) ;
else
throw Exception ( " Illegal column " + src . getName ( ) + " of argument of function " + getName ( ) , ErrorCodes : : ILLEGAL_COLUMN ) ;
block . getByPosition ( result ) . column = std : : move ( dst ) ;
}
} ;
template < typename Name >
class FunctionReinterpretAsFixedStringImpl : public IFunction
{
public :
static constexpr auto name = Name : : name ;
2018-06-03 20:39:06 +00:00
static FunctionPtr create ( const Context & ) { return std : : make_shared < FunctionReinterpretAsFixedStringImpl > ( ) ; }
2018-01-14 00:12:23 +00:00
String getName ( ) const override
{
return name ;
}
size_t getNumberOfArguments ( ) const override { return 1 ; }
DataTypePtr getReturnTypeImpl ( const DataTypes & arguments ) const override
{
const IDataType & type = * arguments [ 0 ] ;
if ( type . isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion ( ) )
return std : : make_shared < DataTypeFixedString > ( type . getSizeOfValueInMemory ( ) ) ;
throw Exception ( " Cannot reinterpret " + type . getName ( ) + " as FixedString because it is not fixed size and contiguous in memory " , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
}
void executeToFixedString ( const IColumn & src , ColumnFixedString & dst , size_t n )
{
size_t rows = src . size ( ) ;
2018-11-25 00:08:50 +00:00
ColumnFixedString : : Chars & data_to = dst . getChars ( ) ;
2018-01-14 00:12:23 +00:00
data_to . resize ( n * rows ) ;
ColumnFixedString : : Offset offset = 0 ;
for ( size_t i = 0 ; i < rows ; + + i )
{
StringRef data = src . getDataAt ( i ) ;
memcpySmallAllowReadWriteOverflow15 ( & data_to [ offset ] , data . data , n ) ;
offset + = n ;
}
}
bool useDefaultImplementationForConstants ( ) const override { return true ; }
2018-04-24 07:16:39 +00:00
void executeImpl ( Block & block , const ColumnNumbers & arguments , size_t result , size_t /*input_rows_count*/ ) override
2017-04-01 07:20:54 +00:00
{
2018-01-13 22:26:49 +00:00
const IColumn & src = * block . getByPosition ( arguments [ 0 ] ) . column ;
MutableColumnPtr dst = block . getByPosition ( result ) . type - > createColumn ( ) ;
if ( ColumnFixedString * dst_concrete = typeid_cast < ColumnFixedString * > ( dst . get ( ) ) )
executeToFixedString ( src , * dst_concrete , dst_concrete - > getN ( ) ) ;
else
throw Exception ( " Illegal column " + src . getName ( ) + " of argument of function " + getName ( ) , ErrorCodes : : ILLEGAL_COLUMN ) ;
block . getByPosition ( result ) . column = std : : move ( dst ) ;
2017-04-01 07:20:54 +00:00
}
2012-12-26 14:31:03 +00:00
} ;
2017-07-23 08:40:43 +00:00
2017-09-15 12:16:12 +00:00
template < typename ToDataType , typename Name >
2012-12-26 14:31:03 +00:00
class FunctionReinterpretStringAs : public IFunction
{
public :
2017-04-01 07:20:54 +00:00
static constexpr auto name = Name : : name ;
2018-06-03 20:39:06 +00:00
static FunctionPtr create ( const Context & ) { return std : : make_shared < FunctionReinterpretStringAs > ( ) ; }
2017-04-01 07:20:54 +00:00
using ToFieldType = typename ToDataType : : FieldType ;
String getName ( ) const override
{
return name ;
}
size_t getNumberOfArguments ( ) const override { return 1 ; }
DataTypePtr getReturnTypeImpl ( const DataTypes & arguments ) const override
{
2017-12-09 14:01:42 +00:00
const IDataType & type = * arguments [ 0 ] ;
2018-09-07 14:37:26 +00:00
if ( ! isStringOrFixedString ( type ) )
2017-12-09 14:01:42 +00:00
throw Exception ( " Cannot reinterpret " + type . getName ( ) + " as " + ToDataType ( ) . getName ( ) , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
2017-04-01 07:20:54 +00:00
return std : : make_shared < ToDataType > ( ) ;
}
2017-07-23 08:40:43 +00:00
bool useDefaultImplementationForConstants ( ) const override { return true ; }
2018-04-24 07:16:39 +00:00
void executeImpl ( Block & block , const ColumnNumbers & arguments , size_t result , size_t /*input_rows_count*/ ) override
2017-04-01 07:20:54 +00:00
{
2017-12-16 05:52:47 +00:00
if ( const ColumnString * col_from = typeid_cast < const ColumnString * > ( block . getByPosition ( arguments [ 0 ] ) . column . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
2017-12-14 01:43:19 +00:00
auto col_res = ColumnVector < ToFieldType > : : create ( ) ;
2017-04-01 07:20:54 +00:00
2018-11-25 00:08:50 +00:00
const ColumnString : : Chars & data_from = col_from - > getChars ( ) ;
2017-12-16 05:52:47 +00:00
const ColumnString : : Offsets & offsets_from = col_from - > getOffsets ( ) ;
2017-04-01 07:20:54 +00:00
size_t size = offsets_from . size ( ) ;
2017-12-15 21:32:25 +00:00
typename ColumnVector < ToFieldType > : : Container & vec_res = col_res - > getData ( ) ;
2017-04-01 07:20:54 +00:00
vec_res . resize ( size ) ;
size_t offset = 0 ;
for ( size_t i = 0 ; i < size ; + + i )
{
ToFieldType value = 0 ;
2017-09-01 17:21:03 +00:00
memcpy ( & value , & data_from [ offset ] , std : : min ( static_cast < UInt64 > ( sizeof ( ToFieldType ) ) , offsets_from [ i ] - offset - 1 ) ) ;
2017-04-01 07:20:54 +00:00
vec_res [ i ] = value ;
offset = offsets_from [ i ] ;
}
2017-12-16 05:21:04 +00:00
block . getByPosition ( result ) . column = std : : move ( col_res ) ;
2017-04-01 07:20:54 +00:00
}
2019-01-04 12:10:00 +00:00
else if ( const ColumnFixedString * col_from_fixed = typeid_cast < const ColumnFixedString * > ( block . getByPosition ( arguments [ 0 ] ) . column . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
2017-12-14 01:43:19 +00:00
auto col_res = ColumnVector < ToFieldType > : : create ( ) ;
2017-04-01 07:20:54 +00:00
2019-01-04 12:10:00 +00:00
const ColumnString : : Chars & data_from = col_from_fixed - > getChars ( ) ;
size_t step = col_from_fixed - > getN ( ) ;
2017-04-01 07:20:54 +00:00
size_t size = data_from . size ( ) / step ;
2017-12-15 21:32:25 +00:00
typename ColumnVector < ToFieldType > : : Container & vec_res = col_res - > getData ( ) ;
2017-04-01 07:20:54 +00:00
vec_res . resize ( size ) ;
size_t offset = 0 ;
size_t copy_size = std : : min ( step , sizeof ( ToFieldType ) ) ;
for ( size_t i = 0 ; i < size ; + + i )
{
ToFieldType value = 0 ;
memcpy ( & value , & data_from [ offset ] , copy_size ) ;
vec_res [ i ] = value ;
offset + = step ;
}
2017-12-16 05:21:04 +00:00
block . getByPosition ( result ) . column = std : : move ( col_res ) ;
2017-04-01 07:20:54 +00:00
}
else
{
2017-07-21 06:35:58 +00:00
throw Exception ( " Illegal column " + block . getByPosition ( arguments [ 0 ] ) . column - > getName ( )
2017-12-16 05:52:47 +00:00
+ " of argument of function " + getName ( ) ,
ErrorCodes : : ILLEGAL_COLUMN ) ;
2017-04-01 07:20:54 +00:00
}
}
2012-12-26 14:31:03 +00:00
} ;
2017-06-13 02:06:53 +00:00
struct NameReinterpretAsUInt8 { static constexpr auto name = " reinterpretAsUInt8 " ; } ;
struct NameReinterpretAsUInt16 { static constexpr auto name = " reinterpretAsUInt16 " ; } ;
struct NameReinterpretAsUInt32 { static constexpr auto name = " reinterpretAsUInt32 " ; } ;
struct NameReinterpretAsUInt64 { static constexpr auto name = " reinterpretAsUInt64 " ; } ;
struct NameReinterpretAsInt8 { static constexpr auto name = " reinterpretAsInt8 " ; } ;
struct NameReinterpretAsInt16 { static constexpr auto name = " reinterpretAsInt16 " ; } ;
struct NameReinterpretAsInt32 { static constexpr auto name = " reinterpretAsInt32 " ; } ;
struct NameReinterpretAsInt64 { static constexpr auto name = " reinterpretAsInt64 " ; } ;
struct NameReinterpretAsFloat32 { static constexpr auto name = " reinterpretAsFloat32 " ; } ;
struct NameReinterpretAsFloat64 { static constexpr auto name = " reinterpretAsFloat64 " ; } ;
2017-04-01 07:20:54 +00:00
struct NameReinterpretAsDate { static constexpr auto name = " reinterpretAsDate " ; } ;
struct NameReinterpretAsDateTime { static constexpr auto name = " reinterpretAsDateTime " ; } ;
2017-06-13 02:06:53 +00:00
struct NameReinterpretAsString { static constexpr auto name = " reinterpretAsString " ; } ;
2018-01-14 00:12:23 +00:00
struct NameReinterpretAsFixedString { static constexpr auto name = " reinterpretAsFixedString " ; } ;
2017-06-13 02:06:53 +00:00
using FunctionReinterpretAsUInt8 = FunctionReinterpretStringAs < DataTypeUInt8 , NameReinterpretAsUInt8 > ;
using FunctionReinterpretAsUInt16 = FunctionReinterpretStringAs < DataTypeUInt16 , NameReinterpretAsUInt16 > ;
using FunctionReinterpretAsUInt32 = FunctionReinterpretStringAs < DataTypeUInt32 , NameReinterpretAsUInt32 > ;
using FunctionReinterpretAsUInt64 = FunctionReinterpretStringAs < DataTypeUInt64 , NameReinterpretAsUInt64 > ;
using FunctionReinterpretAsInt8 = FunctionReinterpretStringAs < DataTypeInt8 , NameReinterpretAsInt8 > ;
using FunctionReinterpretAsInt16 = FunctionReinterpretStringAs < DataTypeInt16 , NameReinterpretAsInt16 > ;
using FunctionReinterpretAsInt32 = FunctionReinterpretStringAs < DataTypeInt32 , NameReinterpretAsInt32 > ;
using FunctionReinterpretAsInt64 = FunctionReinterpretStringAs < DataTypeInt64 , NameReinterpretAsInt64 > ;
using FunctionReinterpretAsFloat32 = FunctionReinterpretStringAs < DataTypeFloat32 , NameReinterpretAsFloat32 > ;
using FunctionReinterpretAsFloat64 = FunctionReinterpretStringAs < DataTypeFloat64 , NameReinterpretAsFloat64 > ;
using FunctionReinterpretAsDate = FunctionReinterpretStringAs < DataTypeDate , NameReinterpretAsDate > ;
using FunctionReinterpretAsDateTime = FunctionReinterpretStringAs < DataTypeDateTime , NameReinterpretAsDateTime > ;
2016-05-28 10:35:44 +00:00
using FunctionReinterpretAsString = FunctionReinterpretAsStringImpl < NameReinterpretAsString > ;
2018-01-14 00:12:23 +00:00
using FunctionReinterpretAsFixedString = FunctionReinterpretAsStringImpl < NameReinterpretAsFixedString > ;
2014-06-26 00:58:14 +00:00
2012-12-26 14:31:03 +00:00
}