Added function accurate cast or null

This commit is contained in:
Maksim Kita 2020-11-05 22:09:17 +03:00
parent 5f424f7c5c
commit d636e1618b
6 changed files with 378 additions and 136 deletions

View File

@ -33,7 +33,7 @@ struct AvgFraction
/// Allow division by zero as sometimes we need to return NaN.
/// Invoked only is either Numerator or Denominator are Decimal.
Float64 NO_SANITIZE_UNDEFINED divideIfAnyDecimal(UInt32 num_scale, UInt32 denom_scale) const
Float64 NO_SANITIZE_UNDEFINED divideIfAnyDecimal(UInt32 num_scale, UInt32 denom_scale [[maybe_unused]]) const
{
if constexpr (IsDecimalNumber<Numerator> && IsDecimalNumber<Denominator>)
{

View File

@ -11,6 +11,11 @@ FunctionOverloadResolverImplPtr CastOverloadResolver::create(const Context & con
return createImpl(context.getSettingsRef().cast_keep_nullable);
}
FunctionOverloadResolverImplPtr AccurateCastOverloadResolver::create(const Context &)
{
return std::make_unique<AccurateCastOverloadResolver>();
}
void registerFunctionFixedString(FunctionFactory & factory);
void registerFunctionsConversion(FunctionFactory & factory)
@ -45,6 +50,7 @@ void registerFunctionsConversion(FunctionFactory & factory)
factory.registerFunction<FunctionToUnixTimestamp>();
factory.registerFunction<CastOverloadResolver>(FunctionFactory::CaseInsensitive);
factory.registerFunction<AccurateCastOverloadResolver>();
factory.registerFunction<FunctionToUInt8OrZero>();
factory.registerFunction<FunctionToUInt16OrZero>();

View File

@ -36,6 +36,7 @@
#include <Common/FieldVisitors.h>
#include <Common/assert_cast.h>
#include <Common/quoteString.h>
#include <Core/AccurateComparison.h>
#include <Functions/IFunctionAdaptors.h>
#include <Functions/FunctionsMiscellaneous.h>
#include <Functions/FunctionHelpers.h>
@ -94,6 +95,16 @@ inline UInt32 extractToDecimalScale(const ColumnWithTypeAndName & named_column)
/// Function toUnixTimestamp has exactly the same implementation as toDateTime of String type.
struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; };
struct AccurateAdditions
{
UInt32 scale;
};
enum class ConvertStrategy
{
NonAccurate,
Accurate
};
/** 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)
@ -109,6 +120,8 @@ struct ConvertImpl
const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/,
Additions additions [[maybe_unused]] = Additions())
{
static constexpr auto convert_strategy
= std::is_same_v<Additions, AccurateAdditions> ? ConvertStrategy::Accurate : ConvertStrategy::NonAccurate;
const ColumnWithTypeAndName & named_from = arguments[0];
using ColVecFrom = typename FromDataType::ColumnType;
@ -136,7 +149,16 @@ struct ConvertImpl
typename ColVecTo::MutablePtr col_to = nullptr;
if constexpr (IsDataTypeDecimal<ToDataType>)
{
UInt32 scale = additions;
UInt32 scale;
if constexpr (convert_strategy == ConvertStrategy::Accurate)
{
scale = additions.scale;
}
else
{
scale = additions;
}
col_to = ColVecTo::create(0, scale);
}
else
@ -147,6 +169,14 @@ struct ConvertImpl
size_t size = vec_from.size();
vec_to.resize(size);
ColumnUInt8::MutablePtr col_null_map_to;
ColumnUInt8::Container * vec_null_map_to [[maybe_unused]] = nullptr;
if constexpr (convert_strategy == ConvertStrategy::Accurate)
{
col_null_map_to = ColumnUInt8::create(size, false);
vec_null_map_to = &col_null_map_to->getData();
}
for (size_t i = 0; i < size; ++i)
{
if constexpr (IsDataTypeDecimal<FromDataType> || IsDataTypeDecimal<ToDataType>)
@ -158,25 +188,61 @@ struct ConvertImpl
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);
{
if constexpr (convert_strategy == ConvertStrategy::Accurate)
vec_null_map_to[i] = true;
else
throw Exception("Unsupported data type in conversion function", ErrorCodes::CANNOT_CONVERT_TYPE);
}
}
else if constexpr (is_big_int_v<FromFieldType> || is_big_int_v<ToFieldType>)
{
if constexpr (std::is_same_v<FromFieldType, UInt128> || std::is_same_v<ToFieldType, UInt128>)
throw Exception("Unexpected UInt128 to big int conversion", ErrorCodes::NOT_IMPLEMENTED);
{
if constexpr (convert_strategy == ConvertStrategy::Accurate)
vec_null_map_to[i] = true;
else
throw Exception("Unexpected UInt128 to big int conversion", ErrorCodes::NOT_IMPLEMENTED);
}
/// If From Data is Nan or Inf, throw exception
else if (!isFinite(vec_from[i]))
throw Exception("Unexpected inf or nan to big int conversion", ErrorCodes::NOT_IMPLEMENTED);
{
if constexpr (convert_strategy == ConvertStrategy::Accurate)
vec_null_map_to[i] = true;
else
throw Exception("Unexpected inf or nan to big int conversion", ErrorCodes::NOT_IMPLEMENTED);
}
else
vec_to[i] = bigint_cast<ToFieldType>(vec_from[i]);
}
else if constexpr (std::is_same_v<ToFieldType, UInt128> && sizeof(FromFieldType) <= sizeof(UInt64))
vec_to[i] = static_cast<ToFieldType>(static_cast<UInt64>(vec_from[i]));
else
vec_to[i] = static_cast<ToFieldType>(vec_from[i]);
{
if constexpr (convert_strategy == ConvertStrategy::Accurate)
{
/// TODO: Add support for big integers
if (accurate::greaterOp(std::numeric_limits<ToFieldType>::max(), vec_from[i]) ||
accurate::greaterOp(vec_from[i], std::numeric_limits<ToFieldType>::min()))
{
vec_null_map_to[i] = true;
}
else
{
vec_to[i] = static_cast<ToFieldType>(vec_from[i]);
}
}
else
{
vec_to[i] = static_cast<ToFieldType>(vec_from[i]);
}
}
}
return col_to;
if constexpr (convert_strategy == ConvertStrategy::Accurate)
return ColumnNullable::create(std::move(col_to), std::move(col_null_map_to));
else
return col_to;
}
else
throw Exception("Illegal column " + named_from.column->getName() + " of first argument of function " + Name::name,
@ -943,7 +1009,9 @@ struct ConvertImpl<DataTypeString, DataTypeUInt32, NameToUnixTimestamp>
template <typename T, typename Name>
struct ConvertImpl<std::enable_if_t<!T::is_parametric, T>, T, Name>
{
static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/)
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;
}
@ -1940,9 +2008,11 @@ public:
using Diagnostic = ExecutableFunctionCast::Diagnostic;
FunctionCast(const char * name_, MonotonicityForRange && monotonicity_for_range_
, const DataTypes & argument_types_, const DataTypePtr & return_type_, std::optional<Diagnostic> diagnostic_)
: name(name_), monotonicity_for_range(monotonicity_for_range_)
, const DataTypes & argument_types_, const DataTypePtr & return_type_
, std::optional<Diagnostic> diagnostic_, bool is_accurate_cast_or_null_)
: name(name_), monotonicity_for_range(std::move(monotonicity_for_range_))
, argument_types(argument_types_), return_type(return_type_), diagnostic(std::move(diagnostic_))
, is_accurate_cast_or_null(is_accurate_cast_or_null_)
{
}
@ -1989,70 +2059,98 @@ private:
DataTypePtr return_type;
std::optional<Diagnostic> diagnostic;
bool is_accurate_cast_or_null;
template <typename DataType>
WrapperType createWrapper(const DataTypePtr & from_type, const DataType * const, bool requested_result_is_nullable) const
WrapperType createFunctionAdaptor(FunctionPtr function, const DataTypePtr & from_type) const
{
FunctionPtr function;
auto function_adaptor = FunctionOverloadResolverAdaptor(std::make_unique<DefaultOverloadResolver>(function))
.build({ColumnWithTypeAndName{nullptr, from_type, ""}});
return [function_adaptor] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count)
{
return function_adaptor->execute(arguments, result_type, input_rows_count);
};
}
WrapperType createToNullableColumnWrapper() const
{
return [] (ColumnsWithTypeAndName &, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count)
{
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));
};
}
template <typename ToDataType>
WrapperType createWrapper(const DataTypePtr & from_type, const ToDataType * const, bool requested_result_is_nullable) const
{
TypeIndex from_type_index = from_type->getTypeId();
WhichDataType which(from_type_index);
bool can_apply_accurate_cast = is_accurate_cast_or_null && (which.isNativeInt() || which.isNativeUInt() || which.isFloat());
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.
function = FunctionConvertFromString<DataType, NameCast, ConvertFromStringExceptionMode::Null>::create();
FunctionPtr function = FunctionConvertFromString<ToDataType, NameCast, ConvertFromStringExceptionMode::Null>::create();
return createFunctionAdaptor(function, from_type);
}
else
function = FunctionTo<DataType>::Type::create();
auto function_adaptor =
FunctionOverloadResolverAdaptor(std::make_unique<DefaultOverloadResolver>(function))
.build({ColumnWithTypeAndName{nullptr, from_type, ""}});
return [function_adaptor] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count)
else if (!can_apply_accurate_cast)
{
return function_adaptor->execute(arguments, result_type, input_rows_count);
FunctionPtr function = FunctionTo<ToDataType>::Type::create();
return createFunctionAdaptor(function, from_type);
}
return [from_type_index]
(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count)
{
ColumnPtr result_column;
auto res = callOnIndexAndDataType<ToDataType>(from_type_index, [&](const auto & types) -> bool
{
using Types = std::decay_t<decltype(types)>;
using LeftDataType = typename Types::LeftType;
using RightDataType = typename Types::RightType;
if constexpr (IsDataTypeNumber<LeftDataType> && IsDataTypeNumber<RightDataType>)
{
result_column = ConvertImpl<LeftDataType, RightDataType, NameCast>::execute(arguments, result_type, input_rows_count, AccurateAdditions());
return true;
}
return false;
});
/// Additionally check if callOnIndexAndDataType wasn't called at all.
if (!res)
{
// throw Exception{"Conversion from " + std::string(getTypeName(from_type)) + " to " + to_type->getName() +
// " is not supported", ErrorCodes::CANNOT_CONVERT_TYPE};
}
return result_column;
};
}
static WrapperType createStringWrapper(const DataTypePtr & from_type)
WrapperType createStringWrapper(const DataTypePtr & from_type) const
{
FunctionPtr function = FunctionToString::create();
auto function_adaptor =
FunctionOverloadResolverAdaptor(std::make_unique<DefaultOverloadResolver>(function))
.build({ColumnWithTypeAndName{nullptr, from_type, ""}});
return [function_adaptor] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count)
{
return function_adaptor->execute(arguments, result_type, input_rows_count);
};
return createFunctionAdaptor(function, from_type);
}
static WrapperType createFixedStringWrapper(const DataTypePtr & from_type, const size_t N)
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};
return [N] (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t /*input_rows_count*/)
bool exception_mode_null = is_accurate_cast_or_null;
return [exception_mode_null, N] (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t /*input_rows_count*/)
{
return FunctionToFixedString::executeForN(arguments, N);
};
}
static WrapperType createUUIDWrapper(const DataTypePtr & from_type, const DataTypeUUID * const, bool requested_result_is_nullable)
{
if (requested_result_is_nullable)
throw Exception{"CAST AS Nullable(UUID) is not implemented", ErrorCodes::NOT_IMPLEMENTED};
FunctionPtr function = FunctionTo<DataTypeUUID>::Type::create();
auto function_adaptor =
FunctionOverloadResolverAdaptor(std::make_unique<DefaultOverloadResolver>(function))
.build({ColumnWithTypeAndName{nullptr, from_type, ""}});
return [function_adaptor] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count)
{
return function_adaptor->execute(arguments, result_type, input_rows_count);
if (exception_mode_null)
return FunctionToFixedString::executeForN<ConvertToFixedStringExceptionMode::Null>(arguments, N);
else
return FunctionToFixedString::executeForN<ConvertToFixedStringExceptionMode::Throw>(arguments, N);
};
}
@ -2070,11 +2168,19 @@ private:
which.isFloat() ||
which.isDateOrDateTime() ||
which.isStringOrFixedString();
if (!ok)
throw Exception{"Conversion from " + from_type->getName() + " to " + to_type->getName() + " is not supported",
ErrorCodes::CANNOT_CONVERT_TYPE};
if (!ok) {
if (is_accurate_cast_or_null)
return createToNullableColumnWrapper();
else
throw Exception{"Conversion from " + from_type->getName() + " to " + to_type->getName() + " is not supported",
ErrorCodes::CANNOT_CONVERT_TYPE};
}
return [type_index, scale, to_type] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count)
auto nullable_column_wrapper = createToNullableColumnWrapper();
bool is_accurate_cast = is_accurate_cast_or_null;
return [is_accurate_cast, nullable_column_wrapper, type_index, scale, to_type]
(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *columnNullable, size_t input_rows_count)
{
ColumnPtr result_column;
auto res = callOnIndexAndDataType<ToDataType>(type_index, [&](const auto & types) -> bool
@ -2090,15 +2196,18 @@ private:
/// Additionally check if callOnIndexAndDataType wasn't called at all.
if (!res)
{
throw Exception{"Conversion from " + std::string(getTypeName(type_index)) + " to " + to_type->getName() +
" is not supported", ErrorCodes::CANNOT_CONVERT_TYPE};
if (is_accurate_cast)
return nullable_column_wrapper(arguments, result_type, columnNullable, input_rows_count);
else
throw Exception{"Conversion from " + std::string(getTypeName(type_index)) + " to " + to_type->getName() +
" is not supported", ErrorCodes::CANNOT_CONVERT_TYPE};
}
return result_column;
};
}
static WrapperType createAggregateFunctionWrapper(const DataTypePtr & from_type_untyped, const DataTypeAggregateFunction * to_type)
WrapperType createAggregateFunctionWrapper(const DataTypePtr & from_type_untyped, const DataTypeAggregateFunction * to_type) const
{
/// Conversion from String through parsing.
if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
@ -2109,8 +2218,13 @@ private:
};
}
else
throw Exception{"Conversion from " + from_type_untyped->getName() + " to " + to_type->getName() +
" is not supported", ErrorCodes::CANNOT_CONVERT_TYPE};
{
if (is_accurate_cast_or_null)
return createToNullableColumnWrapper();
else
throw Exception{"Conversion from " + from_type_untyped->getName() + " to " + to_type->getName() +
" is not supported", ErrorCodes::CANNOT_CONVERT_TYPE};
}
}
WrapperType createArrayWrapper(const DataTypePtr & from_type_untyped, const DataTypeArray * to_type) const
@ -2235,17 +2349,16 @@ private:
else if (isNativeNumber(from_type) || isEnum(from_type))
{
auto function = Function::create();
auto func_or_adaptor = FunctionOverloadResolverAdaptor(std::make_unique<DefaultOverloadResolver>(function))
.build(ColumnsWithTypeAndName{{nullptr, from_type, "" }});
return [func_or_adaptor] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count)
{
return func_or_adaptor->execute(arguments, result_type, input_rows_count);
};
return createFunctionAdaptor(function, from_type);
}
else
throw Exception{"Conversion from " + from_type->getName() + " to " + to_type->getName() +
" is not supported", ErrorCodes::CANNOT_CONVERT_TYPE};
{
if (is_accurate_cast_or_null)
return createToNullableColumnWrapper();
else
throw Exception{"Conversion from " + from_type->getName() + " to " + to_type->getName() +
" is not supported", ErrorCodes::CANNOT_CONVERT_TYPE};
}
}
template <typename EnumTypeFrom, typename EnumTypeTo>
@ -2346,7 +2459,16 @@ private:
if (from_type->onlyNull())
{
if (!to_nested->isNullable())
throw Exception{"Cannot convert NULL to a non-nullable type", ErrorCodes::CANNOT_CONVERT_TYPE};
{
if (is_accurate_cast_or_null)
{
return createToNullableColumnWrapper();
}
else
{
throw Exception{"Cannot convert NULL to a non-nullable type", ErrorCodes::CANNOT_CONVERT_TYPE};
}
}
return [](ColumnsWithTypeAndName &, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count)
{
@ -2536,7 +2658,8 @@ private:
std::is_same_v<ToDataType, DataTypeFloat32> ||
std::is_same_v<ToDataType, DataTypeFloat64> ||
std::is_same_v<ToDataType, DataTypeDate> ||
std::is_same_v<ToDataType, DataTypeDateTime>)
std::is_same_v<ToDataType, DataTypeDateTime> ||
std::is_same_v<ToDataType, DataTypeUUID>)
{
ret = createWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()), requested_result_is_nullable);
return true;
@ -2558,14 +2681,6 @@ private:
ret = createDecimalWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()));
return true;
}
if constexpr (std::is_same_v<ToDataType, DataTypeUUID>)
{
if (isStringOrFixedString(from_type))
{
ret = createUUIDWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()), requested_result_is_nullable);
return true;
}
}
return false;
};
@ -2591,8 +2706,68 @@ private:
break;
}
throw Exception{"Conversion from " + from_type->getName() + " to " + to_type->getName() + " is not supported",
ErrorCodes::CANNOT_CONVERT_TYPE};
if (is_accurate_cast_or_null)
return createToNullableColumnWrapper();
else
throw Exception{"Conversion from " + from_type->getName() + " to " + to_type->getName() + " is not supported",
ErrorCodes::CANNOT_CONVERT_TYPE};
}
};
class MonotonicityHelper
{
public:
using MonotonicityForRange = FunctionCast::MonotonicityForRange;
template <typename DataType>
static auto monotonicityForType(const DataType * const)
{
return FunctionTo<DataType>::Type::Monotonic::get;
}
static MonotonicityForRange getMonotonicityInformation(const DataTypePtr & from_type, const IDataType * to_type)
{
if (const auto type = checkAndGetDataType<DataTypeUInt8>(to_type))
return monotonicityForType(type);
if (const auto type = checkAndGetDataType<DataTypeUInt16>(to_type))
return monotonicityForType(type);
if (const auto type = checkAndGetDataType<DataTypeUInt32>(to_type))
return monotonicityForType(type);
if (const auto type = checkAndGetDataType<DataTypeUInt64>(to_type))
return monotonicityForType(type);
if (const auto type = checkAndGetDataType<DataTypeUInt256>(to_type))
return monotonicityForType(type);
if (const auto type = checkAndGetDataType<DataTypeInt8>(to_type))
return monotonicityForType(type);
if (const auto type = checkAndGetDataType<DataTypeInt16>(to_type))
return monotonicityForType(type);
if (const auto type = checkAndGetDataType<DataTypeInt32>(to_type))
return monotonicityForType(type);
if (const auto type = checkAndGetDataType<DataTypeInt64>(to_type))
return monotonicityForType(type);
if (const auto type = checkAndGetDataType<DataTypeInt128>(to_type))
return monotonicityForType(type);
if (const auto type = checkAndGetDataType<DataTypeInt256>(to_type))
return monotonicityForType(type);
if (const auto type = checkAndGetDataType<DataTypeFloat32>(to_type))
return monotonicityForType(type);
if (const auto type = checkAndGetDataType<DataTypeFloat64>(to_type))
return monotonicityForType(type);
if (const auto type = checkAndGetDataType<DataTypeDate>(to_type))
return monotonicityForType(type);
if (const auto type = checkAndGetDataType<DataTypeDateTime>(to_type))
return monotonicityForType(type);
if (const auto type = checkAndGetDataType<DataTypeString>(to_type))
return monotonicityForType(type);
if (isEnum(from_type))
{
if (const auto type = checkAndGetDataType<DataTypeEnum8>(to_type))
return monotonicityForType(type);
if (const auto type = checkAndGetDataType<DataTypeEnum16>(to_type))
return monotonicityForType(type);
}
/// other types like Null, FixedString, Array and Tuple have no monotonicity defined
return {};
}
};
@ -2630,8 +2805,8 @@ protected:
for (size_t i = 0; i < arguments.size(); ++i)
data_types[i] = arguments[i].type;
auto monotonicity = getMonotonicityInformation(arguments.front().type, return_type.get());
return std::make_unique<FunctionCast>(name, std::move(monotonicity), data_types, return_type, diagnostic);
auto monotonicity = MonotonicityHelper::getMonotonicityInformation(arguments.front().type, return_type.get());
return std::make_unique<FunctionCast>(name, std::move(monotonicity), data_types, return_type, diagnostic, false);
}
DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const override
@ -2660,57 +2835,57 @@ protected:
private:
bool keep_nullable;
std::optional<Diagnostic> diagnostic;
};
template <typename DataType>
static auto monotonicityForType(const DataType * const)
class AccurateCastOverloadResolver : public IFunctionOverloadResolverImpl
{
public:
using MonotonicityForRange = FunctionCast::MonotonicityForRange;
using Diagnostic = FunctionCast::Diagnostic;
static constexpr auto name = "accurateCastOrNull";
static FunctionOverloadResolverImplPtr create(const Context & context);
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
protected:
FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override
{
return FunctionTo<DataType>::Type::Monotonic::get;
DataTypes data_types(arguments.size());
for (size_t i = 0; i < arguments.size(); ++i)
data_types[i] = arguments[i].type;
auto monotonicity = MonotonicityHelper::getMonotonicityInformation(arguments.front().type, return_type.get());
return std::make_unique<FunctionCast>(name, std::move(monotonicity), data_types, return_type, std::optional<Diagnostic>(), true);
}
static MonotonicityForRange getMonotonicityInformation(const DataTypePtr & from_type, const IDataType * to_type)
DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const override
{
if (const auto * type = checkAndGetDataType<DataTypeUInt8>(to_type))
return monotonicityForType(type);
if (const auto * type = checkAndGetDataType<DataTypeUInt16>(to_type))
return monotonicityForType(type);
if (const auto * type = checkAndGetDataType<DataTypeUInt32>(to_type))
return monotonicityForType(type);
if (const auto * type = checkAndGetDataType<DataTypeUInt64>(to_type))
return monotonicityForType(type);
if (const auto * type = checkAndGetDataType<DataTypeUInt256>(to_type))
return monotonicityForType(type);
if (const auto * type = checkAndGetDataType<DataTypeInt8>(to_type))
return monotonicityForType(type);
if (const auto * type = checkAndGetDataType<DataTypeInt16>(to_type))
return monotonicityForType(type);
if (const auto * type = checkAndGetDataType<DataTypeInt32>(to_type))
return monotonicityForType(type);
if (const auto * type = checkAndGetDataType<DataTypeInt64>(to_type))
return monotonicityForType(type);
if (const auto * type = checkAndGetDataType<DataTypeInt128>(to_type))
return monotonicityForType(type);
if (const auto * type = checkAndGetDataType<DataTypeInt256>(to_type))
return monotonicityForType(type);
if (const auto * type = checkAndGetDataType<DataTypeFloat32>(to_type))
return monotonicityForType(type);
if (const auto * type = checkAndGetDataType<DataTypeFloat64>(to_type))
return monotonicityForType(type);
if (const auto * type = checkAndGetDataType<DataTypeDate>(to_type))
return monotonicityForType(type);
if (const auto * type = checkAndGetDataType<DataTypeDateTime>(to_type))
return monotonicityForType(type);
if (const auto * type = checkAndGetDataType<DataTypeString>(to_type))
return monotonicityForType(type);
if (isEnum(from_type))
{
if (const auto * type = checkAndGetDataType<DataTypeEnum8>(to_type))
return monotonicityForType(type);
if (const auto * type = checkAndGetDataType<DataTypeEnum16>(to_type))
return monotonicityForType(type);
}
/// other types like Null, FixedString, Array and Tuple have no monotonicity defined
return {};
const auto & column = arguments.back().column;
if (!column)
throw Exception("Second argument to " + getName() + " must be a constant string describing type."
" Instead there is non-constant column of type " + arguments.back().type->getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto type_col = checkAndGetColumnConst<ColumnString>(column.get());
if (!type_col)
throw Exception("Second argument to " + getName() + " must be a constant string describing type."
" Instead there is a column with the following structure: " + column->dumpStructure(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
DataTypePtr type = DataTypeFactory::instance().get(type_col->getValue<String>());
return makeNullable(type);
}
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
};
}

View File

@ -5,6 +5,8 @@
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnNullable.h>
#include <IO/WriteHelpers.h>
@ -18,6 +20,11 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
}
enum class ConvertToFixedStringExceptionMode
{
Throw,
Null
};
/** Conversion to fixed string is implemented only for strings.
*/
@ -55,13 +62,22 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
{
const auto n = arguments[1].column->getUInt(0);
return executeForN(arguments, n);
return executeForN<ConvertToFixedStringExceptionMode::Throw>(arguments, n);
}
template<ConvertToFixedStringExceptionMode exception_mode>
static ColumnPtr executeForN(const ColumnsWithTypeAndName & arguments, const size_t n)
{
const auto & column = arguments[0].column;
ColumnUInt8::MutablePtr col_null_map_to;
ColumnUInt8::Container * vec_null_map_to [[maybe_unused]] = nullptr;
if constexpr (exception_mode == ConvertToFixedStringExceptionMode::Null)
{
col_null_map_to = ColumnUInt8::create(column->size(), false);
vec_null_map_to = &col_null_map_to->getData();
}
if (const auto * column_string = checkAndGetColumn<ColumnString>(column.get()))
{
auto column_fixed = ColumnFixedString::create(n);
@ -77,18 +93,42 @@ public:
const size_t off = i ? in_offsets[i - 1] : 0;
const size_t len = in_offsets[i] - off - 1;
if (len > n)
throw Exception("String too long for type FixedString(" + toString(n) + ")",
ErrorCodes::TOO_LARGE_STRING_SIZE);
{
if constexpr (exception_mode == ConvertToFixedStringExceptionMode::Throw)
{
throw Exception("String too long for type FixedString(" + toString(n) + ")",
ErrorCodes::TOO_LARGE_STRING_SIZE);
}
else
{
(*vec_null_map_to)[i] = true;
continue;
}
}
memcpy(&out_chars[i * n], &in_chars[off], len);
}
return column_fixed;
if constexpr (exception_mode == ConvertToFixedStringExceptionMode::Null)
return ColumnNullable::create(std::move(column_fixed), std::move(col_null_map_to));
else
return column_fixed;;
}
else if (const auto * column_fixed_string = checkAndGetColumn<ColumnFixedString>(column.get()))
{
const auto src_n = column_fixed_string->getN();
if (src_n > n)
throw Exception{"String too long for type FixedString(" + toString(n) + ")", ErrorCodes::TOO_LARGE_STRING_SIZE};
{
if constexpr (exception_mode == ConvertToFixedStringExceptionMode::Throw)
{
throw Exception{"String too long for type FixedString(" + toString(n) + ")", ErrorCodes::TOO_LARGE_STRING_SIZE};
}
else
{
auto column_fixed = ColumnFixedString::create(n);
std::fill(vec_null_map_to->begin(), vec_null_map_to->end(), true);
return ColumnNullable::create(column_fixed->cloneResized(column->size()), std::move(col_null_map_to));
}
}
auto column_fixed = ColumnFixedString::create(n);
@ -103,7 +143,16 @@ public:
return column_fixed;
}
else
throw Exception("Unexpected column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN);
{
if constexpr (exception_mode == ConvertToFixedStringExceptionMode::Throw)
throw Exception("Unexpected column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN);
else
{
auto column_fixed = ColumnFixedString::create(n);
std::fill(vec_null_map_to->begin(), vec_null_map_to->end(), true);
return ColumnNullable::create(column_fixed->cloneResized(column->size()), std::move(col_null_map_to));
}
}
}
};

View File

@ -0,0 +1,5 @@
255
\N
1
\N
\N

View File

@ -0,0 +1,7 @@
SELECT cast(-1, 'UInt8');
SELECT accurateCastOrNull(-1, 'UInt8');
SELECT cast(257, 'Int8');
SELECT accurateCastOrNull(257, 'Int8');
SELECT accurateCastOrNull('123', 'FixedString(2)');