mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
CAST for decimal [CLICKHOUSE-3905]
This commit is contained in:
parent
409562a397
commit
845db9b46e
@ -82,6 +82,8 @@ enum class TypeIndex
|
||||
Decimal64,
|
||||
Decimal128,
|
||||
UUID,
|
||||
Array,
|
||||
Tuple,
|
||||
};
|
||||
|
||||
template <typename T> struct TypeId;
|
||||
|
@ -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() + ")";
|
||||
|
@ -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"; }
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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>());
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user