Merge pull request #16657 from ucasFL/reinterpret

Allow reinterpret between integers and floats of the same size
This commit is contained in:
alexey-milovidov 2020-11-05 23:21:59 +03:00 committed by GitHub
commit 7bda6dba3c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 102 additions and 30 deletions

View File

@ -3,13 +3,13 @@ namespace DB
class FunctionFactory;
void registerFunctionsReinterpretStringAs(FunctionFactory & factory);
void registerFunctionsReinterpretAs(FunctionFactory & factory);
void registerFunctionReinterpretAsString(FunctionFactory & factory);
void registerFunctionReinterpretAsFixedString(FunctionFactory & factory);
void registerFunctionsReinterpret(FunctionFactory & factory)
{
registerFunctionsReinterpretStringAs(factory);
registerFunctionsReinterpretAs(factory);
registerFunctionReinterpretAsString(factory);
registerFunctionReinterpretAsFixedString(factory);
}

View File

@ -1,4 +1,5 @@
#include <Functions/FunctionFactory.h>
#include <Functions/castTypeToEither.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
@ -14,6 +15,7 @@
#include <Common/typeid_cast.h>
#include <Common/memcpySmall.h>
#include <common/unaligned.h>
namespace DB
{
@ -25,30 +27,60 @@ namespace ErrorCodes
namespace
{
template <typename ToDataType, typename Name>
class FunctionReinterpretStringAs : public IFunction
template <typename ToDataType, typename Name, bool support_between_float_integer>
class FunctionReinterpretAs : public IFunction
{
template <typename F>
static bool castType(const IDataType * type, F && f)
{
return castTypeToEither<DataTypeUInt32, DataTypeInt32, DataTypeUInt64, DataTypeInt64, DataTypeFloat32, DataTypeFloat64>(
type, std::forward<F>(f));
}
template <typename From, typename To>
static void reinterpretImpl(const PaddedPODArray<From> & from, PaddedPODArray<To> & to)
{
size_t size = from.size();
to.resize(size);
for (size_t i = 0; i < size; ++i)
{
to[i] = unalignedLoad<To>(&(from.data()[i]));
}
}
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionReinterpretStringAs>(); }
static FunctionPtr create(const Context &) { return std::make_shared<FunctionReinterpretAs>(); }
using ToFieldType = typename ToDataType::FieldType;
using ColumnType = typename ToDataType::ColumnType;
String getName() const override
{
return name;
}
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 (!isStringOrFixedString(type))
throw Exception("Cannot reinterpret " + type.getName() + " as " + ToDataType().getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if constexpr (support_between_float_integer)
{
if (!isStringOrFixedString(type) && !isNumber(type))
throw Exception(
"Cannot reinterpret " + type.getName() + " as " + ToDataType().getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (isNumber(type))
{
if (type.getSizeOfValueInMemory() != ToDataType{}.getSizeOfValueInMemory())
throw Exception(
"Cannot reinterpret " + type.getName() + " as " + ToDataType().getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
else
{
if (!isStringOrFixedString(type))
throw Exception(
"Cannot reinterpret " + type.getName() + " as " + ToDataType().getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
return std::make_shared<ToDataType>();
}
@ -99,10 +131,35 @@ public:
return col_res;
}
else if constexpr (support_between_float_integer)
{
ColumnPtr res;
if (castType(arguments[0].type.get(), [&](const auto & type)
{
using DataType = std::decay_t<decltype(type)>;
using T = typename DataType::FieldType;
const ColumnVector<T> * col = checkAndGetColumn<ColumnVector<T>>(arguments[0].column.get());
auto col_res = ColumnType::create();
reinterpretImpl(col->getData(), col_res->getData());
res = std::move(col_res);
return true;
}))
{
return res;
}
else
{
throw Exception("Illegal column " + arguments[0].column->getName()
+ " of argument of function " + getName(),
throw Exception(
"Illegal column " + arguments[0].column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
}
else
{
throw Exception(
"Illegal column " + arguments[0].column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
}
@ -123,23 +180,22 @@ struct NameReinterpretAsDate { static constexpr auto name = "reinterpretA
struct NameReinterpretAsDateTime { static constexpr auto name = "reinterpretAsDateTime"; };
struct NameReinterpretAsUUID { static constexpr auto name = "reinterpretAsUUID"; };
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>;
using FunctionReinterpretAsUUID = FunctionReinterpretStringAs<DataTypeUUID, NameReinterpretAsUUID>;
using FunctionReinterpretAsUInt8 = FunctionReinterpretAs<DataTypeUInt8, NameReinterpretAsUInt8, false>;
using FunctionReinterpretAsUInt16 = FunctionReinterpretAs<DataTypeUInt16, NameReinterpretAsUInt16, false>;
using FunctionReinterpretAsUInt32 = FunctionReinterpretAs<DataTypeUInt32, NameReinterpretAsUInt32, true>;
using FunctionReinterpretAsUInt64 = FunctionReinterpretAs<DataTypeUInt64, NameReinterpretAsUInt64, true>;
using FunctionReinterpretAsInt8 = FunctionReinterpretAs<DataTypeInt8, NameReinterpretAsInt8, false>;
using FunctionReinterpretAsInt16 = FunctionReinterpretAs<DataTypeInt16, NameReinterpretAsInt16, false>;
using FunctionReinterpretAsInt32 = FunctionReinterpretAs<DataTypeInt32, NameReinterpretAsInt32, true>;
using FunctionReinterpretAsInt64 = FunctionReinterpretAs<DataTypeInt64, NameReinterpretAsInt64, true>;
using FunctionReinterpretAsFloat32 = FunctionReinterpretAs<DataTypeFloat32, NameReinterpretAsFloat32, true>;
using FunctionReinterpretAsFloat64 = FunctionReinterpretAs<DataTypeFloat64, NameReinterpretAsFloat64, true>;
using FunctionReinterpretAsDate = FunctionReinterpretAs<DataTypeDate, NameReinterpretAsDate, false>;
using FunctionReinterpretAsDateTime = FunctionReinterpretAs<DataTypeDateTime, NameReinterpretAsDateTime, false>;
using FunctionReinterpretAsUUID = FunctionReinterpretAs<DataTypeUUID, NameReinterpretAsUUID, false>;
}
void registerFunctionsReinterpretStringAs(FunctionFactory & factory)
void registerFunctionsReinterpretAs(FunctionFactory & factory)
{
factory.registerFunction<FunctionReinterpretAsUInt8>();
factory.registerFunction<FunctionReinterpretAsUInt16>();

View File

@ -386,9 +386,9 @@ SRCS(
registerFunctionsTuple.cpp
registerFunctionsUnixTimestamp64.cpp
registerFunctionsVisitParam.cpp
reinterpretAs.cpp
reinterpretAsFixedString.cpp
reinterpretAsString.cpp
reinterpretStringAs.cpp
repeat.cpp
replaceAll.cpp
replaceOne.cpp

View File

@ -0,0 +1,8 @@
1.72999e-40
1067320848
1.72999e-40
1067320848
6.09954e-319
4608238783128613432
6.09954e-319
4608238783128613432

View File

@ -0,0 +1,8 @@
SELECT reinterpretAsFloat32(CAST(123456 AS UInt32));
SELECT reinterpretAsUInt32(CAST(1.23456 AS Float32));
SELECT reinterpretAsFloat32(CAST(123456 AS Int32));
SELECT reinterpretAsInt32(CAST(1.23456 AS Float32));
SELECT reinterpretAsFloat64(CAST(123456 AS UInt64));
SELECT reinterpretAsUInt64(CAST(1.23456 AS Float64));
SELECT reinterpretAsFloat64(CAST(123456 AS Int64));
SELECT reinterpretAsInt64(CAST(1.23456 AS Float64));