CAST for decimal [CLICKHOUSE-3905]

This commit is contained in:
chertus 2018-08-31 11:59:21 +03:00
parent 409562a397
commit 845db9b46e
7 changed files with 136 additions and 94 deletions

View File

@ -82,6 +82,8 @@ enum class TypeIndex
Decimal64,
Decimal128,
UUID,
Array,
Tuple,
};
template <typename T> struct TypeId;

View File

@ -18,6 +18,8 @@ public:
DataTypeArray(const DataTypePtr & nested_);
TypeIndex getTypeId() const override { return TypeIndex::Array; }
std::string getName() const override
{
return "Array(" + nested->getName() + ")";

View File

@ -28,6 +28,7 @@ public:
DataTypeTuple(const DataTypes & elems);
DataTypeTuple(const DataTypes & elems, const Strings & names);
TypeIndex getTypeId() const override { return TypeIndex::Tuple; }
std::string getName() const override;
const char * getFamilyName() const override { return "Tuple"; }

View File

@ -217,45 +217,6 @@ template <typename T> class DataTypeEnum;
template <typename T> class DataTypeNumber;
template <typename T> class DataTypeDecimal;
template <typename T, typename F>
bool callOnDataTypeAndIndex(TypeIndex number, F && f)
{
switch (number)
{
case TypeIndex::UInt8: return f(TypePair<T, DataTypeNumber<UInt8>>());
case TypeIndex::UInt16: return f(TypePair<T, DataTypeNumber<UInt16>>());
case TypeIndex::UInt32: return f(TypePair<T, DataTypeNumber<UInt32>>());
case TypeIndex::UInt64: return f(TypePair<T, DataTypeNumber<UInt64>>());
case TypeIndex::Int8: return f(TypePair<T, DataTypeNumber<Int8>>());
case TypeIndex::Int16: return f(TypePair<T, DataTypeNumber<Int16>>());
case TypeIndex::Int32: return f(TypePair<T, DataTypeNumber<Int32>>());
case TypeIndex::Int64: return f(TypePair<T, DataTypeNumber<Int64>>());
case TypeIndex::Float32: return f(TypePair<T, DataTypeNumber<Float32>>());
case TypeIndex::Float64: return f(TypePair<T, DataTypeNumber<Float64>>());
case TypeIndex::Decimal32: return f(TypePair<T, DataTypeDecimal<Decimal32>>());
case TypeIndex::Decimal64: return f(TypePair<T, DataTypeDecimal<Decimal64>>());
case TypeIndex::Decimal128: return f(TypePair<T, DataTypeDecimal<Decimal128>>());
case TypeIndex::Date: return f(TypePair<T, DataTypeDate>());
case TypeIndex::DateTime: return f(TypePair<T, DataTypeDateTime>());
case TypeIndex::String: return f(TypePair<T, DataTypeString>());
case TypeIndex::FixedString: return f(TypePair<T, DataTypeFixedString>());
case TypeIndex::Enum8: return f(TypePair<T, DataTypeEnum<Int8>>());
case TypeIndex::Enum16: return f(TypePair<T, DataTypeEnum<Int16>>());
case TypeIndex::UUID: return f(TypePair<T, DataTypeUUID>());
default:
break;
}
return false;
}
template <typename T, typename F>
bool callOnIndexAndDataType(TypeIndex number, F && f)

View File

@ -98,7 +98,9 @@ struct ConvertImpl
using FromFieldType = typename FromDataType::FieldType;
using ToFieldType = typename ToDataType::FieldType;
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
template <typename Additions = void *>
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/,
Additions additions [[maybe_unused]] = Additions())
{
const ColumnWithTypeAndName & named_from = block.getByPosition(arguments[0]);
@ -110,8 +112,7 @@ struct ConvertImpl
typename ColVecTo::MutablePtr col_to = nullptr;
if constexpr (IsDecimal<ToDataType>)
{
const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]);
UInt32 scale = extractToDecimalScale(scale_column);
UInt32 scale = additions;
col_to = ColVecTo::create(0, scale);
}
else
@ -454,7 +455,9 @@ struct ConvertThroughParsing
return false;
}
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
template <typename Additions = void *>
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count,
Additions additions [[maybe_unused]] = Additions())
{
using ColVecTo = std::conditional_t<IsDecimalNumber<ToFieldType>, ColumnDecimal<ToFieldType>, ColumnVector<ToFieldType>>;
@ -489,10 +492,9 @@ struct ConvertThroughParsing
if constexpr (IsDecimal<ToDataType>)
{
const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]);
UInt32 scale = extractToDecimalScale(scale_column);
col_to = ColVecTo::create(size, scale);
UInt32 scale = additions;
ToDataType check_bounds_in_ctor(ToDataType::maxPrecision(), scale);
col_to = ColVecTo::create(size, scale);
}
else
col_to = ColVecTo::create(size);
@ -878,7 +880,19 @@ private:
using LeftDataType = typename Types::LeftType;
using RightDataType = typename Types::RightType;
ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, arguments, result, input_rows_count);
if constexpr (IsDecimal<RightDataType>)
{
if (arguments.size() != 2)
throw Exception{"Function " + getName() + " expects 2 arguments for Decimal.",
ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION};
const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]);
UInt32 scale = extractToDecimalScale(scale_column);
ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, arguments, result, input_rows_count, scale);
}
else
ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, arguments, result, input_rows_count);
return true;
};
@ -911,6 +925,11 @@ class FunctionConvertFromString : public IFunction
{
public:
static constexpr auto name = Name::name;
static constexpr bool to_decimal =
std::is_same_v<ToDataType, DataTypeDecimal<Decimal32>> ||
std::is_same_v<ToDataType, DataTypeDecimal<Decimal64>> ||
std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>>;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionConvertFromString>(); }
String getName() const override
@ -951,8 +970,10 @@ public:
DataTypePtr res;
if (std::is_same_v<ToDataType, DataTypeDateTime>)
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime>)
res = std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0));
else if constexpr (to_decimal)
throw Exception(getName() + " is only implemented for types String and Decimal", ErrorCodes::NOT_IMPLEMENTED);
else
res = std::make_shared<ToDataType>();
@ -974,7 +995,8 @@ public:
block, arguments, result, input_rows_count);
else
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName()
+ ". Only String or FixedString argument is accepted for try-conversion function. For other arguments, use function without 'orZero' or 'orNull'.",
+ ". Only String or FixedString argument is accepted for try-conversion function."
+ " For other arguments, use function without 'orZero' or 'orNull'.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
};
@ -1447,6 +1469,28 @@ private:
};
}
template <typename FieldType>
WrapperType createDecimalWrapper(const DataTypePtr & from_type, const DataTypeDecimal<FieldType> * to_type) const
{
using ToDataType = DataTypeDecimal<FieldType>;
TypeIndex type_index = from_type->getTypeId();
UInt32 scale = to_type->getScale();
return [type_index, scale] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
{
callOnIndexAndDataType<ToDataType>(type_index, [&](const auto & types) -> bool
{
using Types = std::decay_t<decltype(types)>;
using LeftDataType = typename Types::LeftType;
using RightDataType = typename Types::RightType;
ConvertImpl<LeftDataType, RightDataType, NameCast>::execute(block, arguments, result, input_rows_count, scale);
return true;
});
};
}
WrapperType createArrayWrapper(const DataTypePtr & from_type_untyped, const DataTypeArray * to_type) const
{
/// Conversion from String through parsing.
@ -1474,7 +1518,8 @@ private:
/// both from_type and to_type should be nullptr now is array types had same dimensions
if ((from_type == nullptr) != (to_type == nullptr))
throw Exception{"CAST AS Array can only be performed between same-dimensional array types or from String", ErrorCodes::TYPE_MISMATCH};
throw Exception{"CAST AS Array can only be performed between same-dimensional array types or from String",
ErrorCodes::TYPE_MISMATCH};
/// Prepare nested type conversion
const auto nested_function = prepareUnpackDictionaries(from_nested_type, to_nested_type);
@ -1517,8 +1562,8 @@ private:
const auto from_type = checkAndGetDataType<DataTypeTuple>(from_type_untyped.get());
if (!from_type)
throw Exception{"CAST AS Tuple can only be performed between tuple types or from String.\nLeft type: " + from_type_untyped->getName() +
", right type: " + to_type->getName(), ErrorCodes::TYPE_MISMATCH};
throw Exception{"CAST AS Tuple can only be performed between tuple types or from String.\nLeft type: "
+ from_type_untyped->getName() + ", right type: " + to_type->getName(), ErrorCodes::TYPE_MISMATCH};
if (from_type->getElements().size() != to_type->getElements().size())
throw Exception{"CAST AS Tuple can only be performed between tuple types with the same number of elements or from String.\n"
@ -1827,54 +1872,82 @@ private:
return wrapper;
}
/// '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.
/// '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.
WrapperType prepareImpl(const DataTypePtr & from_type, const DataTypePtr & to_type, bool requested_result_is_nullable) const
{
if (from_type->equals(*to_type))
return createIdentityWrapper(from_type);
else if (checkDataType<DataTypeNothing>(from_type.get()))
return createNothingWrapper(to_type.get());
else if (const auto to_actual_type = checkAndGetDataType<DataTypeUInt8>(to_type.get()))
return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeUInt16>(to_type.get()))
return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeUInt32>(to_type.get()))
return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeUInt64>(to_type.get()))
return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeInt8>(to_type.get()))
return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeInt16>(to_type.get()))
return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeInt32>(to_type.get()))
return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeInt64>(to_type.get()))
return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeFloat32>(to_type.get()))
return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeFloat64>(to_type.get()))
return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeDate>(to_type.get()))
return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeDateTime>(to_type.get()))
return createWrapper(from_type, to_actual_type, requested_result_is_nullable);
else if (const auto to_actual_type = checkAndGetDataType<DataTypeString>(to_type.get()))
return createStringWrapper(from_type);
else if (const auto type_fixed_string = checkAndGetDataType<DataTypeFixedString>(to_type.get()))
return createFixedStringWrapper(from_type, type_fixed_string->getN());
else if (const auto type_array = checkAndGetDataType<DataTypeArray>(to_type.get()))
return createArrayWrapper(from_type, type_array);
else if (const auto type_tuple = checkAndGetDataType<DataTypeTuple>(to_type.get()))
return createTupleWrapper(from_type, type_tuple);
else if (const auto type_enum = checkAndGetDataType<DataTypeEnum8>(to_type.get()))
return createEnumWrapper(from_type, type_enum);
else if (const auto type_enum = checkAndGetDataType<DataTypeEnum16>(to_type.get()))
return createEnumWrapper(from_type, type_enum);
WrapperType ret;
auto make_default_wrapper = [&](const auto & types) -> bool
{
using Types = std::decay_t<decltype(types)>;
using ToDataType = typename Types::LeftType;
if constexpr (
std::is_same_v<ToDataType, DataTypeUInt8> ||
std::is_same_v<ToDataType, DataTypeUInt16> ||
std::is_same_v<ToDataType, DataTypeUInt32> ||
std::is_same_v<ToDataType, DataTypeUInt64> ||
std::is_same_v<ToDataType, DataTypeInt8> ||
std::is_same_v<ToDataType, DataTypeInt16> ||
std::is_same_v<ToDataType, DataTypeInt32> ||
std::is_same_v<ToDataType, DataTypeInt64> ||
std::is_same_v<ToDataType, DataTypeFloat32> ||
std::is_same_v<ToDataType, DataTypeFloat64> ||
std::is_same_v<ToDataType, DataTypeDate> ||
std::is_same_v<ToDataType, DataTypeDateTime>)
{
ret = createWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()), requested_result_is_nullable);
return true;
}
if constexpr (
std::is_same_v<ToDataType, DataTypeEnum8> ||
std::is_same_v<ToDataType, DataTypeEnum16>)
{
ret = createEnumWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()));
return true;
}
if constexpr (
std::is_same_v<ToDataType, DataTypeDecimal<Decimal32>> ||
std::is_same_v<ToDataType, DataTypeDecimal<Decimal64>> ||
std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>>)
{
ret = createDecimalWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()));
return true;
}
return false;
};
if (callOnIndexAndDataType<void>(to_type->getTypeId(), make_default_wrapper))
return ret;
switch (to_type->getTypeId())
{
case TypeIndex::String:
return createStringWrapper(from_type);
case TypeIndex::FixedString:
return createFixedStringWrapper(from_type, checkAndGetDataType<DataTypeFixedString>(to_type.get())->getN());
case TypeIndex::Array:
return createArrayWrapper(from_type, checkAndGetDataType<DataTypeArray>(to_type.get()));
case TypeIndex::Tuple:
return createTupleWrapper(from_type, checkAndGetDataType<DataTypeTuple>(to_type.get()));
default:
break;
}
/// It's possible to use ConvertImplGenericFromString to convert from String to AggregateFunction,
/// but it is disabled because deserializing aggregate functions state might be unsafe.
throw Exception{"Conversion from " + from_type->getName() + " to " + to_type->getName() + " is not supported", ErrorCodes::CANNOT_CONVERT_TYPE};
throw Exception{"Conversion from " + from_type->getName() + " to " + to_type->getName() + " is not supported",
ErrorCodes::CANNOT_CONVERT_TYPE};
}
};
@ -1910,7 +1983,8 @@ protected:
{
const auto type_col = checkAndGetColumnConst<ColumnString>(arguments.back().column.get());
if (!type_col)
throw Exception("Second argument to " + getName() + " must be a constant string describing type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception("Second argument to " + getName() + " must be a constant string describing type",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return DataTypeFactory::instance().get(type_col->getValue<String>());
}

View File

@ -123,3 +123,5 @@
9999 9999 9999
999999999 999999999 999999999
999999999 999999999 999999999
42.42 42.42 42.42 42.42
42.42 42.4200000 42.4200000000000000 42.420000000000000000000000000000000000

View File

@ -83,7 +83,7 @@ SELECT toUInt16(9999) as x, toDecimal32(x, 0), toDecimal64(x, 0);
SELECT toUInt32(999999999) as x, toDecimal32(x, 0), toDecimal64(x, 0);
SELECT toUInt64(999999999) as x, toDecimal32(x, 0), toDecimal64(x, 0);
--SELECT CAST('1.1', 'Decimal(9,0)'), CAST('1.1', 'Decimal(9,1)'), CAST('1.1', 'Decimal(9,2)');
SELECT CAST('42.4200', 'Decimal(9,2)') AS a, CAST(a, 'Decimal(9,2)'), CAST(a, 'Decimal(18, 2)'), CAST(a, 'Decimal(38, 2)');
SELECT CAST('42.42', 'Decimal(9,2)') AS a, CAST(a, 'Decimal(9,7)'), CAST(a, 'Decimal(18, 16)'), CAST(a, 'Decimal(38, 36)');
--SELECT * FROM test.decimal;
--DROP TABLE IF EXISTS test.decimal;
DROP TABLE IF EXISTS test.decimal;