Merge pull request #3008 from 4ertus2/decimal

even more Decimal
This commit is contained in:
alexey-milovidov 2018-09-03 07:19:33 +03:00 committed by GitHub
commit 72c799ad76
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 347 additions and 243 deletions

View File

@ -27,7 +27,7 @@ int ColumnDecimal<T>::compareAt(size_t n, size_t m, const IColumn & rhs_, int )
{
auto other = static_cast<const Self &>(rhs_);
const T & a = data[n];
const T & b = static_cast<const Self &>(rhs_).data[m];
const T & b = other.data[m];
return decimalLess<T>(b, a, other.scale, scale) ? 1 : (decimalLess<T>(a, b, scale, other.scale) ? -1 : 0);
}
@ -56,28 +56,36 @@ void ColumnDecimal<T>::updateHashWithValue(size_t n, SipHash & hash) const
template <typename T>
void ColumnDecimal<T>::getPermutation(bool reverse, size_t limit, int , IColumn::Permutation & res) const
{
size_t s = data.size();
res.resize(s);
for (size_t i = 0; i < s; ++i)
res[i] = i;
if (limit >= s)
limit = 0;
if (limit)
#if 1 /// TODO: perf test
if (data.size() <= std::numeric_limits<UInt32>::max())
{
if (reverse)
std::partial_sort(res.begin(), res.begin() + limit, res.end(), [](T a, T b) { return a > b; });
else
std::partial_sort(res.begin(), res.begin() + limit, res.end(), [](T a, T b) { return a < b; });
}
else
{
if (reverse)
std::sort(res.begin(), res.end(), [](T a, T b) { return a > b; });
else
std::sort(res.begin(), res.end(), [](T a, T b) { return a < b; });
PaddedPODArray<UInt32> tmp_res;
permutation(reverse, limit, tmp_res);
res.resize(tmp_res.size());
for (size_t i = 0; i < tmp_res.size(); ++i)
res[i] = tmp_res[i];
return;
}
#endif
permutation(reverse, limit, res);
}
template <typename T>
ColumnPtr ColumnDecimal<T>::permute(const IColumn::Permutation & perm, size_t limit) const
{
size_t size = limit ? std::min(data.size(), limit) : data.size();
if (perm.size() < size)
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
auto res = this->create(size, scale);
typename Self::Container & res_data = res->getData();
for (size_t i = 0; i < size; ++i)
res_data[i] = data[perm[i]];
return std::move(res);
}
template <typename T>
@ -94,7 +102,10 @@ MutableColumnPtr ColumnDecimal<T>::cloneResized(size_t size) const
memcpy(new_col.data.data(), data.data(), count * sizeof(data[0]));
if (size > count)
memset(static_cast<void *>(&new_col.data[count]), static_cast<int>(value_type()), (size - count) * sizeof(value_type));
{
void * tail = &new_col.data[count];
memset(tail, 0, (size - count) * sizeof(T));
}
}
return std::move(res);
@ -106,10 +117,8 @@ void ColumnDecimal<T>::insertRangeFrom(const IColumn & src, size_t start, size_t
const ColumnDecimal & src_vec = static_cast<const ColumnDecimal &>(src);
if (start + length > src_vec.data.size())
throw Exception("Parameters start = "
+ toString(start) + ", length = "
+ toString(length) + " are out of bound in ColumnVector<T>::insertRangeFrom method"
" (data.size() = " + toString(src_vec.data.size()) + ").",
throw Exception("Parameters start = " + toString(start) + ", length = " + toString(length) +
" are out of bound in ColumnDecimal<T>::insertRangeFrom method (data.size() = " + toString(src_vec.data.size()) + ").",
ErrorCodes::PARAMETER_OUT_OF_BOUND);
size_t old_size = data.size();
@ -146,27 +155,6 @@ ColumnPtr ColumnDecimal<T>::filter(const IColumn::Filter & filt, ssize_t result_
return std::move(res);
}
template <typename T>
ColumnPtr ColumnDecimal<T>::permute(const IColumn::Permutation & perm, size_t limit) const
{
size_t size = data.size();
if (limit == 0)
limit = size;
else
limit = std::min(size, limit);
if (perm.size() < limit)
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
auto res = this->create(limit, scale);
typename Self::Container & res_data = res->getData();
for (size_t i = 0; i < limit; ++i)
res_data[i] = data[perm[i]];
return std::move(res);
}
template <typename T>
ColumnPtr ColumnDecimal<T>::index(const IColumn & indexes, size_t limit) const
{

View File

@ -60,8 +60,7 @@ private:
friend class COWPtrHelper<IColumn, Self>;
public:
using value_type = T;
using Container = DecimalPaddedPODArray<value_type>;
using Container = DecimalPaddedPODArray<T>;
private:
ColumnDecimal(const size_t n, UInt32 scale_)
@ -138,6 +137,24 @@ public:
protected:
Container data;
UInt32 scale;
template <typename U>
void permutation(bool reverse, size_t limit, PaddedPODArray<U> & res) const
{
size_t s = data.size();
res.resize(s);
for (U i = 0; i < s; ++i)
res[i] = i;
std::function greater = [this](size_t a, size_t b) { return data[a] > data[b]; };
std::function less = [this](size_t a, size_t b) { return data[a] < data[b]; };
auto sort_end = res.end();
if (limit && limit < s)
sort_end = res.begin() + limit;
std::partial_sort(res.begin(), sort_end, res.end(), reverse ? greater : less);
}
};
template <typename T>

View File

@ -29,15 +29,7 @@ struct UInt128
UInt128() = default;
explicit UInt128(const UInt64 low, const UInt64 high) : low(low), high(high) {}
#if 1
explicit UInt128(const unsigned __int128 rhs)
: low(rhs & 0xffffffffffffffffll),
high(rhs >> 64)
{}
#else
explicit UInt128(const UInt64 rhs) : low(rhs), high() {}
#endif
auto tuple() const { return std::tie(high, low); }

View File

@ -32,7 +32,10 @@ using DB::UInt64;
// Case 1. Is pair of floats or pair of ints or pair of uints
template <typename A, typename B>
constexpr bool is_safe_conversion = (std::is_floating_point_v<A> && std::is_floating_point_v<B>)
|| (std::is_integral_v<A> && std::is_integral_v<B> && !(std::is_signed_v<A> ^ std::is_signed_v<B>));
|| (std::is_integral_v<A> && std::is_integral_v<B> && !(std::is_signed_v<A> ^ std::is_signed_v<B>))
|| (std::is_same_v<A, DB::Int128> && std::is_same_v<B, DB::Int128>)
|| (std::is_integral_v<A> && std::is_same_v<B, DB::Int128>)
|| (std::is_same_v<A, DB::Int128> && std::is_integral_v<B>);
template <typename A, typename B>
using bool_if_safe_conversion = std::enable_if_t<is_safe_conversion<A, B>, bool>;
template <typename A, typename B>

View File

@ -82,6 +82,8 @@ enum class TypeIndex
Decimal64,
Decimal128,
UUID,
Array,
Tuple,
};
template <typename T> struct TypeId;
@ -99,97 +101,62 @@ template <> struct TypeId<Float64> { static constexpr const TypeIndex value = T
/// Not a data type in database, defined just for convenience.
using Strings = std::vector<String>;
}
#if 1 /// __int128
namespace DB
{
using Int128 = __int128;
template <> constexpr bool IsNumber<Int128> = true;
template <> struct TypeName<Int128> { static const char * get() { return "Int128"; } };
template <> struct TypeId<Int128> { static constexpr const TypeIndex value = TypeIndex::Int128; };
}
namespace std
/// Own FieldType for Decimal.
/// It is only a "storage" for decimal. To perform operations, you also have to provide a scale (number of digits after point).
template <typename T>
struct Decimal
{
using NativeType = T;
template <> struct is_signed<__int128>
{
static constexpr bool value = true;
Decimal() = default;
Decimal(Decimal<T> &&) = default;
Decimal(const Decimal<T> &) = default;
Decimal(const T & value_)
: value(value_)
{}
template <typename U>
Decimal(const Decimal<U> & x)
: value(x)
{}
constexpr Decimal<T> & operator = (Decimal<T> &&) = default;
constexpr Decimal<T> & operator = (const Decimal<T> &) = default;
operator T () const { return value; }
const Decimal<T> & operator += (const T & x) { value += x; return *this; }
const Decimal<T> & operator -= (const T & x) { value -= x; return *this; }
const Decimal<T> & operator *= (const T & x) { value *= x; return *this; }
const Decimal<T> & operator /= (const T & x) { value /= x; return *this; }
const Decimal<T> & operator %= (const T & x) { value %= x; return *this; }
T value;
};
template <> struct is_unsigned<__int128>
{
static constexpr bool value = false;
};
using Decimal32 = Decimal<Int32>;
using Decimal64 = Decimal<Int64>;
using Decimal128 = Decimal<Int128>;
template <> struct is_integral<__int128>
{
static constexpr bool value = true;
};
template <> struct TypeName<Decimal32> { static const char * get() { return "Decimal32"; } };
template <> struct TypeName<Decimal64> { static const char * get() { return "Decimal64"; } };
template <> struct TypeName<Decimal128> { static const char * get() { return "Decimal128"; } };
template <> struct is_arithmetic<__int128>
{
static constexpr bool value = true;
};
template <> struct TypeId<Decimal32> { static constexpr const TypeIndex value = TypeIndex::Decimal32; };
template <> struct TypeId<Decimal64> { static constexpr const TypeIndex value = TypeIndex::Decimal64; };
template <> struct TypeId<Decimal128> { static constexpr const TypeIndex value = TypeIndex::Decimal128; };
template <typename T>
constexpr bool IsDecimalNumber = false;
template <> constexpr bool IsDecimalNumber<Decimal32> = true;
template <> constexpr bool IsDecimalNumber<Decimal64> = true;
template <> constexpr bool IsDecimalNumber<Decimal128> = true;
}
#endif
namespace DB
{
/// Own FieldType for Decimal.
/// It is only a "storage" for decimal. To perform operations, you also have to provide a scale (number of digits after point).
template <typename T>
struct Decimal
{
using NativeType = T;
Decimal() = default;
Decimal(Decimal<T> &&) = default;
Decimal(const Decimal<T> &) = default;
Decimal(const T & value_)
: value(value_)
{}
template <typename U>
Decimal(const Decimal<U> & x)
: value(x)
{}
constexpr Decimal<T> & operator = (Decimal<T> &&) = default;
constexpr Decimal<T> & operator = (const Decimal<T> &) = default;
operator T () const { return value; }
const Decimal<T> & operator += (const T & x) { value += x; return *this; }
const Decimal<T> & operator -= (const T & x) { value -= x; return *this; }
const Decimal<T> & operator *= (const T & x) { value *= x; return *this; }
const Decimal<T> & operator /= (const T & x) { value /= x; return *this; }
const Decimal<T> & operator %= (const T & x) { value %= x; return *this; }
T value;
};
using Decimal32 = Decimal<Int32>;
using Decimal64 = Decimal<Int64>;
using Decimal128 = Decimal<Int128>;
template <> struct TypeName<Decimal32> { static const char * get() { return "Decimal32"; } };
template <> struct TypeName<Decimal64> { static const char * get() { return "Decimal64"; } };
template <> struct TypeName<Decimal128> { static const char * get() { return "Decimal128"; } };
template <> struct TypeId<Decimal32> { static constexpr const TypeIndex value = TypeIndex::Decimal32; };
template <> struct TypeId<Decimal64> { static constexpr const TypeIndex value = TypeIndex::Decimal64; };
template <> struct TypeId<Decimal128> { static constexpr const TypeIndex value = TypeIndex::Decimal128; };
template <typename T>
constexpr bool IsDecimalNumber = false;
template <> constexpr bool IsDecimalNumber<Decimal32> = true;
template <> constexpr bool IsDecimalNumber<Decimal64> = true;
template <> constexpr bool IsDecimalNumber<Decimal128> = true;
}

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

@ -148,7 +148,7 @@ public:
bool canBeUsedInBooleanContext() const override { return true; }
bool isNumber() const override { return true; }
bool isInteger() const override { return false; }
bool canBeInsideNullable() const override { return true; }
bool canBeInsideNullable() const override { return false; }
/// Decimal specific

View File

@ -218,45 +218,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

@ -209,13 +209,12 @@ namespace detail
template <typename T>
std::enable_if_t<std::is_signed_v<T>, void> writeIntText(T x, WriteBuffer & buf)
std::enable_if_t<std::is_signed_v<T> || std::is_same_v<T, Int128>, void> writeIntText(T x, WriteBuffer & buf)
{
detail::writeSIntText(x, buf);
}
template <typename T>
std::enable_if_t<std::is_unsigned_v<T>, void> writeIntText(T x, WriteBuffer & buf)
{

View File

@ -30,7 +30,7 @@ SELECT f + f, f - f, f * f, f / f FROM test.decimal WHERE f > 0; -- { serverErro
SELECT g + g, g - g, g * g, g / g FROM test.decimal WHERE g > 0;
SELECT h + h, h - h, h * h, h / h FROM test.decimal WHERE h > 0; -- { serverError 407 }
SELECT h + h, h - h FROM test.decimal WHERE h > 0;
--SELECT i + i, i - i, i * i, i / i FROM test.decimal WHERE i > 0; -- { serverError 407 }
SELECT i + i, i - i, i * i, i / i FROM test.decimal WHERE i > 0; -- { serverError 407 }
SELECT i + i, i - i FROM test.decimal WHERE i > 0;
SELECT j + j, j - j, j * j, j / j FROM test.decimal WHERE j > 0;
@ -38,8 +38,7 @@ SELECT a + 21, a - 21, a - 84, a * 21, a * -21, a / 21, a / 84 FROM test.decimal
SELECT b + 21, b - 21, b - 84, b * 21, b * -21, b / 21, b / 84 FROM test.decimal WHERE b = 42;
SELECT c + 21, c - 21, c - 84, c * 21, c * -21, c / 21, c / 84 FROM test.decimal WHERE c = 42;
SELECT e + 21, e - 21, e - 84, e * 21, e * -21, e / 21, e / 84 FROM test.decimal WHERE e > 0; -- { serverError 407 }
SELECT 1 LIMIT 0;
--SELECT f + 21, f - 21, f - 84, f * 21, f * -21, f / 21, f / 84 FROM test.decimal WHERE f > 0; -- { serverError 407 }
SELECT f + 21, f - 21, f - 84, f * 21, f * -21, f / 21, f / 84 FROM test.decimal WHERE f > 0; -- { serverError 407 }
SELECT g + 21, g - 21, g - 84, g * 21, g * -21, g / 21, g / 84 FROM test.decimal WHERE g > 0;
SELECT h + 21, h - 21, h - 84, h * 21, h * -21, h / 21, h / 84 FROM test.decimal WHERE h > 0;
SELECT i + 21, i - 21, i - 84, i * 21, i * -21, i / 21, i / 84 FROM test.decimal WHERE i > 0;
@ -49,8 +48,7 @@ SELECT 21 + a, 21 - a, 84 - a, 21 * a, -21 * a, 21 / a, 84 / a FROM test.decimal
SELECT 21 + b, 21 - b, 84 - b, 21 * b, -21 * b, 21 / b, 84 / b FROM test.decimal WHERE b = 42;
SELECT 21 + c, 21 - c, 84 - c, 21 * c, -21 * c, 21 / c, 84 / c FROM test.decimal WHERE c = 42;
SELECT 21 + e, 21 - e, 84 - e, 21 * e, -21 * e, 21 / e, 84 / e FROM test.decimal WHERE e > 0; -- { serverError 407 }
SELECT 1 LIMIT 0;
--SELECT 21 + f, 21 - f, 84 - f, 21 * f, -21 * f, 21 / f, 84 / f FROM test.decimal WHERE f > 0; -- { serverError 407 }
SELECT 21 + f, 21 - f, 84 - f, 21 * f, -21 * f, 21 / f, 84 / f FROM test.decimal WHERE f > 0; -- { serverError 407 }
SELECT 21 + g, 21 - g, 84 - g, 21 * g, -21 * g, 21 / g, 84 / g FROM test.decimal WHERE g > 0;
SELECT 21 + h, 21 - h, 84 - h, 21 * h, -21 * h, 21 / h, 84 / h FROM test.decimal WHERE h > 0; -- { serverError 407 }
SELECT 21 + h, 21 - h, 84 - h, 21 * h, -21 * h FROM test.decimal WHERE h > 0;

View File

@ -123,3 +123,9 @@
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
123456789 123456789123456789
12345678901234567890123456789012345678
0.123456789 0.123456789123456789
0.12345678901234567890123456789012345678

View File

@ -1,6 +1,14 @@
SET allow_experimental_decimal_type = 1;
SET send_logs_level = 'none';
CREATE TABLE IF NOT EXISTS test.x (a Nullable(Decimal(9, 2))) ENGINE = Memory; -- { serverError 43 }
CREATE TABLE IF NOT EXISTS test.x (a Nullable(Decimal(18, 2))) ENGINE = Memory; -- { serverError 43 }
CREATE TABLE IF NOT EXISTS test.x (a Nullable(Decimal(38, 2))) ENGINE = Memory; -- { serverError 43 }
SELECT toNullable(toDecimal32(0, 0)); -- { serverError 43 }
SELECT toNullable(toDecimal64(0, 0)); -- { serverError 43 }
SELECT toNullable(toDecimal128(0, 0)); -- { serverError 43 }
SELECT toDecimal32('1.1', 1), toDecimal32('1.1', 2), toDecimal32('1.1', 8);
SELECT toDecimal32('1.1', 0); -- { serverError 69 }
SELECT toDecimal32(1.1, 0), toDecimal32(1.1, 1), toDecimal32(1.1, 2), toDecimal32(1.1, 8);
@ -83,7 +91,19 @@ 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;
SELECT CAST('123456789', 'Decimal(9,0)'), CAST('123456789123456789', 'Decimal(18,0)');
SELECT CAST('12345678901234567890123456789012345678', 'Decimal(38,0)');
SELECT CAST('123456789', 'Decimal(9,1)'); -- { serverError 69 }
SELECT CAST('123456789123456789', 'Decimal(18,1)'); -- { serverError 69 }
SELECT CAST('12345678901234567890123456789012345678', 'Decimal(38,1)'); -- { serverError 69 }
SELECT CAST('0.123456789', 'Decimal(9,9)'), CAST('0.123456789123456789', 'Decimal(18,18)');
SELECT CAST('0.12345678901234567890123456789012345678', 'Decimal(38,38)');
SELECT CAST('0.123456789', 'Decimal(9,8)'); -- { serverError 69 }
SELECT CAST('0.123456789123456789', 'Decimal(18,17)'); -- { serverError 69 }
SELECT CAST('0.12345678901234567890123456789012345678', 'Decimal(38,37)'); -- { serverError 69 }
DROP TABLE IF EXISTS test.decimal;

View File

@ -0,0 +1,36 @@
{"a":0.100,"b":-0.100000000,"c":0.100000000000000000}
{"a":0.200,"b":-0.200000000,"c":0.200000000000000000}
{"a":0.300,"b":-0.300000000,"c":0.300000000000000000}
{"a":0.420,"b":-0.420000000,"c":0.420000000000000000}
{"a":1.000,"b":-1.000000000,"c":1.000000000000000000}
{"a":1.100,"b":-1.100000000,"c":1.100000000000000000}
{"a":2.000,"b":-2.000000000,"c":2.000000000000000000}
{"a":2.200,"b":-2.200000000,"c":2.200000000000000000}
{"a":3.000,"b":-3.000000000,"c":3.000000000000000000}
{"a":3.300,"b":-3.300000000,"c":3.300000000000000000}
{"a":42.000,"b":-42.000000000,"c":42.000000000000000000}
{"a":42.420,"b":-42.420000000,"c":42.420000000000000000}
0.100,-0.100000000,0.100000000000000000
0.200,-0.200000000,0.200000000000000000
0.300,-0.300000000,0.300000000000000000
0.420,-0.420000000,0.420000000000000000
1.000,-1.000000000,1.000000000000000000
1.100,-1.100000000,1.100000000000000000
2.000,-2.000000000,2.000000000000000000
2.200,-2.200000000,2.200000000000000000
3.000,-3.000000000,3.000000000000000000
3.300,-3.300000000,3.300000000000000000
42.000,-42.000000000,42.000000000000000000
42.420,-42.420000000,42.420000000000000000
0.100 -0.100000000 0.100000000000000000
0.200 -0.200000000 0.200000000000000000
0.300 -0.300000000 0.300000000000000000
0.420 -0.420000000 0.420000000000000000
1.000 -1.000000000 1.000000000000000000
1.100 -1.100000000 1.100000000000000000
2.000 -2.000000000 2.000000000000000000
2.200 -2.200000000 2.200000000000000000
3.000 -3.000000000 3.000000000000000000
3.300 -3.300000000 3.300000000000000000
42.000 -42.000000000 42.000000000000000000
42.420 -42.420000000 42.420000000000000000

View File

@ -0,0 +1,31 @@
SET allow_experimental_decimal_type = 1;
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.decimal;
CREATE TABLE IF NOT EXISTS test.decimal
(
a DEC(9, 3),
b DEC(18, 9),
c DEC(38, 18)
) ENGINE = Memory;
INSERT INTO test.decimal (a, b, c) VALUES (42.0, -42.0, 42) (0.42, -0.42, .42) (42.42, -42.42, 42.42);
INSERT INTO test.decimal (a, b, c) FORMAT JSONEachRow {"a":1.1, "b":-1.1, "c":1.1} {"a":1.0, "b":-1.0, "c":1} {"a":0.1, "b":-0.1, "c":.1};
INSERT INTO test.decimal (a, b, c) FORMAT CSV 2.0, -2.0, 2
;
INSERT INTO test.decimal (a, b, c) FORMAT CSV 0.2, -0.2, .2
;
INSERT INTO test.decimal (a, b, c) FORMAT CSV 2.2 , -2.2 , 2.2
;
INSERT INTO test.decimal (a, b, c) FORMAT TabSeparated 3.3 -3.3 3.3
;
INSERT INTO test.decimal (a, b, c) FORMAT TabSeparated 3.0 -3.0 3
;
INSERT INTO test.decimal (a, b, c) FORMAT TabSeparated 0.3 -0.3 .3
;
SELECT * FROM test.decimal ORDER BY a FORMAT JSONEachRow;
SELECT * FROM test.decimal ORDER BY b DESC FORMAT CSV;
SELECT * FROM test.decimal ORDER BY c FORMAT TabSeparated;
DROP TABLE IF EXISTS test.decimal;

View File

@ -29,8 +29,10 @@ namespace common
template <>
inline bool addOverflow(__int128 x, __int128 y, __int128 & res)
{
static constexpr __int128 min_int128 = __int128(0x8000000000000000ll) << 64;
static constexpr __int128 max_int128 = (__int128(0x7fffffffffffffffll) << 64) + 0xffffffffffffffffll;
res = x + y;
return (res - y) != x;
return (y > 0 && x > max_int128 - y) || (y < 0 && x < min_int128 - y);
}
template <typename T>
@ -60,8 +62,10 @@ namespace common
template <>
inline bool subOverflow(__int128 x, __int128 y, __int128 & res)
{
static constexpr __int128 min_int128 = __int128(0x8000000000000000ll) << 64;
static constexpr __int128 max_int128 = (__int128(0x7fffffffffffffffll) << 64) + 0xffffffffffffffffll;
res = x - y;
return (res + y) != x;
return (y < 0 && x > max_int128 + y) || (y > 0 && x < min_int128 + y);
}
template <typename T>
@ -92,6 +96,11 @@ namespace common
inline bool mulOverflow(__int128 x, __int128 y, __int128 & res)
{
res = x * y;
return (res / y) != x;
if (!x || !y)
return false;
unsigned __int128 a = (x > 0) ? x : -x;
unsigned __int128 b = (y > 0) ? y : -y;
return (a * b) / b != a;
}
}