mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
toDecimalN() functions CLICKHOUSE-3905
This commit is contained in:
parent
cdff502828
commit
1e97f9e549
@ -191,10 +191,4 @@ namespace DB
|
||||
template <> constexpr bool decTrait<Dec32>() { return true; }
|
||||
template <> constexpr bool decTrait<Dec64>() { return true; }
|
||||
template <> constexpr bool decTrait<Dec128>() { return true; }
|
||||
|
||||
template <typename T>
|
||||
inline constexpr bool decBaseTrait() { return false; }
|
||||
template <> constexpr bool decBaseTrait<Int32>() { return true; }
|
||||
template <> constexpr bool decBaseTrait<Int64>() { return true; }
|
||||
template <> constexpr bool decBaseTrait<Int128>() { return true; }
|
||||
}
|
||||
|
@ -39,9 +39,8 @@ bool DataTypeDecimal<T>::equals(const IDataType & rhs) const
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeDecimal<T>::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
void DataTypeDecimal<T>::writeText(T value, WriteBuffer & ostr) const
|
||||
{
|
||||
T value = static_cast<const ColumnType &>(column).getData()[row_num];
|
||||
if (value < T(0))
|
||||
{
|
||||
value *= T(-1);
|
||||
@ -59,14 +58,26 @@ void DataTypeDecimal<T>::serializeText(const IColumn & column, size_t row_num, W
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeDecimal<T>::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
{
|
||||
T value = static_cast<const ColumnType &>(column).getData()[row_num];
|
||||
writeText(value, ostr);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeDecimal<T>::readText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale)
|
||||
{
|
||||
UInt32 unread_scale = scale;
|
||||
readDecimalText(istr, x, precision, unread_scale);
|
||||
x *= getScaleMultiplier(unread_scale);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeDecimal<T>::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
T x;
|
||||
UInt32 unread_scale = scale;
|
||||
readDecimalText(istr, x, precision, unread_scale);
|
||||
x *= getScaleMultiplier(unread_scale);
|
||||
readText(x, istr);
|
||||
static_cast<ColumnType &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
@ -156,6 +167,21 @@ MutableColumnPtr DataTypeDecimal<T>::createColumn() const
|
||||
|
||||
//
|
||||
|
||||
DataTypePtr createDecimal(UInt64 precision_value, UInt64 scale_value)
|
||||
{
|
||||
if (precision_value < minDecimalPrecision() || precision_value > maxDecimalPrecision<Dec128>())
|
||||
throw Exception("Wrong precision", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
if (static_cast<UInt64>(scale_value) > precision_value)
|
||||
throw Exception("Negative scales and scales larger than presicion are not supported", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
if (precision_value <= maxDecimalPrecision<Dec32>())
|
||||
return std::make_shared<DataTypeDecimal<Dec32>>(precision_value, scale_value);
|
||||
else if (precision_value <= maxDecimalPrecision<Dec64>())
|
||||
return std::make_shared<DataTypeDecimal<Dec64>>(precision_value, scale_value);
|
||||
return std::make_shared<DataTypeDecimal<Dec128>>(precision_value, scale_value);
|
||||
}
|
||||
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments || arguments->children.size() != 2)
|
||||
@ -170,19 +196,9 @@ static DataTypePtr create(const ASTPtr & arguments)
|
||||
throw Exception("Decimal data type family must have a two numbers as its arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
UInt64 precision_value = precision->value.get<UInt64>();
|
||||
Int64 scale_value = scale->value.get<Int64>();
|
||||
UInt64 scale_value = scale->value.get<UInt64>();
|
||||
|
||||
if (precision_value < minDecimalPrecision() || precision_value > maxDecimalPrecision<Dec128>())
|
||||
throw Exception("Wrong precision", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
if (scale_value < 0 || static_cast<UInt64>(scale_value) > precision_value)
|
||||
throw Exception("Negative scales and scales larger than presicion are not supported", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
if (precision_value <= maxDecimalPrecision<Dec32>())
|
||||
return std::make_shared<DataTypeDecimal<Dec32>>(precision_value, scale_value);
|
||||
else if (precision_value <= maxDecimalPrecision<Dec64>())
|
||||
return std::make_shared<DataTypeDecimal<Dec64>>(precision_value, scale_value);
|
||||
return std::make_shared<DataTypeDecimal<Dec128>>(precision_value, scale_value);
|
||||
return createDecimal(precision_value, scale_value);
|
||||
}
|
||||
|
||||
|
||||
|
@ -13,6 +13,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
extern const int CANNOT_CONVERT_TYPE;
|
||||
}
|
||||
|
||||
///
|
||||
@ -69,6 +70,9 @@ template <> constexpr size_t maxDecimalPrecision<Dec64>() { return 18; }
|
||||
template <> constexpr size_t maxDecimalPrecision<Dec128>() { return 38; }
|
||||
|
||||
|
||||
DataTypePtr createDecimal(UInt64 precision, UInt64 scale);
|
||||
|
||||
|
||||
/// Implements Decimal(P, S), where P is precision, S is scale.
|
||||
/// Maximum precisions for underlying types are:
|
||||
/// Int32 9
|
||||
@ -88,13 +92,15 @@ public:
|
||||
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
static constexpr size_t maxPrecision() { return maxDecimalPrecision<T>(); }
|
||||
|
||||
DataTypeDecimal(UInt32 precision_, UInt32 scale_)
|
||||
: precision(precision_),
|
||||
scale(scale_)
|
||||
{
|
||||
if (unlikely(precision < 1 || precision > maxDecimalPrecision<T>()))
|
||||
if (unlikely(precision < 1 || precision > maxPrecision()))
|
||||
throw Exception("Precision is out of bounds", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
if (unlikely(scale < 0 || static_cast<UInt32>(scale) > maxDecimalPrecision<T>()))
|
||||
if (unlikely(scale < 0 || static_cast<UInt32>(scale) > maxPrecision()))
|
||||
throw Exception("Scale is out of bounds", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
}
|
||||
|
||||
@ -161,7 +167,7 @@ public:
|
||||
return x % getScaleMultiplier();
|
||||
}
|
||||
|
||||
T maxWholeValue() const { return getScaleMultiplier(maxDecimalPrecision<T>() - scale) - T(1); }
|
||||
T maxWholeValue() const { return getScaleMultiplier(maxPrecision() - scale) - T(1); }
|
||||
|
||||
bool canStoreWhole(T x) const
|
||||
{
|
||||
@ -191,6 +197,10 @@ public:
|
||||
|
||||
T parseFromString(const String & str) const;
|
||||
|
||||
void readText(T & x, ReadBuffer & istr) const { readText(x, istr, precision, scale); }
|
||||
void writeText(T value, WriteBuffer & ostr) const;
|
||||
|
||||
static void readText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale);
|
||||
static T getScaleMultiplier(UInt32 scale);
|
||||
|
||||
private:
|
||||
@ -253,6 +263,17 @@ inline bool isDecimal(const IDataType & data_type)
|
||||
return false;
|
||||
}
|
||||
|
||||
inline UInt32 getDecimalScale(const IDataType & data_type)
|
||||
{
|
||||
if (auto * decimal_type = checkDecimal<Dec32>(data_type))
|
||||
return decimal_type->getScale();
|
||||
if (auto * decimal_type = checkDecimal<Dec64>(data_type))
|
||||
return decimal_type->getScale();
|
||||
if (auto * decimal_type = checkDecimal<Dec128>(data_type))
|
||||
return decimal_type->getScale();
|
||||
return std::numeric_limits<UInt32>::max();
|
||||
}
|
||||
|
||||
///
|
||||
inline bool notDecimalButComparableToDecimal(const IDataType & data_type)
|
||||
{
|
||||
@ -269,4 +290,54 @@ inline bool comparableToDecimal(const IDataType & data_type)
|
||||
return isDecimal(data_type);
|
||||
}
|
||||
|
||||
template <typename DataType> constexpr bool IsDecimal = false;
|
||||
template <> constexpr bool IsDecimal<DataTypeDecimal<Dec32>> = true;
|
||||
template <> constexpr bool IsDecimal<DataTypeDecimal<Dec64>> = true;
|
||||
template <> constexpr bool IsDecimal<DataTypeDecimal<Dec128>> = true;
|
||||
|
||||
template <typename FromDataType, typename ToDataType>
|
||||
inline std::enable_if_t<IsDecimal<FromDataType> && IsDecimal<ToDataType>, typename ToDataType::FieldType>
|
||||
convertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to)
|
||||
{
|
||||
ToDataType type_to(ToDataType::maxPrecision(), scale_to);
|
||||
FromDataType type_from(FromDataType::maxPrecision(), scale_from);
|
||||
|
||||
if (scale_from > scale_to)
|
||||
{
|
||||
typename FromDataType::FieldType factor = type_from.scaleFactorFor(type_to, false);
|
||||
return value / factor;
|
||||
}
|
||||
else
|
||||
{
|
||||
typename ToDataType::FieldType factor = type_to.scaleFactorFor(type_from, false);
|
||||
return value * factor;
|
||||
}
|
||||
}
|
||||
|
||||
template <typename FromDataType, typename ToDataType>
|
||||
inline std::enable_if_t<IsDecimal<FromDataType> && !IsDecimal<ToDataType>, typename ToDataType::FieldType>
|
||||
convertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale [[maybe_unused]])
|
||||
{
|
||||
if (scale > FromDataType::maxPrecision())
|
||||
throw Exception("Wrong decimal scale", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if constexpr (!std::is_same_v<ToDataType, DataTypeNumber<typename ToDataType::FieldType>>)
|
||||
throw Exception("Illegal convertion from decimal", ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
else
|
||||
return static_cast<typename ToDataType::FieldType>(value) / FromDataType::getScaleMultiplier(scale);
|
||||
}
|
||||
|
||||
template <typename FromDataType, typename ToDataType>
|
||||
inline std::enable_if_t<!IsDecimal<FromDataType> && IsDecimal<ToDataType>, typename ToDataType::FieldType>
|
||||
convertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale [[maybe_unused]])
|
||||
{
|
||||
if (scale > ToDataType::maxPrecision())
|
||||
throw Exception("Wrong decimal scale", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if constexpr (!std::is_same_v<FromDataType, DataTypeNumber<typename FromDataType::FieldType>>)
|
||||
throw Exception("Illegal convertion to decimal", ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
else
|
||||
return value * ToDataType::getScaleMultiplier(scale);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -305,6 +305,7 @@ class DataTypeFixedString;
|
||||
class DataTypeUUID;
|
||||
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)
|
||||
@ -324,6 +325,10 @@ bool callOnDataTypeAndIndex(TypeIndex number, F && f)
|
||||
case TypeIndex::Float32: return f(TypePair<T, DataTypeNumber<Float32>>());
|
||||
case TypeIndex::Float64: return f(TypePair<T, DataTypeNumber<Float64>>());
|
||||
|
||||
case TypeIndex::Dec32: return f(TypePair<T, DataTypeDecimal<Dec32>>());
|
||||
case TypeIndex::Dec64: return f(TypePair<T, DataTypeDecimal<Dec64>>());
|
||||
case TypeIndex::Dec128: return f(TypePair<T, DataTypeDecimal<Dec128>>());
|
||||
|
||||
case TypeIndex::Date: return f(TypePair<T, DataTypeDate>());
|
||||
case TypeIndex::DateTime: return f(TypePair<T, DataTypeDateTime>());
|
||||
|
||||
@ -360,6 +365,10 @@ bool callOnIndexAndDataType(TypeIndex number, F && f)
|
||||
case TypeIndex::Float32: return f(TypePair<DataTypeNumber<Float32>, T>());
|
||||
case TypeIndex::Float64: return f(TypePair<DataTypeNumber<Float64>, T>());
|
||||
|
||||
case TypeIndex::Dec32: return f(TypePair<DataTypeDecimal<Dec32>, T>());
|
||||
case TypeIndex::Dec64: return f(TypePair<DataTypeDecimal<Dec64>, T>());
|
||||
case TypeIndex::Dec128: return f(TypePair<DataTypeDecimal<Dec128>, T>());
|
||||
|
||||
case TypeIndex::Date: return f(TypePair<DataTypeDate, T>());
|
||||
case TypeIndex::DateTime: return f(TypePair<DataTypeDateTime, T>());
|
||||
|
||||
|
@ -930,11 +930,6 @@ template <typename DataType> constexpr bool IsDateOrDateTime = false;
|
||||
template <> constexpr bool IsDateOrDateTime<DataTypeDate> = true;
|
||||
template <> constexpr bool IsDateOrDateTime<DataTypeDateTime> = true;
|
||||
|
||||
template <typename DataType> constexpr bool IsDecimal = false;
|
||||
template <> constexpr bool IsDecimal<DataTypeDecimal<Dec32>> = true;
|
||||
template <> constexpr bool IsDecimal<DataTypeDecimal<Dec64>> = true;
|
||||
template <> constexpr bool IsDecimal<DataTypeDecimal<Dec128>> = true;
|
||||
|
||||
template <typename T0, typename T1> constexpr bool UseLeftDecimal = false;
|
||||
template <> constexpr bool UseLeftDecimal<DataTypeDecimal<Dec128>, DataTypeDecimal<Dec32>> = true;
|
||||
template <> constexpr bool UseLeftDecimal<DataTypeDecimal<Dec128>, DataTypeDecimal<Dec64>> = true;
|
||||
|
@ -40,6 +40,10 @@ void registerFunctionsConversion(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionToFloat32>();
|
||||
factory.registerFunction<FunctionToFloat64>();
|
||||
|
||||
factory.registerFunction<FunctionToDecimal9>();
|
||||
factory.registerFunction<FunctionToDecimal18>();
|
||||
//factory.registerFunction<FunctionToDecimal38>(); /// TODO
|
||||
|
||||
factory.registerFunction<FunctionToDate>();
|
||||
factory.registerFunction<FunctionToDateTime>();
|
||||
factory.registerFunction<FunctionToUUID>();
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <IO/parseDateTimeBestEffort.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
@ -72,6 +73,21 @@ namespace ErrorCodes
|
||||
* toType - conversion in "natural way";
|
||||
*/
|
||||
|
||||
inline UInt32 extractToDecimalScale(const ColumnWithTypeAndName & named_column)
|
||||
{
|
||||
const auto * arg_type = named_column.type.get();
|
||||
bool ok = checkAndGetDataType<DataTypeUInt64>(arg_type)
|
||||
|| checkAndGetDataType<DataTypeUInt32>(arg_type)
|
||||
|| checkAndGetDataType<DataTypeUInt16>(arg_type)
|
||||
|| checkAndGetDataType<DataTypeUInt8>(arg_type);
|
||||
if (!ok)
|
||||
throw Exception("Illegal type of toDecimal() scale " + named_column.type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
Field field;
|
||||
named_column.column->get(0, field);
|
||||
return field.get<UInt32>();
|
||||
}
|
||||
|
||||
|
||||
/** 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)
|
||||
@ -84,10 +100,17 @@ struct ConvertImpl
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
||||
{
|
||||
if (const ColumnVector<FromFieldType> * col_from
|
||||
= checkAndGetColumn<ColumnVector<FromFieldType>>(block.getByPosition(arguments[0]).column.get()))
|
||||
const ColumnWithTypeAndName & named_from = block.getByPosition(arguments[0]);
|
||||
|
||||
if (const ColumnVector<FromFieldType> * col_from = checkAndGetColumn<ColumnVector<FromFieldType>>(named_from.column.get()))
|
||||
{
|
||||
auto col_to = ColumnVector<ToFieldType>::create();
|
||||
if constexpr (IsDecimal<ToDataType>)
|
||||
{
|
||||
const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]);
|
||||
UInt32 scale = extractToDecimalScale(scale_column);
|
||||
col_to->getData().setScale(scale);
|
||||
}
|
||||
|
||||
const typename ColumnVector<FromFieldType>::Container & vec_from = col_from->getData();
|
||||
typename ColumnVector<ToFieldType>::Container & vec_to = col_to->getData();
|
||||
@ -95,13 +118,21 @@ struct ConvertImpl
|
||||
vec_to.resize(size);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = static_cast<ToFieldType>(vec_from[i]);
|
||||
{
|
||||
if constexpr (IsDecimal<FromDataType> && IsDecimal<ToDataType>)
|
||||
vec_to[i] = convertDecimals<FromDataType, ToDataType>(vec_from[i], vec_from.getScale(), vec_to.getScale());
|
||||
else if constexpr (IsDecimal<FromDataType>)
|
||||
vec_to[i] = convertFromDecimal<FromDataType, ToDataType>(vec_from[i], vec_from.getScale());
|
||||
else if constexpr (IsDecimal<ToDataType>)
|
||||
vec_to[i] = convertToDecimal<FromDataType, ToDataType>(vec_from[i], vec_to.getScale());
|
||||
else
|
||||
vec_to[i] = static_cast<ToFieldType>(vec_from[i]);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
throw Exception("Illegal column " + named_from.column->getName() + " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
@ -200,6 +231,15 @@ struct FormatImpl<DataTypeEnum<FieldType>>
|
||||
}
|
||||
};
|
||||
|
||||
template <typename FieldType>
|
||||
struct FormatImpl<DataTypeDecimal<FieldType>>
|
||||
{
|
||||
static void execute(const FieldType x, WriteBuffer & wb, const DataTypeDecimal<FieldType> * type, const DateLUTImpl *)
|
||||
{
|
||||
type->writeText(x, wb);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// DataTypeEnum<T> to DataType<T> free conversion
|
||||
template <typename FieldType, typename Name>
|
||||
@ -441,6 +481,14 @@ struct ConvertThroughParsing
|
||||
auto col_to = ColumnVector<ToFieldType>::create(size);
|
||||
typename ColumnVector<ToFieldType>::Container & vec_to = col_to->getData();
|
||||
|
||||
if constexpr (IsDecimal<ToDataType>)
|
||||
{
|
||||
const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]);
|
||||
UInt32 scale = extractToDecimalScale(scale_column);
|
||||
vec_to.setScale(scale);
|
||||
ToDataType check_bounds_in_ctor(ToDataType::maxPrecision(), scale);
|
||||
}
|
||||
|
||||
ColumnUInt8::MutablePtr col_null_map_to;
|
||||
ColumnUInt8::Container * vec_null_map_to [[maybe_unused]] = nullptr;
|
||||
if constexpr (exception_mode == ConvertFromStringExceptionMode::Null)
|
||||
@ -473,7 +521,11 @@ struct ConvertThroughParsing
|
||||
|
||||
ReadBufferFromMemory read_buffer(&(*chars)[current_offset], string_size);
|
||||
|
||||
if constexpr (exception_mode == ConvertFromStringExceptionMode::Throw)
|
||||
if constexpr (IsDecimal<ToDataType>)
|
||||
{
|
||||
ToDataType::readText(vec_to[i], read_buffer, ToDataType::maxPrecision(), vec_to.getScale());
|
||||
}
|
||||
else if constexpr (exception_mode == ConvertFromStringExceptionMode::Throw)
|
||||
{
|
||||
if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort)
|
||||
{
|
||||
@ -530,7 +582,6 @@ template <typename ToDataType, typename Name>
|
||||
struct ConvertImpl<std::enable_if_t<!std::is_same_v<ToDataType, DataTypeFixedString>, DataTypeFixedString>, ToDataType, Name>
|
||||
: ConvertThroughParsing<DataTypeFixedString, ToDataType, Name, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::Normal> {};
|
||||
|
||||
|
||||
/// Generic conversion of any type from String. Used for complex types: Array and Tuple.
|
||||
struct ConvertImplGenericFromString
|
||||
{
|
||||
@ -647,6 +698,9 @@ struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
|
||||
struct NameToDate { static constexpr auto name = "toDate"; };
|
||||
struct NameToDateTime { static constexpr auto name = "toDateTime"; };
|
||||
struct NameToString { static constexpr auto name = "toString"; };
|
||||
struct NameToDecimal9 { static constexpr auto name = "toDecimal9"; };
|
||||
struct NameToDecimal18 { static constexpr auto name = "toDecimal18"; };
|
||||
struct NameToDecimal38 { static constexpr auto name = "toDecimal38"; };
|
||||
|
||||
|
||||
#define DEFINE_NAME_TO_INTERVAL(INTERVAL_KIND) \
|
||||
@ -674,6 +728,9 @@ public:
|
||||
using Monotonic = MonotonicityImpl;
|
||||
|
||||
static constexpr auto name = Name::name;
|
||||
static constexpr bool to_decimal =
|
||||
std::is_same_v<Name, NameToDecimal9> || std::is_same_v<Name, NameToDecimal18> || std::is_same_v<Name, NameToDecimal38>;
|
||||
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionConvert>(); }
|
||||
|
||||
String getName() const override
|
||||
@ -687,7 +744,13 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 1 && arguments.size() != 2)
|
||||
if (to_decimal && arguments.size() != 2)
|
||||
{
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
else if (arguments.size() != 1 && arguments.size() != 2)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1 or 2. Second argument (time zone) is optional only make sense for DateTime.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
@ -696,6 +759,19 @@ public:
|
||||
{
|
||||
return std::make_shared<DataTypeInterval>(DataTypeInterval::Kind(Name::kind));
|
||||
}
|
||||
else if constexpr (to_decimal)
|
||||
{
|
||||
UInt64 scale = extractToDecimalScale(arguments[1]);
|
||||
|
||||
if constexpr (std::is_same_v<Name, NameToDecimal9>)
|
||||
return createDecimal(9, scale);
|
||||
else if constexpr (std::is_same_v<Name, NameToDecimal18>)
|
||||
return createDecimal(18, scale);
|
||||
else if constexpr ( std::is_same_v<Name, NameToDecimal38>)
|
||||
return createDecimal(38, scale);
|
||||
|
||||
throw Exception("Someting wrong with toDecimalNN()", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
else
|
||||
{
|
||||
/** Optional second argument with time zone is supported:
|
||||
@ -709,11 +785,12 @@ public:
|
||||
throw Exception("Illegal type " + arguments[1].type->getName() + " of 2nd argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!(std::is_same_v<Name, NameToDateTime>
|
||||
static constexpr bool to_date_or_time = std::is_same_v<Name, NameToDateTime>
|
||||
|| std::is_same_v<Name, NameToDate>
|
||||
|| std::is_same_v<Name, NameToUnixTimestamp>
|
||||
|| (std::is_same_v<Name, NameToString>
|
||||
&& checkDataType<DataTypeDateTime>(arguments[0].type.get()))))
|
||||
|| std::is_same_v<Name, NameToUnixTimestamp>;
|
||||
|
||||
if (!(to_date_or_time
|
||||
|| (std::is_same_v<Name, NameToString> && checkDataType<DataTypeDateTime>(arguments[0].type.get()))))
|
||||
{
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
@ -1128,6 +1205,9 @@ using FunctionToDateTime = FunctionConvert<DataTypeDateTime, NameToDateTime, ToI
|
||||
using FunctionToUUID = FunctionConvert<DataTypeUUID, NameToUUID, ToIntMonotonicity<UInt128>>;
|
||||
using FunctionToString = FunctionConvert<DataTypeString, NameToString, ToStringMonotonicity>;
|
||||
using FunctionToUnixTimestamp = FunctionConvert<DataTypeUInt32, NameToUnixTimestamp, ToIntMonotonicity<UInt32>>;
|
||||
using FunctionToDecimal9 = FunctionConvert<DataTypeDecimal<Dec32>, NameToDecimal9, ToIntMonotonicity<Int32>>;
|
||||
using FunctionToDecimal18 = FunctionConvert<DataTypeDecimal<Dec64>, NameToDecimal18, ToIntMonotonicity<Int64>>;
|
||||
using FunctionToDecimal38 = FunctionConvert<DataTypeDecimal<Dec128>, NameToDecimal38, ToIntMonotonicity<Int128>>;
|
||||
|
||||
|
||||
template <typename DataType> struct FunctionTo;
|
||||
@ -1147,6 +1227,9 @@ template <> struct FunctionTo<DataTypeDateTime> { using Type = FunctionToDateTim
|
||||
template <> struct FunctionTo<DataTypeUUID> { using Type = FunctionToUUID; };
|
||||
template <> struct FunctionTo<DataTypeString> { using Type = FunctionToString; };
|
||||
template <> struct FunctionTo<DataTypeFixedString> { using Type = FunctionToFixedString; };
|
||||
template <> struct FunctionTo<DataTypeDecimal<Int32>> { using Type = FunctionToDecimal9; };
|
||||
template <> struct FunctionTo<DataTypeDecimal<Int64>> { using Type = FunctionToDecimal18; };
|
||||
template <> struct FunctionTo<DataTypeDecimal<Int128>> { using Type = FunctionToDecimal38; };
|
||||
|
||||
template <typename FieldType> struct FunctionTo<DataTypeEnum<FieldType>>
|
||||
: FunctionTo<DataTypeNumber<FieldType>>
|
||||
|
89
dbms/tests/queries/0_stateless/00700_decimal_casts.reference
Normal file
89
dbms/tests/queries/0_stateless/00700_decimal_casts.reference
Normal file
@ -0,0 +1,89 @@
|
||||
1.1 1.10 1.10000000
|
||||
1 1.1 1.10 1.10000000
|
||||
9999999 9999999 -9999999 9999999 -9999999
|
||||
999999.9 999999.9 -999999.9 999999.9 -999999.9
|
||||
99999.99 99999.99 -99999.99 99999.99 -99999.99
|
||||
9999.999 9999.999 -9999.999 9999.999 -9999.999
|
||||
999.9999 999.9999 -999.9999 999.9999 -999.9999
|
||||
99.99999 99.99999 -99.99999 99.99999 -99.99999
|
||||
9.999999 9.999999 -9.999999 9.999999 -9.999999
|
||||
0.9999999 0.9999999 -0.9999999 0.9999999 -0.9999999
|
||||
999999999 999999999 -999999999 999999999 -999999999
|
||||
99999999.9 99999999.9 -99999999.9 99999999.9 -99999999.9
|
||||
9999999.99 9999999.99 -9999999.99 9999999.99 -9999999.99
|
||||
999999.999 999999.999 -999999.999 999999.999 -999999.999
|
||||
99999.9999 99999.9999 -99999.9999 99999.9999 -99999.9999
|
||||
9999.99999 9999.99999 -9999.99999 9999.99999 -9999.99999
|
||||
999.999999 999.999999 -999.999999 999.999999 -999.999999
|
||||
99.9999999 99.9999999 -99.9999999 99.9999999 -99.9999999
|
||||
9.99999999 9.99999998 -9.99999998 9.99999998 -9.99999998
|
||||
0.999999999 0.999999999 -0.999999999 0.999999999 -0.999999999
|
||||
0.0000 0.00 0.00000000
|
||||
1.0000 0.11 0.11000000
|
||||
2.0000 0.22 0.22000000
|
||||
3.0000 0.33 0.33000000
|
||||
4.0000 0.44 0.44000000
|
||||
5.0000 0.55 0.55000000
|
||||
6.0000 0.66 0.66000000
|
||||
7.0000 0.77 0.77000000
|
||||
8.0000 0.88 0.88000000
|
||||
9.0000 1.00 1.00000000
|
||||
0.0000 0.00000000 0.00
|
||||
1.0000 0.11110000 0.11
|
||||
2.0000 0.22220000 0.22
|
||||
3.0000 0.33330000 0.33
|
||||
4.0000 0.44440000 0.44
|
||||
5.0000 0.55550000 0.55
|
||||
6.0000 0.66660000 0.66
|
||||
7.0000 0.77770000 0.77
|
||||
8.0000 0.88880000 0.88
|
||||
9.0000 1.00000000 1.00
|
||||
0.00000000 0.0000 0.00
|
||||
1.00000000 0.1111 0.11
|
||||
2.00000000 0.2222 0.22
|
||||
3.00000000 0.3333 0.33
|
||||
4.00000000 0.4444 0.44
|
||||
5.00000000 0.5555 0.55
|
||||
6.00000000 0.6666 0.66
|
||||
7.00000000 0.7777 0.77
|
||||
8.00000000 0.8888 0.88
|
||||
9.00000000 1.0000 1.00
|
||||
0.0000 0.00 0.00000000
|
||||
1.0000 0.11 0.11000000
|
||||
2.0000 0.22 0.22000000
|
||||
3.0000 0.33 0.33000000
|
||||
4.0000 0.44 0.44000000
|
||||
5.0000 0.55 0.55000000
|
||||
6.0000 0.66 0.66000000
|
||||
7.0000 0.77 0.77000000
|
||||
8.0000 0.88 0.88000000
|
||||
9.0000 1.00 1.00000000
|
||||
0.0000 0.00000000 0.00
|
||||
1.0000 0.11110000 0.11
|
||||
2.0000 0.22220000 0.22
|
||||
3.0000 0.33330000 0.33
|
||||
4.0000 0.44440000 0.44
|
||||
5.0000 0.55550000 0.55
|
||||
6.0000 0.66660000 0.66
|
||||
7.0000 0.77770000 0.77
|
||||
8.0000 0.88880000 0.88
|
||||
9.0000 1.00000000 1.00
|
||||
0.00000000 0.0000 0.00
|
||||
1.00000000 0.1111 0.11
|
||||
2.00000000 0.2222 0.22
|
||||
3.00000000 0.3333 0.33
|
||||
4.00000000 0.4444 0.44
|
||||
5.00000000 0.5555 0.55
|
||||
6.00000000 0.6666 0.66
|
||||
7.00000000 0.7777 0.77
|
||||
8.00000000 0.8888 0.88
|
||||
9.00000000 1.0000 1.00
|
||||
99 99 -99 99 -99
|
||||
9999 9999 -9999 9999 -9999
|
||||
999999999 999999999 -999999999 999999999 -999999999
|
||||
999999999 999999999 -999999999 999999999 -999999999
|
||||
999999999999999999 999999999999999999 -999999999999999999
|
||||
99 99 99
|
||||
9999 9999 9999
|
||||
999999999 999999999 999999999
|
||||
999999999 999999999 999999999
|
49
dbms/tests/queries/0_stateless/00700_decimal_casts.sql
Normal file
49
dbms/tests/queries/0_stateless/00700_decimal_casts.sql
Normal file
@ -0,0 +1,49 @@
|
||||
SET allow_experimental_decimal_type = 1;
|
||||
|
||||
SELECT toDecimal9('1.1', 1), toDecimal9('1.1', 2), toDecimal9('1.1', 8);
|
||||
SELECT toDecimal9('1.1', 0); -- { serverError 69 }
|
||||
SELECT toDecimal9(1.1, 0), toDecimal9(1.1, 1), toDecimal9(1.1, 2), toDecimal9(1.1, 8);
|
||||
|
||||
SELECT toFloat32(9999999) as x, toDecimal9(x, 0), toDecimal9(-x, 0), toDecimal18(x, 0), toDecimal18(-x, 0);
|
||||
SELECT toFloat32(999999.9) as x, toDecimal9(x, 1), toDecimal9(-x, 1), toDecimal18(x, 1), toDecimal18(-x, 1);
|
||||
SELECT toFloat32(99999.99) as x, toDecimal9(x, 2), toDecimal9(-x, 2), toDecimal18(x, 2), toDecimal18(-x, 2);
|
||||
SELECT toFloat32(9999.999) as x, toDecimal9(x, 3), toDecimal9(-x, 3), toDecimal18(x, 3), toDecimal18(-x, 3);
|
||||
SELECT toFloat32(999.9999) as x, toDecimal9(x, 4), toDecimal9(-x, 4), toDecimal18(x, 4), toDecimal18(-x, 4);
|
||||
SELECT toFloat32(99.99999) as x, toDecimal9(x, 5), toDecimal9(-x, 5), toDecimal18(x, 5), toDecimal18(-x, 5);
|
||||
SELECT toFloat32(9.999999) as x, toDecimal9(x, 6), toDecimal9(-x, 6), toDecimal18(x, 6), toDecimal18(-x, 6);
|
||||
SELECT toFloat32(0.9999999) as x, toDecimal9(x, 7), toDecimal9(-x, 7), toDecimal18(x, 7), toDecimal18(-x, 7);
|
||||
|
||||
SELECT toFloat64(999999999) as x, toDecimal9(x, 0), toDecimal9(-x, 0), toDecimal18(x, 0), toDecimal18(-x, 0);
|
||||
SELECT toFloat64(99999999.9) as x, toDecimal9(x, 1), toDecimal9(-x, 1), toDecimal18(x, 1), toDecimal18(-x, 1);
|
||||
SELECT toFloat64(9999999.99) as x, toDecimal9(x, 2), toDecimal9(-x, 2), toDecimal18(x, 2), toDecimal18(-x, 2);
|
||||
SELECT toFloat64(999999.999) as x, toDecimal9(x, 3), toDecimal9(-x, 3), toDecimal18(x, 3), toDecimal18(-x, 3);
|
||||
SELECT toFloat64(99999.9999) as x, toDecimal9(x, 4), toDecimal9(-x, 4), toDecimal18(x, 4), toDecimal18(-x, 4);
|
||||
SELECT toFloat64(9999.99999) as x, toDecimal9(x, 5), toDecimal9(-x, 5), toDecimal18(x, 5), toDecimal18(-x, 5);
|
||||
SELECT toFloat64(999.999999) as x, toDecimal9(x, 6), toDecimal9(-x, 6), toDecimal18(x, 6), toDecimal18(-x, 6);
|
||||
SELECT toFloat64(99.9999999) as x, toDecimal9(x, 7), toDecimal9(-x, 7), toDecimal18(x, 7), toDecimal18(-x, 7);
|
||||
SELECT toFloat64(9.99999999) as x, toDecimal9(x, 8), toDecimal9(-x, 8), toDecimal18(x, 8), toDecimal18(-x, 8);
|
||||
SELECT toFloat64(0.999999999) as x, toDecimal9(x, 9), toDecimal9(-x, 9), toDecimal18(x, 9), toDecimal18(-x, 9);
|
||||
|
||||
SELECT toDecimal9(number, 4) as n1, toDecimal9(n1 / 9, 2) as n2, toDecimal9(n2, 8) FROM system.numbers LIMIT 10;
|
||||
SELECT toDecimal9(number, 4) as n1, toDecimal9(n1 / 9, 8) as n2, toDecimal9(n2, 2) FROM system.numbers LIMIT 10;
|
||||
SELECT toDecimal9(number, 8) as n1, toDecimal9(n1 / 9, 4) as n2, toDecimal9(n2, 2) FROM system.numbers LIMIT 10;
|
||||
|
||||
SELECT toDecimal18(number, 4) as n1, toDecimal18(n1 / 9, 2) as n2, toDecimal18(n2, 8) FROM system.numbers LIMIT 10;
|
||||
SELECT toDecimal18(number, 4) as n1, toDecimal18(n1 / 9, 8) as n2, toDecimal18(n2, 2) FROM system.numbers LIMIT 10;
|
||||
SELECT toDecimal18(number, 8) as n1, toDecimal18(n1 / 9, 4) as n2, toDecimal18(n2, 2) FROM system.numbers LIMIT 10;
|
||||
|
||||
SELECT toInt8(99) as x, toDecimal9(x, 0), toDecimal9(-x, 0), toDecimal18(x, 0), toDecimal18(-x, 0);
|
||||
SELECT toInt16(9999) as x, toDecimal9(x, 0), toDecimal9(-x, 0), toDecimal18(x, 0), toDecimal18(-x, 0);
|
||||
SELECT toInt32(999999999) as x, toDecimal9(x, 0), toDecimal9(-x, 0), toDecimal18(x, 0), toDecimal18(-x, 0);
|
||||
SELECT toInt64(999999999) as x, toDecimal9(x, 0), toDecimal9(-x, 0), toDecimal18(x, 0), toDecimal18(-x, 0);
|
||||
SELECT toInt64(999999999999999999) as x, toDecimal18(x, 0), toDecimal18(-x, 0);
|
||||
|
||||
SELECT toUInt8(99) as x, toDecimal9(x, 0), toDecimal18(x, 0);
|
||||
SELECT toUInt16(9999) as x, toDecimal9(x, 0), toDecimal18(x, 0);
|
||||
SELECT toUInt32(999999999) as x, toDecimal9(x, 0), toDecimal18(x, 0);
|
||||
SELECT toUInt64(999999999) as x, toDecimal9(x, 0), toDecimal18(x, 0);
|
||||
|
||||
--SELECT CAST('1.1', 'Decimal(9,0)'), CAST('1.1', 'Decimal(9,1)'), CAST('1.1', 'Decimal(9,2)');
|
||||
|
||||
--SELECT * FROM test.decimal;
|
||||
--DROP TABLE IF EXISTS test.decimal;
|
Loading…
Reference in New Issue
Block a user