Merge pull request #2970 from 4ertus2/decimal

Some more Decimal
This commit is contained in:
alexey-milovidov 2018-08-27 19:59:26 +03:00 committed by GitHub
commit 95b2bcbc6b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
38 changed files with 1787 additions and 462 deletions

View File

@ -0,0 +1,237 @@
//#include <cstring>
#include <cmath>
#include <Common/Exception.h>
#include <Common/Arena.h>
#include <Common/SipHash.h>
#include <IO/WriteHelpers.h>
#include <Columns/ColumnsCommon.h>
#include <Columns/ColumnDecimal.h>
#include <DataStreams/ColumnGathererStream.h>
template <typename T> bool decimalLess(T x, T y, UInt32 x_scale, UInt32 y_scale);
namespace DB
{
namespace ErrorCodes
{
extern const int PARAMETER_OUT_OF_BOUND;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
}
template <typename T>
int ColumnDecimal<T>::compareAt(size_t n, size_t m, const IColumn & rhs_, int ) const
{
auto other = static_cast<const Self &>(rhs_);
const T & a = data[n];
const T & b = static_cast<const Self &>(rhs_).data[m];
return decimalLess<T>(b, a, other.scale, scale) ? 1 : (decimalLess<T>(a, b, scale, other.scale) ? -1 : 0);
}
template <typename T>
StringRef ColumnDecimal<T>::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
{
auto pos = arena.allocContinue(sizeof(T), begin);
memcpy(pos, &data[n], sizeof(T));
return StringRef(pos, sizeof(T));
}
template <typename T>
const char * ColumnDecimal<T>::deserializeAndInsertFromArena(const char * pos)
{
data.push_back(*reinterpret_cast<const T *>(pos));
return pos + sizeof(T);
}
template <typename T>
void ColumnDecimal<T>::updateHashWithValue(size_t n, SipHash & hash) const
{
hash.update(data[n]);
}
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 (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; });
}
}
template <typename T>
MutableColumnPtr ColumnDecimal<T>::cloneResized(size_t size) const
{
auto res = this->create(0, scale);
if (size > 0)
{
auto & new_col = static_cast<Self &>(*res);
new_col.data.resize(size);
size_t count = std::min(this->size(), size);
memcpy(&new_col.data[0], &data[0], 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));
}
return std::move(res);
}
template <typename T>
void ColumnDecimal<T>::insertRangeFrom(const IColumn & src, size_t start, size_t length)
{
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()) + ").",
ErrorCodes::PARAMETER_OUT_OF_BOUND);
size_t old_size = data.size();
data.resize(old_size + length);
memcpy(&data[old_size], &src_vec.data[start], length * sizeof(data[0]));
}
template <typename T>
ColumnPtr ColumnDecimal<T>::filter(const IColumn::Filter & filt, ssize_t result_size_hint) const
{
size_t size = data.size();
if (size != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
auto res = this->create(0, scale);
Container & res_data = res->getData();
if (result_size_hint)
res_data.reserve(result_size_hint > 0 ? result_size_hint : size);
const UInt8 * filt_pos = &filt[0];
const UInt8 * filt_end = filt_pos + size;
const T * data_pos = &data[0];
while (filt_pos < filt_end)
{
if (*filt_pos)
res_data.push_back(*data_pos);
++filt_pos;
++data_pos;
}
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
{
return selectIndexImpl(*this, indexes, limit);
}
template <typename T>
ColumnPtr ColumnDecimal<T>::replicate(const IColumn::Offsets & offsets) const
{
size_t size = data.size();
if (size != offsets.size())
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
auto res = this->create(0, scale);
if (0 == size)
return res;
typename Self::Container & res_data = res->getData();
res_data.reserve(offsets.back());
IColumn::Offset prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
size_t size_to_replicate = offsets[i] - prev_offset;
prev_offset = offsets[i];
for (size_t j = 0; j < size_to_replicate; ++j)
res_data.push_back(data[i]);
}
return std::move(res);
}
template <typename T>
void ColumnDecimal<T>::gather(ColumnGathererStream & gatherer)
{
gatherer.gather(*this);
}
template <typename T>
void ColumnDecimal<T>::getExtremes(Field & min, Field & max) const
{
if (data.size() == 0)
{
min = typename NearestFieldType<T>::Type(0, scale);
max = typename NearestFieldType<T>::Type(0, scale);
return;
}
T cur_min = data[0];
T cur_max = data[0];
for (const T & x : data)
{
if (x < cur_min)
cur_min = x;
else if (x > cur_max)
cur_max = x;
}
min = typename NearestFieldType<T>::Type(cur_min, scale);
max = typename NearestFieldType<T>::Type(cur_max, scale);
}
template class ColumnDecimal<Decimal32>;
template class ColumnDecimal<Decimal64>;
template class ColumnDecimal<Decimal128>;
}

View File

@ -0,0 +1,162 @@
#pragma once
#include <cmath>
#include <Columns/IColumn.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/// PaddedPODArray extended by Decimal scale
template <typename T>
class DecimalPaddedPODArray : public PaddedPODArray<T>
{
public:
using Base = PaddedPODArray<T>;
using Base::operator[];
DecimalPaddedPODArray(size_t size, UInt32 scale_)
: Base(size),
scale(scale_)
{}
DecimalPaddedPODArray(const DecimalPaddedPODArray & other)
: Base(other.begin(), other.end()),
scale(other.scale)
{}
DecimalPaddedPODArray(DecimalPaddedPODArray && other)
{
this->swap(other);
std::swap(scale, other.scale);
}
DecimalPaddedPODArray & operator=(DecimalPaddedPODArray && other)
{
this->swap(other);
std::swap(scale, other.scale);
return *this;
}
UInt32 getScale() const { return scale; }
private:
UInt32 scale;
};
/// A ColumnVector for Decimals
template <typename T>
class ColumnDecimal final : public COWPtrHelper<IColumn, ColumnDecimal<T>>
{
private:
using Self = ColumnDecimal<T>;
friend class COWPtrHelper<IColumn, Self>;
public:
using value_type = T;
using Container = DecimalPaddedPODArray<value_type>;
private:
ColumnDecimal(const size_t n, UInt32 scale_)
: data(n, scale_),
scale(scale_)
{}
ColumnDecimal(const ColumnDecimal & src)
: data(src.data),
scale(src.scale)
{}
public:
const char * getFamilyName() const override { return TypeName<T>::get(); }
bool isNumeric() const override { return false; }
bool canBeInsideNullable() const override { return false; }
bool isFixedAndContiguous() const override { return true; }
size_t sizeOfValueIfFixed() const override { return sizeof(T); }
size_t size() const override { return data.size(); }
size_t byteSize() const override { return data.size() * sizeof(data[0]); }
size_t allocatedBytes() const override { return data.allocated_bytes(); }
void reserve(size_t n) override { data.reserve(n); }
void insertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast<const Self &>(src).getData()[n]); }
void insertData(const char * pos, size_t /*length*/) override { data.push_back(*reinterpret_cast<const T *>(pos)); }
void insertDefault() override { data.push_back(T()); }
void insert(const Field & x) override { data.push_back(DB::get<typename NearestFieldType<T>::Type>(x)); }
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
void popBack(size_t n) override { data.resize_assume_reserved(data.size() - n); }
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
const char * deserializeAndInsertFromArena(const char * pos) override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override;
void getPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override;
MutableColumnPtr cloneResized(size_t size) const override;
Field operator[](size_t n) const override { return DecimalField(data[n], scale); }
StringRef getRawData() const override { return StringRef(reinterpret_cast<const char*>(data.data()), data.size()); }
StringRef getDataAt(size_t n) const override { return StringRef(reinterpret_cast<const char *>(&data[n]), sizeof(data[n])); }
void get(size_t n, Field & res) const override { res = (*this)[n]; }
bool getBool(size_t n) const override { return bool(data[n]); }
Int64 getInt(size_t n) const override { return Int64(data[n] * scale); }
ColumnPtr filter(const IColumn::Filter & filt, ssize_t result_size_hint) const override;
ColumnPtr permute(const IColumn::Permutation & perm, size_t limit) const override;
ColumnPtr index(const IColumn & indexes, size_t limit) const override;
template <typename Type>
ColumnPtr indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const;
ColumnPtr replicate(const IColumn::Offsets & offsets) const override;
void getExtremes(Field & min, Field & max) const override;
MutableColumns scatter(IColumn::ColumnIndex num_columns, const IColumn::Selector & selector) const override
{
return this->template scatterImpl<Self>(num_columns, selector);
}
void gather(ColumnGathererStream & gatherer_stream) override;
void insert(const T value) { data.push_back(value); }
Container & getData() { return data; }
const Container & getData() const { return data; }
const T & getElement(size_t n) const { return data[n]; }
T & getElement(size_t n) { return data[n]; }
protected:
Container data;
UInt32 scale;
};
template <typename T>
template <typename Type>
ColumnPtr ColumnDecimal<T>::indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const
{
size_t size = indexes.size();
if (limit == 0)
limit = size;
else
limit = std::min(size, limit);
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[indexes[i]];
return std::move(res);
}
}

View File

@ -330,8 +330,4 @@ template class ColumnVector<Int64>;
template class ColumnVector<Int128>;
template class ColumnVector<Float32>;
template class ColumnVector<Float64>;
template class ColumnVector<Decimal32>;
template class ColumnVector<Decimal64>;
template class ColumnVector<Decimal128>;
}

View File

@ -8,12 +8,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/** Stuff for comparing numbers.
* Integer values are compared as usual.
* Floating-point numbers are compared this way that NaNs always end up at the end
@ -123,40 +117,6 @@ template <> inline UInt64 unionCastToUInt64(Float32 x)
}
/// PaddedPODArray extended by Decimal scale
template <typename T>
class DecimalPaddedPODArray : public PaddedPODArray<T>
{
public:
using Base = PaddedPODArray<T>;
using Base::operator[];
using Base::Base;
DecimalPaddedPODArray(std::initializer_list<T> il)
: DecimalPaddedPODArray(std::begin(il), std::end(il))
{}
DecimalPaddedPODArray(DecimalPaddedPODArray && other)
{
this->swap(other);
std::swap(scale, other.scale);
}
DecimalPaddedPODArray & operator=(DecimalPaddedPODArray && other)
{
this->swap(other);
std::swap(scale, other.scale);
return *this;
}
void setScale(UInt32 s) { scale = s; }
UInt32 getScale() const { return scale; }
private:
UInt32 scale = DecimalField::wrongScale();
};
/** A template for columns that use a simple array to store.
*/
template <typename T>
@ -171,7 +131,7 @@ private:
public:
using value_type = T;
using Container = std::conditional_t<IsDecimalNumber<T>, DecimalPaddedPODArray<value_type>, PaddedPODArray<value_type>>;
using Container = PaddedPODArray<value_type>;
private:
ColumnVector() {}
@ -255,15 +215,7 @@ public:
Field operator[](size_t n) const override
{
if constexpr (IsDecimalNumber<T>)
{
UInt32 scale = data.getScale();
if (scale == DecimalField::wrongScale())
throw Exception("Extracting Decimal field with unknown scale. Scale is lost.", ErrorCodes::LOGICAL_ERROR);
return DecimalField(data[n], scale);
}
else
return typename NearestFieldType<T>::Type(data[n]);
return typename NearestFieldType<T>::Type(data[n]);
}
void get(size_t n, Field & res) const override

View File

@ -30,11 +30,22 @@ static inline String formatQuotedWithPrefix(T x, const char * prefix)
return wb.str();
}
template <typename T>
static inline void writeQuoted(const DecimalField<T> & x, WriteBuffer & buf)
{
writeChar('\'', buf);
writeText(x.getValue(), x.getScale(), buf);
writeChar('\'', buf);
}
String FieldVisitorDump::operator() (const Null &) const { return "NULL"; }
String FieldVisitorDump::operator() (const UInt64 & x) const { return formatQuotedWithPrefix(x, "UInt64_"); }
String FieldVisitorDump::operator() (const Int64 & x) const { return formatQuotedWithPrefix(x, "Int64_"); }
String FieldVisitorDump::operator() (const Float64 & x) const { return formatQuotedWithPrefix(x, "Float64_"); }
String FieldVisitorDump::operator() (const DecimalField<Decimal32> & x) const { return formatQuotedWithPrefix(x, "Decimal32_"); }
String FieldVisitorDump::operator() (const DecimalField<Decimal64> & x) const { return formatQuotedWithPrefix(x, "Decimal64_"); }
String FieldVisitorDump::operator() (const DecimalField<Decimal128> & x) const { return formatQuotedWithPrefix(x, "Decimal128_"); }
String FieldVisitorDump::operator() (const UInt128 & x) const
{
@ -112,6 +123,9 @@ String FieldVisitorToString::operator() (const UInt64 & x) const { return format
String FieldVisitorToString::operator() (const Int64 & x) const { return formatQuoted(x); }
String FieldVisitorToString::operator() (const Float64 & x) const { return formatFloat(x); }
String FieldVisitorToString::operator() (const String & x) const { return formatQuoted(x); }
String FieldVisitorToString::operator() (const DecimalField<Decimal32> & x) const { return formatQuoted(x); }
String FieldVisitorToString::operator() (const DecimalField<Decimal64> & x) const { return formatQuoted(x); }
String FieldVisitorToString::operator() (const DecimalField<Decimal128> & x) const { return formatQuoted(x); }
String FieldVisitorToString::operator() (const UInt128 & x) const
{
@ -207,4 +221,26 @@ void FieldVisitorHash::operator() (const Array & x) const
applyVisitor(*this, elem);
}
void FieldVisitorHash::operator() (const DecimalField<Decimal32> & x) const
{
UInt8 type = Field::Types::Decimal32;
hash.update(type);
hash.update(x);
}
void FieldVisitorHash::operator() (const DecimalField<Decimal64> & x) const
{
UInt8 type = Field::Types::Decimal64;
hash.update(type);
hash.update(x);
}
void FieldVisitorHash::operator() (const DecimalField<Decimal128> & x) const
{
UInt8 type = Field::Types::Decimal128;
hash.update(type);
hash.update(x);
}
}

View File

@ -44,6 +44,9 @@ typename std::decay_t<Visitor>::ResultType applyVisitor(Visitor && visitor, F &&
case Field::Types::String: return visitor(field.template get<String>());
case Field::Types::Array: return visitor(field.template get<Array>());
case Field::Types::Tuple: return visitor(field.template get<Tuple>());
case Field::Types::Decimal32: return visitor(field.template get<DecimalField<Decimal32>>());
case Field::Types::Decimal64: return visitor(field.template get<DecimalField<Decimal64>>());
case Field::Types::Decimal128: return visitor(field.template get<DecimalField<Decimal128>>());
default:
throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD);
@ -64,6 +67,9 @@ static typename std::decay_t<Visitor>::ResultType applyBinaryVisitorImpl(Visitor
case Field::Types::String: return visitor(field1, field2.template get<String>());
case Field::Types::Array: return visitor(field1, field2.template get<Array>());
case Field::Types::Tuple: return visitor(field1, field2.template get<Tuple>());
case Field::Types::Decimal32: return visitor(field1, field2.template get<DecimalField<Decimal32>>());
case Field::Types::Decimal64: return visitor(field1, field2.template get<DecimalField<Decimal64>>());
case Field::Types::Decimal128: return visitor(field1, field2.template get<DecimalField<Decimal128>>());
default:
throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD);
@ -99,6 +105,15 @@ typename std::decay_t<Visitor>::ResultType applyVisitor(Visitor && visitor, F1 &
case Field::Types::Tuple:
return applyBinaryVisitorImpl(
std::forward<Visitor>(visitor), field1.template get<Tuple>(), std::forward<F2>(field2));
case Field::Types::Decimal32:
return applyBinaryVisitorImpl(
std::forward<Visitor>(visitor), field1.template get<DecimalField<Decimal32>>(), std::forward<F2>(field2));
case Field::Types::Decimal64:
return applyBinaryVisitorImpl(
std::forward<Visitor>(visitor), field1.template get<DecimalField<Decimal64>>(), std::forward<F2>(field2));
case Field::Types::Decimal128:
return applyBinaryVisitorImpl(
std::forward<Visitor>(visitor), field1.template get<DecimalField<Decimal128>>(), std::forward<F2>(field2));
default:
throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD);
@ -118,6 +133,9 @@ public:
String operator() (const String & x) const;
String operator() (const Array & x) const;
String operator() (const Tuple & x) const;
String operator() (const DecimalField<Decimal32> & x) const;
String operator() (const DecimalField<Decimal64> & x) const;
String operator() (const DecimalField<Decimal128> & x) const;
};
@ -133,6 +151,9 @@ public:
String operator() (const String & x) const;
String operator() (const Array & x) const;
String operator() (const Tuple & x) const;
String operator() (const DecimalField<Decimal32> & x) const;
String operator() (const DecimalField<Decimal64> & x) const;
String operator() (const DecimalField<Decimal128> & x) const;
};
@ -169,6 +190,15 @@ public:
{
throw Exception("Cannot convert UInt128 to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
}
template <typename U>
T operator() (const DecimalField<U> & x) const
{
if constexpr (std::is_floating_point_v<T>)
return static_cast<T>(x.getValue()) / x.getScaleMultiplier();
else
return x.getValue() / x.getScaleMultiplier();
}
};
@ -187,9 +217,18 @@ public:
void operator() (const Float64 & x) const;
void operator() (const String & x) const;
void operator() (const Array & x) const;
void operator() (const DecimalField<Decimal32> & x) const;
void operator() (const DecimalField<Decimal64> & x) const;
void operator() (const DecimalField<Decimal128> & x) const;
};
template <typename T> constexpr bool isDecimalField() { return false; }
template <> constexpr bool isDecimalField<DecimalField<Decimal32>>() { return true; }
template <> constexpr bool isDecimalField<DecimalField<Decimal64>>() { return true; }
template <> constexpr bool isDecimalField<DecimalField<Decimal128>>() { return true; }
/** More precise comparison, used for index.
* Differs from Field::operator< and Field::operator== in that it also compares values of different types.
* Comparison rules are same as in FunctionsComparison (to be consistent with expression evaluation in query).
@ -199,15 +238,6 @@ public:
class FieldVisitorAccurateEquals : public StaticVisitor<bool>
{
public:
bool operator() (const Null &, const Null &) const { return true; }
bool operator() (const Null &, const UInt64 &) const { return false; }
bool operator() (const Null &, const UInt128 &) const { return false; }
bool operator() (const Null &, const Int64 &) const { return false; }
bool operator() (const Null &, const Float64 &) const { return false; }
bool operator() (const Null &, const String &) const { return false; }
bool operator() (const Null &, const Array &) const { return false; }
bool operator() (const Null &, const Tuple &) const { return false; }
bool operator() (const UInt64 &, const Null &) const { return false; }
bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; }
bool operator() (const UInt64 &, const UInt128) const { return true; }
@ -253,37 +283,49 @@ public:
bool operator() (const String &, const Array &) const { return false; }
bool operator() (const String &, const Tuple &) const { return false; }
bool operator() (const Array &, const Null &) const { return false; }
bool operator() (const Array &, const UInt64 &) const { return false; }
bool operator() (const Array &, const UInt128 &) const { return false; }
bool operator() (const Array &, const Int64 &) const { return false; }
bool operator() (const Array &, const Float64 &) const { return false; }
bool operator() (const Array &, const String &) const { return false; }
bool operator() (const Array & l, const Array & r) const { return l == r; }
bool operator() (const Array &, const Tuple &) const { return false; }
template <typename T>
bool operator() (const Null &, const T &) const
{
return std::is_same_v<T, Null>;
}
bool operator() (const Tuple &, const Null &) const { return false; }
bool operator() (const Tuple &, const UInt64 &) const { return false; }
bool operator() (const Tuple &, const UInt128 &) const { return false; }
bool operator() (const Tuple &, const Int64 &) const { return false; }
bool operator() (const Tuple &, const Float64 &) const { return false; }
bool operator() (const Tuple &, const String &) const { return false; }
bool operator() (const Tuple &, const Array &) const { return false; }
bool operator() (const Tuple & l, const Tuple & r) const { return l == r; }
template <typename T>
bool operator() (const Array & l, const T & r) const
{
if constexpr (std::is_same_v<T, Array>)
return l == r;
return false;
}
template <typename T>
bool operator() (const Tuple & l, const T & r) const
{
if constexpr (std::is_same_v<T, Tuple>)
return l == r;
return false;
}
template <typename T, typename U>
bool operator() (const DecimalField<T> & l, const U & r) const
{
if constexpr (isDecimalField<U>())
return l == r;
else if constexpr (std::is_same_v<U, Int64> || std::is_same_v<U, UInt64>)
return l == DecimalField<Decimal128>(r, 0);
return false;
}
template <typename T> bool operator() (const UInt64 & l, const DecimalField<T> & r) const { return DecimalField<Decimal128>(l, 0) == r; }
template <typename T> bool operator() (const UInt128 &, const DecimalField<T> &) const { return false; }
template <typename T> bool operator() (const Int64 & l, const DecimalField<T> & r) const { return DecimalField<Decimal128>(l, 0) == r; }
template <typename T> bool operator() (const Float64 &, const DecimalField<T> &) const { return false; }
template <typename T> bool operator() (const String &, const DecimalField<T> &) const { return false; }
};
class FieldVisitorAccurateLess : public StaticVisitor<bool>
{
public:
bool operator() (const Null &, const Null &) const { return false; }
bool operator() (const Null &, const UInt64 &) const { return true; }
bool operator() (const Null &, const Int64 &) const { return true; }
bool operator() (const Null &, const UInt128 &) const { return true; }
bool operator() (const Null &, const Float64 &) const { return true; }
bool operator() (const Null &, const String &) const { return true; }
bool operator() (const Null &, const Array &) const { return true; }
bool operator() (const Null &, const Tuple &) const { return true; }
bool operator() (const UInt64 &, const Null &) const { return false; }
bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; }
bool operator() (const UInt64 &, const UInt128 &) const { return true; }
@ -329,25 +371,46 @@ public:
bool operator() (const String &, const Array &) const { return true; }
bool operator() (const String &, const Tuple &) const { return true; }
bool operator() (const Array &, const Null &) const { return false; }
bool operator() (const Array &, const UInt64 &) const { return false; }
bool operator() (const Array &, const UInt128 &) const { return false; }
bool operator() (const Array &, const Int64 &) const { return false; }
bool operator() (const Array &, const Float64 &) const { return false; }
bool operator() (const Array &, const String &) const { return false; }
bool operator() (const Array & l, const Array & r) const { return l < r; }
bool operator() (const Array &, const Tuple &) const { return false; }
template <typename T>
bool operator() (const Null &, const T &) const
{
return !std::is_same_v<T, Null>;
}
bool operator() (const Tuple &, const Null &) const { return false; }
bool operator() (const Tuple &, const UInt64 &) const { return false; }
bool operator() (const Tuple &, const UInt128 &) const { return false; }
bool operator() (const Tuple &, const Int64 &) const { return false; }
bool operator() (const Tuple &, const Float64 &) const { return false; }
bool operator() (const Tuple &, const String &) const { return false; }
bool operator() (const Tuple &, const Array &) const { return false; }
bool operator() (const Tuple & l, const Tuple & r) const { return l < r; }
template <typename T>
bool operator() (const Array & l, const T & r) const
{
if constexpr (std::is_same_v<T, Array>)
return l < r;
return false;
}
template <typename T>
bool operator() (const Tuple & l, const T & r) const
{
if constexpr (std::is_same_v<T, Tuple>)
return l < r;
return false;
}
template <typename T, typename U>
bool operator() (const DecimalField<T> & l, const U & r) const
{
if constexpr (isDecimalField<U>())
return l < r;
else if constexpr (std::is_same_v<U, Int64> || std::is_same_v<U, UInt64>)
return l < DecimalField<Decimal128>(r, 0);
return false;
}
template <typename T> bool operator() (const UInt64 & l, const DecimalField<T> & r) const { return DecimalField<Decimal128>(l, 0) < r; }
template <typename T> bool operator() (const UInt128 &, const DecimalField<T> &) const { return false; }
template <typename T> bool operator() (const Int64 & l, const DecimalField<T> & r) const { return DecimalField<Decimal128>(l, 0) < r; }
template <typename T> bool operator() (const Float64 &, const DecimalField<T> &) const { return false; }
template <typename T> bool operator() (const String &, const DecimalField<T> &) const { return false; }
};
/** Implements `+=` operation.
* Returns false if the result is zero.
*/
@ -366,6 +429,13 @@ public:
bool operator() (String &) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); }
bool operator() (Array &) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); }
bool operator() (UInt128 &) const { throw Exception("Cannot sum UUIDs", ErrorCodes::LOGICAL_ERROR); }
template <typename T>
bool operator() (DecimalField<T> & x) const
{
x += get<DecimalField<T>>(rhs);
return x.getValue() != 0;
}
};
}

View File

@ -73,7 +73,7 @@ template <typename T> bool inline operator< (T a, const UInt128 b) { return UIn
template <> constexpr bool IsNumber<UInt128> = true;
template <> struct TypeName<UInt128> { static const char * get() { return "UInt128"; } };
template <> struct TypeId<UInt128> { static constexpr const size_t value = 5; };
template <> struct TypeId<UInt128> { static constexpr const TypeIndex value = TypeIndex::UInt128; };
struct UInt128Hash
{

View File

@ -273,21 +273,51 @@ namespace DB
}
bool DecimalField::operator < (const DecimalField & r) const
template <> Decimal32 DecimalField<Decimal32>::getScaleMultiplier() const
{
using Comparator = DecimalComparison<Decimal128, Decimal128, LessOp>;
return Comparator::compare(Decimal128(dec), Decimal128(r.dec), scale, r.scale);
return DataTypeDecimal<Decimal32>::getScaleMultiplier(scale);
}
bool DecimalField::operator <= (const DecimalField & r) const
template <> Decimal64 DecimalField<Decimal64>::getScaleMultiplier() const
{
using Comparator = DecimalComparison<Decimal128, Decimal128, LessOrEqualsOp>;
return Comparator::compare(Decimal128(dec), Decimal128(r.dec), scale, r.scale);
return DataTypeDecimal<Decimal64>::getScaleMultiplier(scale);
}
bool DecimalField::operator == (const DecimalField & r) const
template <> Decimal128 DecimalField<Decimal128>::getScaleMultiplier() const
{
using Comparator = DecimalComparison<Decimal128, Decimal128, EqualsOp>;
return Comparator::compare(Decimal128(dec), Decimal128(r.dec), scale, r.scale);
return DataTypeDecimal<Decimal128>::getScaleMultiplier(scale);
}
template <typename T>
static bool decEqual(T x, T y, UInt32 x_scale, UInt32 y_scale)
{
using Comparator = DecimalComparison<T, T, EqualsOp>;
return Comparator::compare(x, y, x_scale, y_scale);
}
template <typename T>
static bool decLess(T x, T y, UInt32 x_scale, UInt32 y_scale)
{
using Comparator = DecimalComparison<T, T, LessOp>;
return Comparator::compare(x, y, x_scale, y_scale);
}
template <typename T>
static bool decLessOrEqual(T x, T y, UInt32 x_scale, UInt32 y_scale)
{
using Comparator = DecimalComparison<T, T, LessOrEqualsOp>;
return Comparator::compare(x, y, x_scale, y_scale);
}
template <> bool decimalEqual(Decimal32 x, Decimal32 y, UInt32 xs, UInt32 ys) { return decEqual(x, y, xs, ys); }
template <> bool decimalLess(Decimal32 x, Decimal32 y, UInt32 xs, UInt32 ys) { return decLess(x, y, xs, ys); }
template <> bool decimalLessOrEqual(Decimal32 x, Decimal32 y, UInt32 xs, UInt32 ys) { return decLessOrEqual(x, y, xs, ys); }
template <> bool decimalEqual(Decimal64 x, Decimal64 y, UInt32 xs, UInt32 ys) { return decEqual(x, y, xs, ys); }
template <> bool decimalLess(Decimal64 x, Decimal64 y, UInt32 xs, UInt32 ys) { return decLess(x, y, xs, ys); }
template <> bool decimalLessOrEqual(Decimal64 x, Decimal64 y, UInt32 xs, UInt32 ys) { return decLessOrEqual(x, y, xs, ys); }
template <> bool decimalEqual(Decimal128 x, Decimal128 y, UInt32 xs, UInt32 ys) { return decEqual(x, y, xs, ys); }
template <> bool decimalLess(Decimal128 x, Decimal128 y, UInt32 xs, UInt32 ys) { return decLess(x, y, xs, ys); }
template <> bool decimalLessOrEqual(Decimal128 x, Decimal128 y, UInt32 xs, UInt32 ys) { return decLessOrEqual(x, y, xs, ys); }
}

View File

@ -20,6 +20,7 @@ namespace ErrorCodes
extern const int BAD_TYPE_OF_FIELD;
extern const int BAD_GET;
extern const int NOT_IMPLEMENTED;
extern const int LOGICAL_ERROR;
}
class Field;
@ -27,33 +28,67 @@ using Array = std::vector<Field>;
using TupleBackend = std::vector<Field>;
STRONG_TYPEDEF(TupleBackend, Tuple) /// Array and Tuple are different types with equal representation inside Field.
template <typename T> bool decimalEqual(T x, T y, UInt32 x_scale, UInt32 y_scale);
template <typename T> bool decimalLess(T x, T y, UInt32 x_scale, UInt32 y_scale);
template <typename T> bool decimalLessOrEqual(T x, T y, UInt32 x_scale, UInt32 y_scale);
template <typename T>
class DecimalField
{
public:
static constexpr UInt32 wrongScale() { return std::numeric_limits<UInt32>::max(); }
DecimalField(Int128 value, UInt32 scale_ = wrongScale())
DecimalField(T value, UInt32 scale_)
: dec(value),
scale(scale_)
{}
operator Decimal32() const { return dec; }
operator Decimal64() const { return dec; }
operator Decimal128() const { return dec; }
operator T() const { return dec; }
T getValue() const { return dec; }
T getScaleMultiplier() const;
UInt32 getScale() const { return scale; }
bool operator < (const DecimalField & r) const;
bool operator <= (const DecimalField & r) const;
bool operator == (const DecimalField & r) const;
template <typename U>
bool operator < (const DecimalField<U> & r) const
{
using MaxType = std::conditional_t<(sizeof(T) > sizeof(U)), T, U>;
return decimalLess<MaxType>(dec, r.getValue(), scale, r.getScale());
}
bool operator > (const DecimalField & r) const { return r < *this; }
bool operator >= (const DecimalField & r) const { return r <= * this; }
bool operator != (const DecimalField & r) const { return !(*this == r); }
template <typename U>
bool operator <= (const DecimalField<U> & r) const
{
using MaxType = std::conditional_t<(sizeof(T) > sizeof(U)), T, U>;
return decimalLessOrEqual<MaxType>(dec, r.getValue(), scale, r.getScale());
}
template <typename U>
bool operator == (const DecimalField<U> & r) const
{
using MaxType = std::conditional_t<(sizeof(T) > sizeof(U)), T, U>;
return decimalEqual<MaxType>(dec, r.getValue(), scale, r.getScale());
}
template <typename U> bool operator > (const DecimalField<U> & r) const { return r < *this; }
template <typename U> bool operator >= (const DecimalField<U> & r) const { return r <= * this; }
template <typename U> bool operator != (const DecimalField<U> & r) const { return !(*this == r); }
const DecimalField<T> & operator += (const DecimalField<T> & r)
{
if (scale != r.getScale())
throw Exception("Add different decimal fields", ErrorCodes::LOGICAL_ERROR);
dec += r.getValue();
return *this;
}
const DecimalField<T> & operator -= (const DecimalField<T> & r)
{
if (scale != r.getScale())
throw Exception("Sub different decimal fields", ErrorCodes::LOGICAL_ERROR);
dec -= r.getValue();
return *this;
}
private:
Int128 dec;
T dec;
UInt32 scale;
};
@ -91,7 +126,9 @@ public:
String = 16,
Array = 17,
Tuple = 18,
Decimal = 19,
Decimal32 = 19,
Decimal64 = 20,
Decimal128 = 21,
};
static const int MIN_NON_POD = 16;
@ -109,7 +146,9 @@ public:
case String: return "String";
case Array: return "Array";
case Tuple: return "Tuple";
case Decimal: return "Decimal";
case Decimal32: return "Decimal32";
case Decimal64: return "Decimal64";
case Decimal128: return "Decimal128";
}
throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD);
@ -121,6 +160,7 @@ public:
template <typename T> struct TypeToEnum;
template <Types::Which which> struct EnumToType;
static bool IsDecimal(Types::Which which) { return which >= Types::Decimal32 && which <= Types::Decimal128; }
Field()
: which(Types::Null)
@ -294,7 +334,9 @@ public:
case Types::String: return get<String>() < rhs.get<String>();
case Types::Array: return get<Array>() < rhs.get<Array>();
case Types::Tuple: return get<Tuple>() < rhs.get<Tuple>();
case Types::Decimal: return get<DecimalField>() < rhs.get<DecimalField>();
case Types::Decimal32: return get<DecimalField<Decimal32>>() < rhs.get<DecimalField<Decimal32>>();
case Types::Decimal64: return get<DecimalField<Decimal64>>() < rhs.get<DecimalField<Decimal64>>();
case Types::Decimal128: return get<DecimalField<Decimal128>>() < rhs.get<DecimalField<Decimal128>>();
}
throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD);
@ -323,7 +365,9 @@ public:
case Types::String: return get<String>() <= rhs.get<String>();
case Types::Array: return get<Array>() <= rhs.get<Array>();
case Types::Tuple: return get<Tuple>() <= rhs.get<Tuple>();
case Types::Decimal: return get<DecimalField>() <= rhs.get<DecimalField>();
case Types::Decimal32: return get<DecimalField<Decimal32>>() <= rhs.get<DecimalField<Decimal32>>();
case Types::Decimal64: return get<DecimalField<Decimal64>>() <= rhs.get<DecimalField<Decimal64>>();
case Types::Decimal128: return get<DecimalField<Decimal128>>() <= rhs.get<DecimalField<Decimal128>>();
}
throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD);
@ -350,7 +394,9 @@ public:
case Types::Tuple: return get<Tuple>() == rhs.get<Tuple>();
case Types::UInt128: return get<UInt128>() == rhs.get<UInt128>();
case Types::Int128: return get<Int128>() == rhs.get<Int128>();
case Types::Decimal: return get<DecimalField>() == rhs.get<DecimalField>();
case Types::Decimal32: return get<DecimalField<Decimal32>>() == rhs.get<DecimalField<Decimal32>>();
case Types::Decimal64: return get<DecimalField<Decimal64>>() == rhs.get<DecimalField<Decimal64>>();
case Types::Decimal128: return get<DecimalField<Decimal128>>() == rhs.get<DecimalField<Decimal128>>();
}
throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD);
@ -363,7 +409,8 @@ public:
private:
std::aligned_union_t<DBMS_MIN_FIELD_SIZE - sizeof(Types::Which),
Null, UInt64, UInt128, Int64, Int128, Float64, String, Array, Tuple, DecimalField
Null, UInt64, UInt128, Int64, Int128, Float64, String, Array, Tuple,
DecimalField<Decimal32>, DecimalField<Decimal64>, DecimalField<Decimal128>
> storage;
Types::Which which;
@ -412,7 +459,9 @@ private:
case Types::String: f(field.template get<String>()); return;
case Types::Array: f(field.template get<Array>()); return;
case Types::Tuple: f(field.template get<Tuple>()); return;
case Types::Decimal: f(field.template get<DecimalField>()); return;
case Types::Decimal32: f(field.template get<DecimalField<Decimal32>>()); return;
case Types::Decimal64: f(field.template get<DecimalField<Decimal64>>()); return;
case Types::Decimal128: f(field.template get<DecimalField<Decimal128>>()); return;
default:
throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD);
@ -496,7 +545,9 @@ template <> struct Field::TypeToEnum<Float64> { static const Types::Which value
template <> struct Field::TypeToEnum<String> { static const Types::Which value = Types::String; };
template <> struct Field::TypeToEnum<Array> { static const Types::Which value = Types::Array; };
template <> struct Field::TypeToEnum<Tuple> { static const Types::Which value = Types::Tuple; };
template <> struct Field::TypeToEnum<DecimalField>{ static const Types::Which value = Types::Decimal; };
template <> struct Field::TypeToEnum<DecimalField<Decimal32>>{ static const Types::Which value = Types::Decimal32; };
template <> struct Field::TypeToEnum<DecimalField<Decimal64>>{ static const Types::Which value = Types::Decimal64; };
template <> struct Field::TypeToEnum<DecimalField<Decimal128>>{ static const Types::Which value = Types::Decimal128; };
template <> struct Field::EnumToType<Field::Types::Null> { using Type = Null; };
template <> struct Field::EnumToType<Field::Types::UInt64> { using Type = UInt64; };
@ -507,7 +558,9 @@ template <> struct Field::EnumToType<Field::Types::Float64> { using Type = Float
template <> struct Field::EnumToType<Field::Types::String> { using Type = String; };
template <> struct Field::EnumToType<Field::Types::Array> { using Type = Array; };
template <> struct Field::EnumToType<Field::Types::Tuple> { using Type = Tuple; };
template <> struct Field::EnumToType<Field::Types::Decimal> { using Type = DecimalField; };
template <> struct Field::EnumToType<Field::Types::Decimal32> { using Type = DecimalField<Decimal32>; };
template <> struct Field::EnumToType<Field::Types::Decimal64> { using Type = DecimalField<Decimal64>; };
template <> struct Field::EnumToType<Field::Types::Decimal128> { using Type = DecimalField<Decimal128>; };
template <typename T>
@ -551,9 +604,9 @@ template <> struct NearestFieldType<Int16> { using Type = Int64; };
template <> struct NearestFieldType<Int32> { using Type = Int64; };
template <> struct NearestFieldType<Int64> { using Type = Int64; };
template <> struct NearestFieldType<Int128> { using Type = Int128; };
template <> struct NearestFieldType<Decimal32> { using Type = DecimalField; };
template <> struct NearestFieldType<Decimal64> { using Type = DecimalField; };
template <> struct NearestFieldType<Decimal128> { using Type = DecimalField; };
template <> struct NearestFieldType<Decimal32> { using Type = DecimalField<Decimal32>; };
template <> struct NearestFieldType<Decimal64> { using Type = DecimalField<Decimal64>; };
template <> struct NearestFieldType<Decimal128> { using Type = DecimalField<Decimal128>; };
template <> struct NearestFieldType<Float32> { using Type = Float64; };
template <> struct NearestFieldType<Float64> { using Type = Float64; };
template <> struct NearestFieldType<String> { using Type = String; };

View File

@ -57,21 +57,44 @@ template <> struct TypeName<Float32> { static const char * get() { return "Float
template <> struct TypeName<Float64> { static const char * get() { return "Float64"; } };
template <> struct TypeName<String> { static const char * get() { return "String"; } };
template <typename T> struct TypeId;
enum class TypeIndex
{
None = 0,
UInt8,
UInt16,
UInt32,
UInt64,
UInt128,
Int8,
Int16,
Int32,
Int64,
Int128,
Float32,
Float64,
Date,
DateTime,
String,
FixedString,
Enum8,
Enum16,
Decimal32,
Decimal64,
Decimal128,
UUID,
};
/// 0 reserved for types without number
template <> struct TypeId<UInt8> { static constexpr const size_t value = 1; };
template <> struct TypeId<UInt16> { static constexpr const size_t value = 2; };
template <> struct TypeId<UInt32> { static constexpr const size_t value = 3; };
template <> struct TypeId<UInt64> { static constexpr const size_t value = 4; };
/// 5 reserved for TypeId<UInt128>
template <> struct TypeId<Float32> { static constexpr const size_t value = 7; };
template <> struct TypeId<Float64> { static constexpr const size_t value = 8; };
template <> struct TypeId<Int8> { static constexpr const size_t value = 9; };
template <> struct TypeId<Int16> { static constexpr const size_t value = 10; };
template <> struct TypeId<Int32> { static constexpr const size_t value = 11; };
template <> struct TypeId<Int64> { static constexpr const size_t value = 12; };
/// 13 reserved for TypeId<Int128>
template <typename T> struct TypeId;
template <> struct TypeId<UInt8> { static constexpr const TypeIndex value = TypeIndex::UInt8; };
template <> struct TypeId<UInt16> { static constexpr const TypeIndex value = TypeIndex::UInt16; };
template <> struct TypeId<UInt32> { static constexpr const TypeIndex value = TypeIndex::UInt32; };
template <> struct TypeId<UInt64> { static constexpr const TypeIndex value = TypeIndex::UInt64; };
template <> struct TypeId<Int8> { static constexpr const TypeIndex value = TypeIndex::Int8; };
template <> struct TypeId<Int16> { static constexpr const TypeIndex value = TypeIndex::Int16; };
template <> struct TypeId<Int32> { static constexpr const TypeIndex value = TypeIndex::Int32; };
template <> struct TypeId<Int64> { static constexpr const TypeIndex value = TypeIndex::Int64; };
template <> struct TypeId<Float32> { static constexpr const TypeIndex value = TypeIndex::Float32; };
template <> struct TypeId<Float64> { static constexpr const TypeIndex value = TypeIndex::Float64; };
/// Not a data type in database, defined just for convenience.
using Strings = std::vector<String>;
@ -85,7 +108,7 @@ 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 size_t value = 13; };
template <> struct TypeId<Int128> { static constexpr const TypeIndex value = TypeIndex::Int128; };
}
@ -160,9 +183,9 @@ namespace DB
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 size_t value = 16; };
template <> struct TypeId<Decimal64> { static constexpr const size_t value = 17; };
template <> struct TypeId<Decimal128> { static constexpr const size_t value = 18; };
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;

View File

@ -9,6 +9,7 @@ namespace DB
class DataTypeDate final : public DataTypeNumberBase<UInt16>
{
public:
TypeIndex getTypeId() const override { return TypeIndex::Date; }
const char * getFamilyName() const override { return "Date"; }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;

View File

@ -35,6 +35,7 @@ public:
const char * getFamilyName() const override { return "DateTime"; }
std::string getName() const override;
TypeIndex getTypeId() const override { return TypeIndex::DateTime; }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;

View File

@ -65,6 +65,8 @@ public:
std::string getName() const override { return type_name; }
const char * getFamilyName() const override;
TypeIndex getTypeId() const override { return sizeof(FieldType) == 1 ? TypeIndex::Enum8 : TypeIndex::Enum16; }
const StringRef & getNameForValue(const FieldType & value) const
{
const auto it = value_to_name_map.find(value);

View File

@ -31,6 +31,7 @@ public:
}
std::string getName() const override;
TypeIndex getTypeId() const override { return TypeIndex::FixedString; }
const char * getFamilyName() const override { return "FixedString"; }

View File

@ -16,7 +16,7 @@ public:
using FieldType = T;
const char * getFamilyName() const override { return TypeName<T>::get(); }
size_t getTypeId() const override { return TypeId<T>::value; }
TypeIndex getTypeId() const override { return TypeId<T>::value; }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;

View File

@ -19,6 +19,8 @@ public:
return "String";
}
TypeIndex getTypeId() const override { return TypeIndex::String; }
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;

View File

@ -14,6 +14,7 @@ class DataTypeUUID final : public DataTypeNumberBase<UInt128>
public:
const char * getFamilyName() const override { return "UUID"; }
TypeIndex getTypeId() const override { return TypeIndex::UUID; }
bool equals(const IDataType & rhs) const override;

View File

@ -1,5 +1,4 @@
#include <type_traits>
#include <common/intExp.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeFactory.h>
@ -8,6 +7,7 @@
#include <IO/readFloatText.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h>
#include <Interpreters/Context.h>
namespace DB
{
@ -20,6 +20,10 @@ namespace ErrorCodes
}
bool decimalCheckComparisonOverflow(const Context & context) { return context.getSettingsRef().decimal_check_comparison_overflow; }
bool decimalCheckArithmeticOverflow(const Context & context) { return context.getSettingsRef().decimal_check_arithmetic_overflow; }
//
template <typename T>
@ -42,31 +46,22 @@ 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];
if (value < T(0))
{
value *= T(-1);
writeChar('-', ostr); /// avoid crop leading minus when whole part is zero
}
writeIntText(static_cast<typename T::NativeType>(wholePart(value)), ostr);
if (scale)
{
writeChar('.', ostr);
String str_fractional(scale, '0');
for (Int32 pos = scale - 1; pos >= 0; --pos, value /= T(10))
str_fractional[pos] += value % T(10);
ostr.write(str_fractional.data(), scale);
}
writeText(value, scale, 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);
}
@ -86,7 +81,7 @@ T DataTypeDecimal<T>::parseFromString(const String & str) const
template <typename T>
void DataTypeDecimal<T>::serializeBinary(const Field & field, WriteBuffer & ostr) const
{
FieldType x = get<DecimalField>(field);
FieldType x = get<DecimalField<T>>(field);
writeBinary(x, ostr);
}
@ -148,14 +143,27 @@ Field DataTypeDecimal<T>::getDefault() const
template <typename T>
MutableColumnPtr DataTypeDecimal<T>::createColumn() const
{
auto column = ColumnType::create();
column->getData().setScale(scale);
return column;
return ColumnType::create(0, scale);
}
//
DataTypePtr createDecimal(UInt64 precision_value, UInt64 scale_value)
{
if (precision_value < minDecimalPrecision() || precision_value > maxDecimalPrecision<Decimal128>())
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<Decimal32>())
return std::make_shared<DataTypeDecimal<Decimal32>>(precision_value, scale_value);
else if (precision_value <= maxDecimalPrecision<Decimal64>())
return std::make_shared<DataTypeDecimal<Decimal64>>(precision_value, scale_value);
return std::make_shared<DataTypeDecimal<Decimal128>>(precision_value, scale_value);
}
static DataTypePtr create(const ASTPtr & arguments)
{
if (!arguments || arguments->children.size() != 2)
@ -170,24 +178,35 @@ 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<Decimal128>())
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<Decimal32>())
return std::make_shared<DataTypeDecimal<Decimal32>>(precision_value, scale_value);
else if (precision_value <= maxDecimalPrecision<Decimal64>())
return std::make_shared<DataTypeDecimal<Decimal64>>(precision_value, scale_value);
return std::make_shared<DataTypeDecimal<Decimal128>>(precision_value, scale_value);
return createDecimal(precision_value, scale_value);
}
template <typename T>
static DataTypePtr createExect(const ASTPtr & arguments)
{
if (!arguments || arguments->children.size() != 1)
throw Exception("Decimal data type family must have exactly two arguments: precision and scale",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const ASTLiteral * scale_arg = typeid_cast<const ASTLiteral *>(arguments->children[0].get());
if (!scale_arg || !(scale_arg->value.getType() == Field::Types::Int64 || scale_arg->value.getType() == Field::Types::UInt64))
throw Exception("Decimal data type family must have a two numbers as its arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
UInt64 precision = maxDecimalPrecision<T>();
UInt64 scale = scale_arg->value.get<UInt64>();
return createDecimal(precision, scale);
}
void registerDataTypeDecimal(DataTypeFactory & factory)
{
factory.registerDataType("Decimal32", createExect<Decimal32>, DataTypeFactory::CaseInsensitive);
factory.registerDataType("Decimal64", createExect<Decimal64>, DataTypeFactory::CaseInsensitive);
factory.registerDataType("Decimal128", createExect<Decimal128>, DataTypeFactory::CaseInsensitive);
factory.registerDataType("Decimal", create, DataTypeFactory::CaseInsensitive);
factory.registerAlias("DEC", "Decimal", DataTypeFactory::CaseInsensitive);
}
@ -196,19 +215,19 @@ void registerDataTypeDecimal(DataTypeFactory & factory)
template <>
Decimal32 DataTypeDecimal<Decimal32>::getScaleMultiplier(UInt32 scale_)
{
return common::exp10_i32(scale_);
return decimalScaleMultiplier<Int32>(scale_);
}
template <>
Decimal64 DataTypeDecimal<Decimal64>::getScaleMultiplier(UInt32 scale_)
{
return common::exp10_i64(scale_);
return decimalScaleMultiplier<Int64>(scale_);
}
template <>
Decimal128 DataTypeDecimal<Decimal128>::getScaleMultiplier(UInt32 scale_)
{
return common::exp10_i128(scale_);
return decimalScaleMultiplier<Int128>(scale_);
}

View File

@ -1,7 +1,7 @@
#pragma once
#include <common/likely.h>
#include <Common/typeid_cast.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypesNumber.h>
@ -13,6 +13,7 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int CANNOT_CONVERT_TYPE;
}
///
@ -62,6 +63,11 @@ class DataTypeSimpleSerialization : public IDataType
};
class Context;
bool decimalCheckComparisonOverflow(const Context & context);
bool decimalCheckArithmeticOverflow(const Context & context);
static constexpr size_t minDecimalPrecision() { return 1; }
template <typename T> static constexpr size_t maxDecimalPrecision() { return 0; }
template <> constexpr size_t maxDecimalPrecision<Decimal32>() { return 9; }
@ -69,6 +75,9 @@ template <> constexpr size_t maxDecimalPrecision<Decimal64>() { return 18; }
template <> constexpr size_t maxDecimalPrecision<Decimal128>() { 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
@ -84,23 +93,25 @@ class DataTypeDecimal final : public DataTypeSimpleSerialization
{
public:
using FieldType = T;
using ColumnType = ColumnVector<T>;
using ColumnType = ColumnDecimal<T>;
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);
}
const char * getFamilyName() const override { return "Decimal"; }
std::string getName() const override;
size_t getTypeId() const override { return TypeId<T>::value; }
TypeIndex getTypeId() const override { return TypeId<T>::value; }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
@ -161,7 +172,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 +202,8 @@ public:
T parseFromString(const String & str) const;
void readText(T & x, ReadBuffer & istr) const { readText(x, istr, precision, scale); }
static void readText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale);
static T getScaleMultiplier(UInt32 scale);
private:
@ -253,6 +266,17 @@ inline bool isDecimal(const IDataType & data_type)
return false;
}
inline UInt32 getDecimalScale(const IDataType & data_type)
{
if (auto * decimal_type = checkDecimal<Decimal32>(data_type))
return decimal_type->getScale();
if (auto * decimal_type = checkDecimal<Decimal64>(data_type))
return decimal_type->getScale();
if (auto * decimal_type = checkDecimal<Decimal128>(data_type))
return decimal_type->getScale();
return std::numeric_limits<UInt32>::max();
}
///
inline bool notDecimalButComparableToDecimal(const IDataType & data_type)
{
@ -269,4 +293,54 @@ inline bool comparableToDecimal(const IDataType & data_type)
return isDecimal(data_type);
}
template <typename DataType> constexpr bool IsDecimal = false;
template <> constexpr bool IsDecimal<DataTypeDecimal<Decimal32>> = true;
template <> constexpr bool IsDecimal<DataTypeDecimal<Decimal64>> = true;
template <> constexpr bool IsDecimal<DataTypeDecimal<Decimal128>> = 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);
}
}

View File

@ -2,6 +2,7 @@
#include <DataTypes/FieldToDataType.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeNullable.h>
@ -58,6 +59,24 @@ DataTypePtr FieldToDataType::operator() (const String &) const
return std::make_shared<DataTypeString>();
}
DataTypePtr FieldToDataType::operator() (const DecimalField<Decimal32> & x) const
{
using Type = DataTypeDecimal<Decimal32>;
return std::make_shared<Type>(Type::maxPrecision(), x.getScale());
}
DataTypePtr FieldToDataType::operator() (const DecimalField<Decimal64> & x) const
{
using Type = DataTypeDecimal<Decimal64>;
return std::make_shared<Type>(Type::maxPrecision(), x.getScale());
}
DataTypePtr FieldToDataType::operator() (const DecimalField<Decimal128> & x) const
{
using Type = DataTypeDecimal<Decimal64>;
return std::make_shared<Type>(Type::maxPrecision(), x.getScale());
}
DataTypePtr FieldToDataType::operator() (const Array & x) const
{

View File

@ -25,6 +25,9 @@ public:
DataTypePtr operator() (const String & x) const;
DataTypePtr operator() (const Array & x) const;
DataTypePtr operator() (const Tuple & x) const;
DataTypePtr operator() (const DecimalField<Decimal32> & x) const;
DataTypePtr operator() (const DecimalField<Decimal64> & x) const;
DataTypePtr operator() (const DecimalField<Decimal128> & x) const;
};
}

View File

@ -46,7 +46,7 @@ public:
virtual const char * getFamilyName() const = 0;
/// Unique type number or zero
virtual size_t getTypeId() const { return 0; }
virtual TypeIndex getTypeId() const { return TypeIndex::None; }
/** Binary serialization for range of values in column - for writing to disk/network, etc.
*

View File

@ -7,102 +7,6 @@
#include <Core/Block.h>
#include <Core/ColumnNumbers.h>
namespace common
{
template <typename T>
inline bool addOverflow(T x, T y, T & res)
{
return __builtin_add_overflow(x, y, &res);
}
template <>
inline bool addOverflow(Int32 x, Int32 y, Int32 & res)
{
return __builtin_sadd_overflow(x, y, &res);
}
template <>
inline bool addOverflow(long x, long y, long & res)
{
return __builtin_saddl_overflow(x, y, &res);
}
template <>
inline bool addOverflow(long long x, long long y, long long & res)
{
return __builtin_saddll_overflow(x, y, &res);
}
template <>
inline bool addOverflow(__int128 x, __int128 y, __int128 & res)
{
res = x + y;
return (res - y) != x;
}
template <typename T>
inline bool subOverflow(T x, T y, T & res)
{
return __builtin_sub_overflow(x, y, &res);
}
template <>
inline bool subOverflow(Int32 x, Int32 y, Int32 & res)
{
return __builtin_ssub_overflow(x, y, &res);
}
template <>
inline bool subOverflow(long x, long y, long & res)
{
return __builtin_ssubl_overflow(x, y, &res);
}
template <>
inline bool subOverflow(long long x, long long y, long long & res)
{
return __builtin_ssubll_overflow(x, y, &res);
}
template <>
inline bool subOverflow(__int128 x, __int128 y, __int128 & res)
{
res = x - y;
return (res + y) != x;
}
template <typename T>
inline bool mulOverflow(T x, T y, T & res)
{
return __builtin_mul_overflow(x, y, &res);
}
template <>
inline bool mulOverflow(Int32 x, Int32 y, Int32 & res)
{
return __builtin_smul_overflow(x, y, &res);
}
template <>
inline bool mulOverflow(long x, long y, long & res)
{
return __builtin_smull_overflow(x, y, &res);
}
template <>
inline bool mulOverflow(long long x, long long y, long long & res)
{
return __builtin_smulll_overflow(x, y, &res);
}
template <>
inline bool mulOverflow(__int128 x, __int128 y, __int128 & res)
{
res = x * y;
return (res / y) != x;
}
}
namespace DB
{
@ -172,11 +76,17 @@ const ColumnConst * checkAndGetColumnConstStringOrFixedString(const IColumn * co
/// Transform anything to Field.
template <typename T>
inline Field toField(const T & x)
inline std::enable_if_t<!IsDecimalNumber<T>, Field> toField(const T & x)
{
return Field(typename NearestFieldType<T>::Type(x));
}
template <typename T>
inline std::enable_if_t<IsDecimalNumber<T>, Field> toField(const T & x, UInt32 scale)
{
return Field(typename NearestFieldType<T>::Type(x, scale));
}
Columns convertConstTupleToConstantElements(const ColumnConst & column);
@ -189,58 +99,200 @@ Block createBlockWithNestedColumns(const Block & block, const ColumnNumbers & ar
/// Similar function as above. Additionally transform the result type if needed.
Block createBlockWithNestedColumns(const Block & block, const ColumnNumbers & args, size_t result);
template <typename T, typename U>
struct TypePair
{
using LeftType = T;
using RightType = U;
};
template <typename T, bool _int, bool _dec, bool _float, typename F>
bool callOnBasicType(TypeIndex number, F && f)
{
if constexpr (_int)
{
switch (number)
{
case TypeIndex::UInt8: return f(TypePair<T, UInt8>());
case TypeIndex::UInt16: return f(TypePair<T, UInt16>());
case TypeIndex::UInt32: return f(TypePair<T, UInt32>());
case TypeIndex::UInt64: return f(TypePair<T, UInt64>());
//case TypeIndex::UInt128>: return f(TypePair<T, UInt128>());
case TypeIndex::Int8: return f(TypePair<T, Int8>());
case TypeIndex::Int16: return f(TypePair<T, Int16>());
case TypeIndex::Int32: return f(TypePair<T, Int32>());
case TypeIndex::Int64: return f(TypePair<T, Int64>());
case TypeIndex::Int128: return f(TypePair<T, Int128>());
default:
break;
}
}
if constexpr (_dec)
{
switch (number)
{
case TypeIndex::Decimal32: return f(TypePair<T, Decimal32>());
case TypeIndex::Decimal64: return f(TypePair<T, Decimal64>());
case TypeIndex::Decimal128: return f(TypePair<T, Decimal128>());
default:
break;
}
}
if constexpr (_float)
{
switch (number)
{
case TypeIndex::Float32: return f(TypePair<T, Float32>());
case TypeIndex::Float64: return f(TypePair<T, Float64>());
default:
break;
}
}
return false;
}
/// Unroll template using TypeIndex
template <typename F, bool _int = true, bool _dec = true, bool _float = false>
inline bool callOnBasicTypes(TypeIndex type_num1, TypeIndex type_num2, F && f)
{
if constexpr (_int)
{
switch (type_num1)
{
case TypeIndex::UInt8: return callOnBasicType<UInt8, _int, _dec, _float>(type_num2, std::forward<F>(f));
case TypeIndex::UInt16: return callOnBasicType<UInt16, _int, _dec, _float>(type_num2, std::forward<F>(f));
case TypeIndex::UInt32: return callOnBasicType<UInt32, _int, _dec, _float>(type_num2, std::forward<F>(f));
case TypeIndex::UInt64: return callOnBasicType<UInt64, _int, _dec, _float>(type_num2, std::forward<F>(f));
//case TypeIndex::UInt128: return callOnBasicType<UInt128, _int, _dec, _float>(type_num2, std::forward<F>(f));
case TypeIndex::Int8: return callOnBasicType<Int8, _int, _dec, _float>(type_num2, std::forward<F>(f));
case TypeIndex::Int16: return callOnBasicType<Int16, _int, _dec, _float>(type_num2, std::forward<F>(f));
case TypeIndex::Int32: return callOnBasicType<Int32, _int, _dec, _float>(type_num2, std::forward<F>(f));
case TypeIndex::Int64: return callOnBasicType<Int64, _int, _dec, _float>(type_num2, std::forward<F>(f));
case TypeIndex::Int128: return callOnBasicType<Int128, _int, _dec, _float>(type_num2, std::forward<F>(f));
default:
break;
}
}
if constexpr (_dec)
{
switch (type_num1)
{
case TypeIndex::Decimal32: return callOnBasicType<Decimal32, _int, _dec, _float>(type_num2, std::forward<F>(f));
case TypeIndex::Decimal64: return callOnBasicType<Decimal64, _int, _dec, _float>(type_num2, std::forward<F>(f));
case TypeIndex::Decimal128: return callOnBasicType<Decimal128, _int, _dec, _float>(type_num2, std::forward<F>(f));
default:
break;
}
}
if constexpr (_float)
{
switch (type_num1)
{
case TypeIndex::Float32: return callOnBasicType<Float32, _int, _dec, _float>(type_num2, std::forward<F>(f));
case TypeIndex::Float64: return callOnBasicType<Float64, _int, _dec, _float>(type_num2, std::forward<F>(f));
default:
break;
}
}
return false;
}
class DataTypeDate;
class DataTypeDateTime;
class DataTypeString;
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 callByTypeAndNumber(UInt8 number, F && f)
bool callOnDataTypeAndIndex(TypeIndex number, F && f)
{
switch (number)
{
case TypeId<UInt8>::value: f(T(), UInt8()); break;
case TypeId<UInt16>::value: f(T(), UInt16()); break;
case TypeId<UInt32>::value: f(T(), UInt32()); break;
case TypeId<UInt64>::value: f(T(), UInt64()); break;
//case TypeId<UInt128>::value: f(T(), UInt128()); break;
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 TypeId<Int8>::value: f(T(), Int8()); break;
case TypeId<Int16>::value: f(T(), Int16()); break;
case TypeId<Int32>::value: f(T(), Int32()); break;
case TypeId<Int64>::value: f(T(), Int64()); break;
case TypeId<Int128>::value: f(T(), Int128()); break;
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 TypeId<Decimal32>::value: f(T(), Decimal32()); break;
case TypeId<Decimal64>::value: f(T(), Decimal64()); break;
case TypeId<Decimal128>::value: f(T(), Decimal128()); break;
default:
return false;
}
case TypeIndex::Float32: return f(TypePair<T, DataTypeNumber<Float32>>());
case TypeIndex::Float64: return f(TypePair<T, DataTypeNumber<Float64>>());
return true;
}
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>>());
/// Unroll template using TypeNumber<T>
template <typename F>
inline bool callByNumbers(UInt8 type_num1, UInt8 type_num2, F && f)
{
switch (type_num1)
{
case TypeId<UInt8>::value: return callByTypeAndNumber<UInt8>(type_num2, std::forward<F>(f));
case TypeId<UInt16>::value: return callByTypeAndNumber<UInt16>(type_num2, std::forward<F>(f));
case TypeId<UInt32>::value: return callByTypeAndNumber<UInt32>(type_num2, std::forward<F>(f));
case TypeId<UInt64>::value: return callByTypeAndNumber<UInt64>(type_num2, std::forward<F>(f));
//case TypeId<UInt128>::value: return callByTypeAndNumber<UInt128>(type_num2, std::forward<F>(f));
case TypeIndex::Date: return f(TypePair<T, DataTypeDate>());
case TypeIndex::DateTime: return f(TypePair<T, DataTypeDateTime>());
case TypeId<Int8>::value: return callByTypeAndNumber<Int8>(type_num2, std::forward<F>(f));
case TypeId<Int16>::value: return callByTypeAndNumber<Int16>(type_num2, std::forward<F>(f));
case TypeId<Int32>::value: return callByTypeAndNumber<Int32>(type_num2, std::forward<F>(f));
case TypeId<Int64>::value: return callByTypeAndNumber<Int64>(type_num2, std::forward<F>(f));
case TypeId<Int128>::value: return callByTypeAndNumber<Int128>(type_num2, std::forward<F>(f));
case TypeIndex::String: return f(TypePair<T, DataTypeString>());
case TypeIndex::FixedString: return f(TypePair<T, DataTypeFixedString>());
case TypeId<Decimal32>::value: return callByTypeAndNumber<Decimal32>(type_num2, std::forward<F>(f));
case TypeId<Decimal64>::value: return callByTypeAndNumber<Decimal64>(type_num2, std::forward<F>(f));
case TypeId<Decimal128>::value: return callByTypeAndNumber<Decimal128>(type_num2, std::forward<F>(f));
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)
{
switch (number)
{
case TypeIndex::UInt8: return f(TypePair<DataTypeNumber<UInt8>, T>());
case TypeIndex::UInt16: return f(TypePair<DataTypeNumber<UInt16>, T>());
case TypeIndex::UInt32: return f(TypePair<DataTypeNumber<UInt32>, T>());
case TypeIndex::UInt64: return f(TypePair<DataTypeNumber<UInt64>, T>());
case TypeIndex::Int8: return f(TypePair<DataTypeNumber<Int8>, T>());
case TypeIndex::Int16: return f(TypePair<DataTypeNumber<Int16>, T>());
case TypeIndex::Int32: return f(TypePair<DataTypeNumber<Int32>, T>());
case TypeIndex::Int64: return f(TypePair<DataTypeNumber<Int64>, T>());
case TypeIndex::Float32: return f(TypePair<DataTypeNumber<Float32>, T>());
case TypeIndex::Float64: return f(TypePair<DataTypeNumber<Float64>, T>());
case TypeIndex::Decimal32: return f(TypePair<DataTypeDecimal<Decimal32>, T>());
case TypeIndex::Decimal64: return f(TypePair<DataTypeDecimal<Decimal64>, T>());
case TypeIndex::Decimal128: return f(TypePair<DataTypeDecimal<Decimal128>, T>());
case TypeIndex::Date: return f(TypePair<DataTypeDate, T>());
case TypeIndex::DateTime: return f(TypePair<DataTypeDateTime, T>());
case TypeIndex::String: return f(TypePair<DataTypeString, T>());
case TypeIndex::FixedString: return f(TypePair<DataTypeFixedString, T>());
case TypeIndex::Enum8: return f(TypePair<DataTypeEnum<Int8>, T>());
case TypeIndex::Enum16: return f(TypePair<DataTypeEnum<Int16>, T>());
case TypeIndex::UUID: return f(TypePair<DataTypeUUID, T>());
default:
break;
}
return false;
}

View File

@ -7,6 +7,7 @@
#include <DataTypes/DataTypeInterval.h>
#include <DataTypes/Native.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnConst.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
@ -19,6 +20,7 @@
#include <Interpreters/ExpressionActions.h>
#include <ext/range.h>
#include <common/intExp.h>
#include <common/arithmeticOverflow.h>
#include <boost/integer/common_factor.hpp>
#if USE_EMBEDDED_COMPILER
@ -120,6 +122,7 @@ struct PlusImpl
return static_cast<Result>(a) + b;
}
/// Apply operation and check overflow. It's used for Deciamal operations. @returns true if overflowed, false othervise.
template <typename Result = ResultType>
static inline bool apply(A a, B b, Result & c)
{
@ -149,6 +152,7 @@ struct MultiplyImpl
return static_cast<Result>(a) * b;
}
/// Apply operation and check overflow. It's used for Deciamal operations. @returns true if overflowed, false othervise.
template <typename Result = ResultType>
static inline bool apply(A a, B b, Result & c)
{
@ -177,6 +181,7 @@ struct MinusImpl
return static_cast<Result>(a) - b;
}
/// Apply operation and check overflow. It's used for Deciamal operations. @returns true if overflowed, false othervise.
template <typename Result = ResultType>
static inline bool apply(A a, B b, Result & c)
{
@ -725,15 +730,16 @@ template <> struct NativeType<Decimal128> { using Type = Int128; };
/// +|- scale one of args (which scale factor is not 1). ScaleR = oneof(Scale1, Scale2);
/// * no agrs scale. ScaleR = Scale1 + Scale2;
/// / first arg scale. ScaleR = Scale1 (scale_a = DecimalType<B>::getScale()).
template <typename A, typename B, template <typename, typename> typename Operation, typename ResultType_>
template <typename A, typename B, template <typename, typename> typename Operation, typename ResultType_, bool _check_overflow = true>
struct DecimalBinaryOperation
{
using ResultType = ResultType_;
using NativeResultType = typename NativeType<ResultType>::Type;
using Op = Operation<NativeResultType, NativeResultType>;
using ArrayA = typename ColumnVector<A>::Container;
using ArrayB = typename ColumnVector<B>::Container;
using ArrayC = typename ColumnVector<ResultType>::Container;
using ArrayA = std::conditional_t<IsDecimalNumber<A>, typename ColumnDecimal<A>::Container, typename ColumnVector<A>::Container>;
using ArrayB = std::conditional_t<IsDecimalNumber<B>, typename ColumnDecimal<B>::Container, typename ColumnVector<B>::Container>;
using ArrayC = typename ColumnDecimal<ResultType>::Container;
using XOverflow = DecimalBinaryOperation<A, B, Operation, ResultType_, !_check_overflow>;
static constexpr bool is_plus_minus = std::is_same_v<Operation<Int32, Int32>, PlusImpl<Int32, Int32>> ||
std::is_same_v<Operation<Int32, Int32>, MinusImpl<Int32, Int32>>;
@ -855,7 +861,7 @@ private:
/// there's implicit type convertion here
static NativeResultType apply(NativeResultType a, NativeResultType b)
{
if constexpr (can_overflow)
if constexpr (can_overflow && _check_overflow)
{
NativeResultType res;
if (Op::template apply<NativeResultType>(a, b, res))
@ -873,19 +879,30 @@ private:
{
NativeResultType res;
bool overflow = false;
if constexpr (scale_left)
overflow |= common::mulOverflow(a, scale, a);
else
overflow |= common::mulOverflow(b, scale, b);
if constexpr (_check_overflow)
{
bool overflow = false;
if constexpr (scale_left)
overflow |= common::mulOverflow(a, scale, a);
else
overflow |= common::mulOverflow(b, scale, b);
if constexpr (can_overflow)
overflow |= Op::template apply<NativeResultType>(a, b, res);
if constexpr (can_overflow)
overflow |= Op::template apply<NativeResultType>(a, b, res);
else
res = Op::template apply<NativeResultType>(a, b);
if (overflow)
throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW);
}
else
{
if constexpr (scale_left)
a *= scale;
else
b *= scale;
res = Op::template apply<NativeResultType>(a, b);
if (overflow)
throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW);
}
return res;
}
@ -895,12 +912,21 @@ private:
{
if constexpr (is_division)
{
bool overflow = false;
if constexpr (!IsDecimalNumber<A>)
overflow |= common::mulOverflow(scale, scale, scale);
overflow |= common::mulOverflow(a, scale, a);
if (overflow)
throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW);
if constexpr (_check_overflow)
{
bool overflow = false;
if constexpr (!IsDecimalNumber<A>)
overflow |= common::mulOverflow(scale, scale, scale);
overflow |= common::mulOverflow(a, scale, a);
if (overflow)
throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW);
}
else
{
if constexpr (!IsDecimalNumber<A>)
scale *= scale;
a *= scale;
}
return Op::template apply<NativeResultType>(a, b);
}
@ -930,11 +956,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<Decimal32>> = true;
template <> constexpr bool IsDecimal<DataTypeDecimal<Decimal64>> = true;
template <> constexpr bool IsDecimal<DataTypeDecimal<Decimal128>> = true;
template <typename T0, typename T1> constexpr bool UseLeftDecimal = false;
template <> constexpr bool UseLeftDecimal<DataTypeDecimal<Decimal128>, DataTypeDecimal<Decimal32>> = true;
template <> constexpr bool UseLeftDecimal<DataTypeDecimal<Decimal128>, DataTypeDecimal<Decimal64>> = true;
@ -1077,7 +1098,7 @@ public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionBinaryArithmetic>(context); }
FunctionBinaryArithmetic(const Context & context) : context(context) {}
FunctionBinaryArithmetic(const Context & context_) : context(context_) {}
String getName() const override
{
@ -1179,9 +1200,9 @@ public:
using T0 = typename LeftDataType::FieldType;
using T1 = typename RightDataType::FieldType;
using ResultType = typename ResultDataType::FieldType;
using ColVecT0 = ColumnVector<T0>;
using ColVecT1 = ColumnVector<T1>;
using ColVecResult = ColumnVector<ResultType>;
using ColVecT0 = std::conditional_t<IsDecimalNumber<T0>, ColumnDecimal<T0>, ColumnVector<T0>>;
using ColVecT1 = std::conditional_t<IsDecimalNumber<T1>, ColumnDecimal<T1>, ColumnVector<T1>>;
using ColVecResult = std::conditional_t<IsDecimalNumber<ResultType>, ColumnDecimal<ResultType>, ColumnVector<ResultType>>;
/// Decimal operations need scale. Operations are on result type.
using OpImpl = std::conditional_t<IsDecimal<ResultDataType>,
@ -1202,10 +1223,22 @@ public:
typename ResultDataType::FieldType scale_b = type.scaleFactorFor(right, is_multiply || is_division);
if constexpr (IsDecimal<RightDataType> && is_division)
scale_a = right.getScaleMultiplier();
auto res = OpImpl::constant_constant(col_left->template getValue<T0>(), col_right->template getValue<T1>(),
scale_a, scale_b);
block.getByPosition(result).column =
ResultDataType(type.getPrecision(), type.getScale()).createColumnConst(col_left->size(), toField(res));
if (decimalCheckArithmeticOverflow(context))
{
auto res = OpImpl::constant_constant(
col_left->template getValue<T0>(), col_right->template getValue<T1>(), scale_a, scale_b);
block.getByPosition(result).column =
ResultDataType(type.getPrecision(), type.getScale()).createColumnConst(
col_left->size(), toField(res, type.getScale()));
}
else
{
auto res = OpImpl::XOverflow::constant_constant(
col_left->template getValue<T0>(), col_right->template getValue<T1>(), scale_a, scale_b);
block.getByPosition(result).column =
ResultDataType(type.getPrecision(), type.getScale()).createColumnConst(
col_left->size(), toField(res, type.getScale()));
}
}
else
{
@ -1216,9 +1249,18 @@ public:
}
}
auto col_res = ColVecResult::create();
typename ColVecResult::MutablePtr col_res = nullptr;
if constexpr (result_is_decimal)
{
ResultDataType type = decimalResultType(left, right, is_multiply, is_division);
col_res = ColVecResult::create(0, type.getScale());
}
else
col_res = ColVecResult::create();
auto & vec_res = col_res->getData();
vec_res.resize(block.rows());
if (auto col_left = checkAndGetColumnConst<ColVecT0>(col_left_raw))
{
if (auto col_right = checkAndGetColumn<ColVecT1>(col_right_raw))
@ -1226,13 +1268,16 @@ public:
if constexpr (result_is_decimal)
{
ResultDataType type = decimalResultType(left, right, is_multiply, is_division);
vec_res.setScale(type.getScale());
typename ResultDataType::FieldType scale_a = type.scaleFactorFor(left, is_multiply);
typename ResultDataType::FieldType scale_b = type.scaleFactorFor(right, is_multiply || is_division);
if constexpr (IsDecimal<RightDataType> && is_division)
scale_a = right.getScaleMultiplier();
OpImpl::constant_vector(col_left->template getValue<T0>(), col_right->getData(), vec_res, scale_a, scale_b);
if (decimalCheckArithmeticOverflow(context))
OpImpl::constant_vector(col_left->template getValue<T0>(), col_right->getData(), vec_res, scale_a, scale_b);
else
OpImpl::XOverflow::constant_vector(
col_left->template getValue<T0>(), col_right->getData(), vec_res, scale_a, scale_b);
}
else
OpImpl::constant_vector(col_left->template getValue<T0>(), col_right->getData(), vec_res);
@ -1245,16 +1290,26 @@ public:
if constexpr (result_is_decimal)
{
ResultDataType type = decimalResultType(left, right, is_multiply, is_division);
vec_res.setScale(type.getScale());
typename ResultDataType::FieldType scale_a = type.scaleFactorFor(left, is_multiply);
typename ResultDataType::FieldType scale_b = type.scaleFactorFor(right, is_multiply || is_division);
if constexpr (IsDecimal<RightDataType> && is_division)
scale_a = right.getScaleMultiplier();
if (auto col_right = checkAndGetColumn<ColVecT1>(col_right_raw))
OpImpl::vector_vector(col_left->getData(), col_right->getData(), vec_res, scale_a, scale_b);
{
if (decimalCheckArithmeticOverflow(context))
OpImpl::vector_vector(col_left->getData(), col_right->getData(), vec_res, scale_a, scale_b);
else
OpImpl::XOverflow::vector_vector(col_left->getData(), col_right->getData(), vec_res, scale_a, scale_b);
}
else if (auto col_right = checkAndGetColumnConst<ColVecT1>(col_right_raw))
OpImpl::vector_constant(col_left->getData(), col_right->template getValue<T1>(), vec_res, scale_a, scale_b);
{
if (decimalCheckArithmeticOverflow(context))
OpImpl::vector_constant(col_left->getData(), col_right->template getValue<T1>(), vec_res, scale_a, scale_b);
else
OpImpl::XOverflow::vector_constant(
col_left->getData(), col_right->template getValue<T1>(), vec_res, scale_a, scale_b);
}
else
return false;
}
@ -1400,9 +1455,9 @@ public:
{
if constexpr (allow_decimal)
{
if (auto col = checkAndGetColumn<ColumnVector<T0>>(block.getByPosition(arguments[0]).column.get()))
if (auto col = checkAndGetColumn<ColumnDecimal<T0>>(block.getByPosition(arguments[0]).column.get()))
{
auto col_res = ColumnVector<typename Op<T0>::ResultType>::create();
auto col_res = ColumnDecimal<typename Op<T0>::ResultType>::create(0, type.getScale());
auto & vec_res = col_res->getData();
vec_res.resize(col->getData().size());
UnaryOperationImpl<T0, Op<T0>>::vector(col->getData(), vec_res);

View File

@ -2,6 +2,7 @@
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnTuple.h>
@ -221,14 +222,15 @@ struct DecCompareInt
};
///
template <typename A, typename B, template <typename, typename> typename Operation, bool _actual = IsDecimalNumber<A> || IsDecimalNumber<B>>
template <typename A, typename B, template <typename, typename> typename Operation, bool _check_overflow = true,
bool _actual = IsDecimalNumber<A> || IsDecimalNumber<B>>
class DecimalComparison
{
public:
using CompareInt = typename DecCompareInt<A, B>::Type;
using Op = Operation<CompareInt, CompareInt>;
using ColVecA = ColumnVector<A>;
using ColVecB = ColumnVector<B>;
using ColVecA = std::conditional_t<IsDecimalNumber<A>, ColumnDecimal<A>, ColumnVector<A>>;
using ColVecB = std::conditional_t<IsDecimalNumber<B>, ColumnDecimal<B>, ColumnVector<B>>;
using ArrayA = typename ColVecA::Container;
using ArrayB = typename ColVecB::Container;
@ -365,8 +367,8 @@ private:
A a = c0_const->template getValue<A>();
if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
constant_vector<scale_left, scale_right>(a, c1_vec->getData(), vec_res, scale);
else if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
constant_vector<scale_left, scale_right>(a, c1_vec->getData(), vec_res, scale);
else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
}
else if (c1_const)
{
@ -374,8 +376,8 @@ private:
B b = c1_const->template getValue<B>();
if (const ColVecA * c0_vec = checkAndGetColumn<ColVecA>(c0.get()))
vector_constant<scale_left, scale_right>(c0_vec->getData(), b, vec_res, scale);
else if (const ColVecA * c0_vec = checkAndGetColumn<ColVecA>(c0.get()))
vector_constant<scale_left, scale_right>(c0_vec->getData(), b, vec_res, scale);
else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
}
else
{
@ -383,15 +385,15 @@ private:
{
if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
vector_vector<scale_left, scale_right>(c0_vec->getData(), c1_vec->getData(), vec_res, scale);
else if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
vector_vector<scale_left, scale_right>(c0_vec->getData(), c1_vec->getData(), vec_res, scale);
else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
}
else if (const ColVecA * c0_vec = checkAndGetColumn<ColVecA>(c0.get()))
{
if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
vector_vector<scale_left, scale_right>(c0_vec->getData(), c1_vec->getData(), vec_res, scale);
else if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
vector_vector<scale_left, scale_right>(c0_vec->getData(), c1_vec->getData(), vec_res, scale);
else
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
}
}
}
@ -404,24 +406,35 @@ private:
{
CompareInt x = a;
CompareInt y = b;
bool overflow = false;
if constexpr (sizeof(A) > sizeof(CompareInt))
overflow |= (A(x) != a);
if constexpr (sizeof(B) > sizeof(CompareInt))
overflow |= (B(y) != b);
if constexpr (std::is_unsigned_v<A>)
overflow |= (x < 0);
if constexpr (std::is_unsigned_v<B>)
overflow |= (y < 0);
if constexpr (_check_overflow)
{
bool overflow = false;
if constexpr (scale_left)
overflow |= common::mulOverflow(x, scale, x);
if constexpr (scale_right)
overflow |= common::mulOverflow(y, scale, y);
if constexpr (sizeof(A) > sizeof(CompareInt))
overflow |= (A(x) != a);
if constexpr (sizeof(B) > sizeof(CompareInt))
overflow |= (B(y) != b);
if constexpr (std::is_unsigned_v<A>)
overflow |= (x < 0);
if constexpr (std::is_unsigned_v<B>)
overflow |= (y < 0);
if (overflow)
throw Exception("Can't compare", ErrorCodes::DECIMAL_OVERFLOW);
if constexpr (scale_left)
overflow |= common::mulOverflow(x, scale, x);
if constexpr (scale_right)
overflow |= common::mulOverflow(y, scale, y);
if (overflow)
throw Exception("Can't compare", ErrorCodes::DECIMAL_OVERFLOW);
}
else
{
if constexpr (scale_left)
x *= scale;
if constexpr (scale_right)
y *= scale;
}
return Op::apply(x, y);
}
@ -1021,18 +1034,23 @@ private:
void executeDecimal(Block & block, size_t result, const ColumnWithTypeAndName & col_left, const ColumnWithTypeAndName & col_right)
{
size_t left_number = col_left.type->getTypeId();
size_t right_number = col_right.type->getTypeId();
TypeIndex left_number = col_left.type->getTypeId();
TypeIndex right_number = col_right.type->getTypeId();
auto call = [&](const auto & left, const auto & right)
auto call = [&](const auto & types) -> bool
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
using Types = std::decay_t<decltype(types)>;
using LeftDataType = typename Types::LeftType;
using RightDataType = typename Types::RightType;
DecimalComparison<LeftDataType, RightDataType, Op>(block, result, col_left, col_right);
if (decimalCheckComparisonOverflow(context))
DecimalComparison<LeftDataType, RightDataType, Op, true>(block, result, col_left, col_right);
else
DecimalComparison<LeftDataType, RightDataType, Op, false>(block, result, col_left, col_right);
return true;
};
if (!callByNumbers(left_number, right_number, call))
if (!callOnBasicTypes(left_number, right_number, call))
throw Exception("Wrong call for " + getName() + " with " + col_left.type->getName() + " and " + col_right.type->getName(),
ErrorCodes::LOGICAL_ERROR);
}

View File

@ -40,6 +40,10 @@ void registerFunctionsConversion(FunctionFactory & factory)
factory.registerFunction<FunctionToFloat32>();
factory.registerFunction<FunctionToFloat64>();
factory.registerFunction<FunctionToDecimal32>();
factory.registerFunction<FunctionToDecimal64>();
factory.registerFunction<FunctionToDecimal128>();
factory.registerFunction<FunctionToDate>();
factory.registerFunction<FunctionToDateTime>();
factory.registerFunction<FunctionToUUID>();

View File

@ -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,24 +100,44 @@ 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()))
{
auto col_to = ColumnVector<ToFieldType>::create();
const ColumnWithTypeAndName & named_from = block.getByPosition(arguments[0]);
const typename ColumnVector<FromFieldType>::Container & vec_from = col_from->getData();
typename ColumnVector<ToFieldType>::Container & vec_to = col_to->getData();
using ColVecFrom = std::conditional_t<IsDecimalNumber<FromFieldType>, ColumnDecimal<FromFieldType>, ColumnVector<FromFieldType>>;
using ColVecTo = std::conditional_t<IsDecimalNumber<ToFieldType>, ColumnDecimal<ToFieldType>, ColumnVector<ToFieldType>>;
if (const ColVecFrom * col_from = checkAndGetColumn<ColVecFrom>(named_from.column.get()))
{
typename ColVecTo::MutablePtr col_to = nullptr;
if constexpr (IsDecimal<ToDataType>)
{
const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]);
UInt32 scale = extractToDecimalScale(scale_column);
col_to = ColVecTo::create(0, scale);
}
else
col_to = ColVecTo::create();
const auto & vec_from = col_from->getData();
auto & vec_to = col_to->getData();
size_t size = vec_from.size();
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 +236,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 *)
{
writeText(x, type->getScale(), wb);
}
};
/// DataTypeEnum<T> to DataType<T> free conversion
template <typename FieldType, typename Name>
@ -411,6 +456,8 @@ struct ConvertThroughParsing
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
using ColVecTo = std::conditional_t<IsDecimalNumber<ToFieldType>, ColumnDecimal<ToFieldType>, ColumnVector<ToFieldType>>;
const DateLUTImpl * local_time_zone [[maybe_unused]] = nullptr;
const DateLUTImpl * utc_time_zone [[maybe_unused]] = nullptr;
@ -438,8 +485,19 @@ struct ConvertThroughParsing
ErrorCodes::ILLEGAL_COLUMN);
size_t size = input_rows_count;
auto col_to = ColumnVector<ToFieldType>::create(size);
typename ColumnVector<ToFieldType>::Container & vec_to = col_to->getData();
typename ColVecTo::MutablePtr col_to = nullptr;
if constexpr (IsDecimal<ToDataType>)
{
const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]);
UInt32 scale = extractToDecimalScale(scale_column);
col_to = ColVecTo::create(size, scale);
ToDataType check_bounds_in_ctor(ToDataType::maxPrecision(), scale);
}
else
col_to = ColVecTo::create(size);
typename ColVecTo::Container & vec_to = col_to->getData();
ColumnUInt8::MutablePtr col_null_map_to;
ColumnUInt8::Container * vec_null_map_to [[maybe_unused]] = nullptr;
@ -473,7 +531,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 +592,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 +708,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 NameToDecimal32 { static constexpr auto name = "toDecimal32"; };
struct NameToDecimal64 { static constexpr auto name = "toDecimal64"; };
struct NameToDecimal128 { static constexpr auto name = "toDecimal128"; };
#define DEFINE_NAME_TO_INTERVAL(INTERVAL_KIND) \
@ -674,6 +738,9 @@ public:
using Monotonic = MonotonicityImpl;
static constexpr auto name = Name::name;
static constexpr bool to_decimal =
std::is_same_v<Name, NameToDecimal32> || std::is_same_v<Name, NameToDecimal64> || std::is_same_v<Name, NameToDecimal128>;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionConvert>(); }
String getName() const override
@ -687,7 +754,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 +769,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, NameToDecimal32>)
return createDecimal(9, scale);
else if constexpr (std::is_same_v<Name, NameToDecimal64>)
return createDecimal(18, scale);
else if constexpr ( std::is_same_v<Name, NameToDecimal128>)
return createDecimal(38, scale);
throw Exception("Someting wrong with toDecimalNN()", ErrorCodes::LOGICAL_ERROR);
}
else
{
/** Optional second argument with time zone is supported:
@ -709,11 +795,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.",
@ -785,26 +872,18 @@ private:
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
if (checkDataType<DataTypeUInt8>(from_type)) ConvertImpl<DataTypeUInt8, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeUInt16>(from_type)) ConvertImpl<DataTypeUInt16, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeUInt32>(from_type)) ConvertImpl<DataTypeUInt32, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeUInt64>(from_type)) ConvertImpl<DataTypeUInt64, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeInt8>(from_type)) ConvertImpl<DataTypeInt8, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeInt16>(from_type)) ConvertImpl<DataTypeInt16, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeInt32>(from_type)) ConvertImpl<DataTypeInt32, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeInt64>(from_type)) ConvertImpl<DataTypeInt64, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeFloat32>(from_type)) ConvertImpl<DataTypeFloat32, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeFloat64>(from_type)) ConvertImpl<DataTypeFloat64, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeDate>(from_type)) ConvertImpl<DataTypeDate, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeDateTime>(from_type)) ConvertImpl<DataTypeDateTime, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeUUID>(from_type)) ConvertImpl<DataTypeUUID, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeString>(from_type))
ConvertImpl<DataTypeString, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeFixedString>(from_type))
ConvertImpl<DataTypeFixedString, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeEnum8>(from_type)) ConvertImpl<DataTypeEnum8, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeEnum16>(from_type)) ConvertImpl<DataTypeEnum16, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else
auto call = [&](const auto & types) -> bool
{
using Types = std::decay_t<decltype(types)>;
using LeftDataType = typename Types::LeftType;
using RightDataType = typename Types::RightType;
ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, arguments, result, input_rows_count);
return true;
};
bool done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call);
if (!done)
{
/// Generic conversion of any type to String.
if (std::is_same_v<ToDataType, DataTypeString>)
@ -1000,6 +1079,15 @@ struct PositiveMonotonicity
}
};
struct UnknownMonotonicity
{
static bool has() { return false; }
static IFunction::Monotonicity get(const IDataType &, const Field &, const Field &)
{
return { false };
}
};
template <typename T>
struct ToIntMonotonicity
{
@ -1136,6 +1224,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 FunctionToDecimal32 = FunctionConvert<DataTypeDecimal<Decimal32>, NameToDecimal32, UnknownMonotonicity>;
using FunctionToDecimal64 = FunctionConvert<DataTypeDecimal<Decimal64>, NameToDecimal64, UnknownMonotonicity>;
using FunctionToDecimal128 = FunctionConvert<DataTypeDecimal<Decimal128>, NameToDecimal128, UnknownMonotonicity>;
template <typename DataType> struct FunctionTo;
@ -1155,6 +1246,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<Decimal32>> { using Type = FunctionToDecimal32; };
template <> struct FunctionTo<DataTypeDecimal<Decimal64>> { using Type = FunctionToDecimal64; };
template <> struct FunctionTo<DataTypeDecimal<Decimal128>> { using Type = FunctionToDecimal128; };
template <typename FieldType> struct FunctionTo<DataTypeEnum<FieldType>>
: FunctionTo<DataTypeNumber<FieldType>>

View File

@ -10,6 +10,7 @@
#include <common/LocalDate.h>
#include <common/LocalDateTime.h>
#include <common/find_first_symbols.h>
#include <common/intExp.h>
#include <Core/Types.h>
#include <Core/UUID.h>
@ -711,6 +712,36 @@ inline void writeText(const UInt128 &, WriteBuffer &)
throw Exception("UInt128 cannot be write as a text", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
template <typename T> inline T decimalScaleMultiplier(UInt32 scale);
template <> inline Int32 decimalScaleMultiplier<Int32>(UInt32 scale) { return common::exp10_i32(scale); }
template <> inline Int64 decimalScaleMultiplier<Int64>(UInt32 scale) { return common::exp10_i64(scale); }
template <> inline Int128 decimalScaleMultiplier<Int128>(UInt32 scale) { return common::exp10_i128(scale); }
template <typename T>
void writeText(Decimal<T> value, UInt32 scale, WriteBuffer & ostr)
{
if (value < Decimal<T>(0))
{
value *= Decimal<T>(-1);
writeChar('-', ostr); /// avoid crop leading minus when whole part is zero
}
T whole_part = value;
if (scale)
whole_part = value / decimalScaleMultiplier<T>(scale);
writeIntText(whole_part, ostr);
if (scale)
{
writeChar('.', ostr);
String str_fractional(scale, '0');
for (Int32 pos = scale - 1; pos >= 0; --pos, value /= Decimal<T>(10))
str_fractional[pos] += value % Decimal<T>(10);
ostr.write(str_fractional.data(), scale);
}
}
/// String, date, datetime are in single quotes with C-style escaping. Numbers - without.
template <typename T>
inline std::enable_if_t<std::is_arithmetic_v<T>, void>

View File

@ -278,6 +278,8 @@ struct Settings
M(SettingBool, low_cardinality_use_single_dictionary_for_part, false, "LowCardinality type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.") \
M(SettingBool, allow_experimental_low_cardinality_type, false, "Allows to create table with LowCardinality types.") \
M(SettingBool, allow_experimental_decimal_type, false, "Enables Decimal data type.") \
M(SettingBool, decimal_check_comparison_overflow, true, "Check overflow of decimal comparison operations") \
M(SettingBool, decimal_check_arithmetic_overflow, true, "Check overflow of decimal arithmetic operations") \
\
M(SettingBool, prefer_localhost_replica, 1, "1 - always send query to local replica, if it exists. 0 - choose replica to send query between local and remote ones according to load_balancing") \
M(SettingUInt64, max_fetch_partition_retries_count, 5, "Amount of retries while fetching partition from another host.") \

View File

@ -84,7 +84,7 @@ static Field convertIntToDecimalType(const Field & from, const To & type)
throw Exception("Number is too much to place in " + type.getName(), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
FieldType scaled_value = type.getScaleMultiplier() * value;
return Field(typename NearestFieldType<FieldType>::Type(scaled_value));
return Field(typename NearestFieldType<FieldType>::Type(scaled_value, type.getScale()));
}
@ -95,7 +95,7 @@ static Field convertStringToDecimalType(const Field & from, const DataTypeDecima
const String & str_value = from.get<String>();
T value = type.parseFromString(str_value);
return Field(typename NearestFieldType<FieldType>::Type(value));
return Field(typename NearestFieldType<FieldType>::Type(value, type.getScale()));
}

View File

@ -2,6 +2,8 @@
84 0 1764 1
84 0 1764 1
84.840 0.000 1799.456400 1.000
84.840000000 0.000000000
84.840000000000000000 0.000000000000000000
84.84 0.00 1799.4564 1.00
63 21 -42 882 -882 2 0
63 21 -42 882 -882 2 0
@ -23,3 +25,7 @@
42 42 42 0.420000000 0.420000000000000000 0.42000000000000000000000000000000000000 42.420 42.420000000 42.42
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.000 0.000000000 0.00
42 42 42 0.420000000 0.420000000000000000 0.42000000000000000000000000000000000000 42.420 42.420000000 42.42
1 1
1 1
1 0 1 0
1 0 1 0

View File

@ -29,8 +29,9 @@ SELECT e + e, e - e, e * e, e / e FROM test.decimal WHERE e > 0; -- { serverErro
SELECT f + f, f - f, f * f, f / f FROM test.decimal WHERE f > 0; -- { serverError 69 }
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 1 LIMIT 0;
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 FROM test.decimal WHERE i > 0;
SELECT j + j, j - j, j * j, j / j FROM test.decimal WHERE j > 0;
SELECT a + 21, a - 21, a - 84, a * 21, a * -21, a / 21, a / 84 FROM test.decimal WHERE a = 42;
@ -51,11 +52,20 @@ SELECT 21 + e, 21 - e, 84 - e, 21 * e, -21 * e, 21 / e, 84 / e FROM test.decimal
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 + 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 FROM test.decimal WHERE h > 0; --overflow 21 / h, 84 / h
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;
SELECT 21 + i, 21 - i, 84 - i, 21 * i, -21 * i, 21 / i, 84 / i FROM test.decimal WHERE i > 0;
SELECT 21 + j, 21 - j, 84 - j, 21 * j, -21 * j, 21 / j, 84 / j FROM test.decimal WHERE j > 0;
SELECT a, -a, -b, -c, -d, -e, -f, -g, -h, -j from test.decimal ORDER BY a;
SELECT abs(a), abs(b), abs(c), abs(d), abs(e), abs(f), abs(g), abs(h), abs(j) from test.decimal ORDER BY a;
SET decimal_check_arithmetic_overflow = 0;
SELECT (h * h) != 0, (h / h) != 1 FROM test.decimal WHERE h > 0;
SELECT (i * i) != 0, (i / i) = 1 FROM test.decimal WHERE i > 0;
SELECT e + 1 > e, e + 10 > e, 1 + e > e, 10 + e > e FROM test.decimal WHERE e > 0;
SELECT f + 1 > f, f + 10 > f, 1 + f > f, 10 + f > f FROM test.decimal WHERE f > 0;
DROP TABLE IF EXISTS test.decimal;

View File

@ -0,0 +1,125 @@
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
10 10.00000000 -10.00000000 10.00000000 -10.00000000
1 1.000000000 -1.000000000 1.000000000 -1.000000000
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
1000000000 1000000000.000000000 -1000000000.000000000
100000000 100000000.0000000000 -100000000.0000000000
10000000 10000000.00000000000 -10000000.00000000000
1000000 1000000.000000000000 -1000000.000000000000
100000 100000.0000000000000 -100000.0000000000000
10000 10000.00000000000000 -10000.00000000000000
1000 1000.000000000000000 -1000.000000000000000
100 100.0000000000000000 -100.0000000000000000
10 10.00000000000000000 -10.00000000000000000
1 1.000000000000000000 -1.000000000000000000
1000000000000000000 1000000000000000000 -1000000000000000000
100000000000000000 100000000000000000.0 -100000000000000000.0
10000000000000000 10000000000000000.00 -10000000000000000.00
1000000000000000 1000000000000000.000 -1000000000000000.000
100000000000000 100000000000000.0000 -100000000000000.0000
10000000000000 10000000000000.00000 -10000000000000.00000
1000000000000 1000000000000.000000 -1000000000000.000000
100000000000 100000000000.0000000 -100000000000.0000000
10000000000 10000000000.00000000 -10000000000.00000000
1000000000 1000000000.000000000 -1000000000.000000000
1000000000 1000000000.000000000 -1000000000.000000000
100000000 100000000.0000000000 -100000000.0000000000
10000000 10000000.00000000000 -10000000.00000000000
1000000 1000000.000000000000 -1000000.000000000000
100000 100000.0000000000000 -100000.0000000000000
10000 10000.00000000000000 -10000.00000000000000
1000 1000.000000000000000 -1000.000000000000000
100 100.0000000000000000 -100.0000000000000000
10 10.00000000000000000 -10.00000000000000000
1 1.000000000000000000 -1.000000000000000000
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
999999999 999999999.000000000 -999999999.000000000 999999999.00000000000000000000000000000 -999999999.00000000000000000000000000000
999999999 999999999.000000000 -999999999.000000000 999999999.00000000000000000000000000000 -999999999.00000000000000000000000000000
999999999999999999 999999999999999999 -999999999999999999
999999999999999999 999999999999999999 -999999999999999999
999999999999999999 999999999999999999.00000000000000000000 -999999999999999999.00000000000000000000
99 99 99
9999 9999 9999
999999999 999999999 999999999
999999999 999999999 999999999

View File

@ -0,0 +1,89 @@
SET allow_experimental_decimal_type = 1;
SET send_logs_level = 'none';
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);
SELECT toFloat32(9999999) as x, toDecimal32(x, 0), toDecimal32(-x, 0), toDecimal64(x, 0), toDecimal64(-x, 0);
SELECT toFloat32(999999.9) as x, toDecimal32(x, 1), toDecimal32(-x, 1), toDecimal64(x, 1), toDecimal64(-x, 1);
SELECT toFloat32(99999.99) as x, toDecimal32(x, 2), toDecimal32(-x, 2), toDecimal64(x, 2), toDecimal64(-x, 2);
SELECT toFloat32(9999.999) as x, toDecimal32(x, 3), toDecimal32(-x, 3), toDecimal64(x, 3), toDecimal64(-x, 3);
SELECT toFloat32(999.9999) as x, toDecimal32(x, 4), toDecimal32(-x, 4), toDecimal64(x, 4), toDecimal64(-x, 4);
SELECT toFloat32(99.99999) as x, toDecimal32(x, 5), toDecimal32(-x, 5), toDecimal64(x, 5), toDecimal64(-x, 5);
SELECT toFloat32(9.999999) as x, toDecimal32(x, 6), toDecimal32(-x, 6), toDecimal64(x, 6), toDecimal64(-x, 6);
SELECT toFloat32(0.9999999) as x, toDecimal32(x, 7), toDecimal32(-x, 7), toDecimal64(x, 7), toDecimal64(-x, 7);
SELECT toFloat32(9.99999999) as x, toDecimal32(x, 8), toDecimal32(-x, 8), toDecimal64(x, 8), toDecimal64(-x, 8);
SELECT toFloat32(0.999999999) as x, toDecimal32(x, 9), toDecimal32(-x, 9), toDecimal64(x, 9), toDecimal64(-x, 9);
SELECT toFloat64(999999999) as x, toDecimal32(x, 0), toDecimal32(-x, 0), toDecimal64(x, 0), toDecimal64(-x, 0);
SELECT toFloat64(99999999.9) as x, toDecimal32(x, 1), toDecimal32(-x, 1), toDecimal64(x, 1), toDecimal64(-x, 1);
SELECT toFloat64(9999999.99) as x, toDecimal32(x, 2), toDecimal32(-x, 2), toDecimal64(x, 2), toDecimal64(-x, 2);
SELECT toFloat64(999999.999) as x, toDecimal32(x, 3), toDecimal32(-x, 3), toDecimal64(x, 3), toDecimal64(-x, 3);
SELECT toFloat64(99999.9999) as x, toDecimal32(x, 4), toDecimal32(-x, 4), toDecimal64(x, 4), toDecimal64(-x, 4);
SELECT toFloat64(9999.99999) as x, toDecimal32(x, 5), toDecimal32(-x, 5), toDecimal64(x, 5), toDecimal64(-x, 5);
SELECT toFloat64(999.999999) as x, toDecimal32(x, 6), toDecimal32(-x, 6), toDecimal64(x, 6), toDecimal64(-x, 6);
SELECT toFloat64(99.9999999) as x, toDecimal32(x, 7), toDecimal32(-x, 7), toDecimal64(x, 7), toDecimal64(-x, 7);
SELECT toFloat64(9.99999999) as x, toDecimal32(x, 8), toDecimal32(-x, 8), toDecimal64(x, 8), toDecimal64(-x, 8);
SELECT toFloat64(0.999999999) as x, toDecimal32(x, 9), toDecimal32(-x, 9), toDecimal64(x, 9), toDecimal64(-x, 9);
SELECT toFloat64(999999999.999999999) as x, toDecimal64(x, 9), toDecimal64(-x, 9);
SELECT toFloat64(99999999.9999999999) as x, toDecimal64(x, 10), toDecimal64(-x, 10);
SELECT toFloat64(9999999.99999999999) as x, toDecimal64(x, 11), toDecimal64(-x, 11);
SELECT toFloat64(999999.999999999999) as x, toDecimal64(x, 12), toDecimal64(-x, 12);
SELECT toFloat64(99999.9999999999999) as x, toDecimal64(x, 13), toDecimal64(-x, 13);
SELECT toFloat64(9999.99999999999999) as x, toDecimal64(x, 14), toDecimal64(-x, 14);
SELECT toFloat64(999.999999999999999) as x, toDecimal64(x, 15), toDecimal64(-x, 15);
SELECT toFloat64(99.9999999999999999) as x, toDecimal64(x, 16), toDecimal64(-x, 16);
SELECT toFloat64(9.99999999999999999) as x, toDecimal64(x, 17), toDecimal64(-x, 17);
SELECT toFloat64(0.999999999999999999) as x, toDecimal64(x, 18), toDecimal64(-x, 18);
SELECT toFloat64(999999999999999999) as x, toDecimal128(x, 0), toDecimal128(-x, 0);
SELECT toFloat64(99999999999999999.9) as x, toDecimal128(x, 1), toDecimal128(-x, 1);
SELECT toFloat64(9999999999999999.99) as x, toDecimal128(x, 2), toDecimal128(-x, 2);
SELECT toFloat64(999999999999999.999) as x, toDecimal128(x, 3), toDecimal128(-x, 3);
SELECT toFloat64(99999999999999.9999) as x, toDecimal128(x, 4), toDecimal128(-x, 4);
SELECT toFloat64(9999999999999.99999) as x, toDecimal128(x, 5), toDecimal128(-x, 5);
SELECT toFloat64(999999999999.999999) as x, toDecimal128(x, 6), toDecimal128(-x, 6);
SELECT toFloat64(99999999999.9999999) as x, toDecimal128(x, 7), toDecimal128(-x, 7);
SELECT toFloat64(9999999999.99999999) as x, toDecimal128(x, 8), toDecimal128(-x, 8);
SELECT toFloat64(999999999.999999999) as x, toDecimal128(x, 9), toDecimal128(-x, 9);
SELECT toFloat64(999999999.999999999) as x, toDecimal128(x, 9), toDecimal128(-x, 9);
SELECT toFloat64(99999999.9999999999) as x, toDecimal128(x, 10), toDecimal128(-x, 10);
SELECT toFloat64(9999999.99999999999) as x, toDecimal128(x, 11), toDecimal128(-x, 11);
SELECT toFloat64(999999.999999999999) as x, toDecimal128(x, 12), toDecimal128(-x, 12);
SELECT toFloat64(99999.9999999999999) as x, toDecimal128(x, 13), toDecimal128(-x, 13);
SELECT toFloat64(9999.99999999999999) as x, toDecimal128(x, 14), toDecimal128(-x, 14);
SELECT toFloat64(999.999999999999999) as x, toDecimal128(x, 15), toDecimal128(-x, 15);
SELECT toFloat64(99.9999999999999999) as x, toDecimal128(x, 16), toDecimal128(-x, 16);
SELECT toFloat64(9.99999999999999999) as x, toDecimal128(x, 17), toDecimal128(-x, 17);
SELECT toFloat64(0.999999999999999999) as x, toDecimal128(x, 18), toDecimal128(-x, 18);
SELECT toDecimal32(number, 4) as n1, toDecimal32(n1 / 9, 2) as n2, toDecimal32(n2, 8) FROM system.numbers LIMIT 10;
SELECT toDecimal32(number, 4) as n1, toDecimal32(n1 / 9, 8) as n2, toDecimal32(n2, 2) FROM system.numbers LIMIT 10;
SELECT toDecimal32(number, 8) as n1, toDecimal32(n1 / 9, 4) as n2, toDecimal32(n2, 2) FROM system.numbers LIMIT 10;
SELECT toDecimal64(number, 4) as n1, toDecimal64(n1 / 9, 2) as n2, toDecimal64(n2, 8) FROM system.numbers LIMIT 10;
SELECT toDecimal64(number, 4) as n1, toDecimal64(n1 / 9, 8) as n2, toDecimal64(n2, 2) FROM system.numbers LIMIT 10;
SELECT toDecimal64(number, 8) as n1, toDecimal64(n1 / 9, 4) as n2, toDecimal64(n2, 2) FROM system.numbers LIMIT 10;
SELECT toInt8(99) as x, toDecimal32(x, 0), toDecimal32(-x, 0), toDecimal64(x, 0), toDecimal64(-x, 0);
SELECT toInt16(9999) as x, toDecimal32(x, 0), toDecimal32(-x, 0), toDecimal64(x, 0), toDecimal64(-x, 0);
SELECT toInt32(999999999) as x, toDecimal32(x, 0), toDecimal32(-x, 0), toDecimal64(x, 0), toDecimal64(-x, 0);
SELECT toInt64(999999999) as x, toDecimal32(x, 0), toDecimal32(-x, 0), toDecimal64(x, 0), toDecimal64(-x, 0);
SELECT toInt32(999999999) as x, toDecimal64(x, 9), toDecimal64(-x, 9), toDecimal128(x, 29), toDecimal128(-x, 29);
SELECT toInt64(999999999) as x, toDecimal64(x, 9), toDecimal64(-x, 9), toDecimal128(x, 29), toDecimal128(-x, 29);
SELECT toInt64(999999999999999999) as x, toDecimal64(x, 0), toDecimal64(-x, 0);
SELECT toInt64(999999999999999999) as x, toDecimal128(x, 0), toDecimal128(-x, 0);
SELECT toInt64(999999999999999999) as x, toDecimal128(x, 20), toDecimal128(-x, 20);
SELECT toUInt8(99) as x, toDecimal32(x, 0), toDecimal64(x, 0);
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 * FROM test.decimal;
--DROP TABLE IF EXISTS test.decimal;

View File

@ -10,11 +10,11 @@ CREATE TABLE IF NOT EXISTS test.decimal
b DECIMAL(18,0),
c DECIMAL(38,0),
d DECIMAL(9, 9),
e DEC(18, 18),
f dec(38, 38),
g Decimal(9, 5),
h decimal(18, 9),
i deciMAL(38, 18),
e Decimal64(18),
f Decimal128(38),
g Decimal32(5),
h Decimal64(9),
i Decimal128(18),
j dec(4,2)
) ENGINE = Memory;

View File

@ -0,0 +1,29 @@
1
0
0
0
1
1
5
9.00000000 29.00000000 29.00000000
8.00000000 28.00000000 28.00000000
7.00000000 27.00000000 27.00000000
6.00000000 26.00000000 26.00000000
9.00000000 19.00000000 19.00000000
8.00000000 18.00000000 18.00000000
7.00000000 17.00000000 17.00000000
6.00000000 16.00000000 16.00000000
9.00000000 9.00000000 9.00000000
8.00000000 8.00000000 8.00000000
7.00000000 7.00000000 7.00000000
6.00000000 6.00000000 6.00000000
1.00000000 1.00000000 1.00000000
3.00000000 3.00000000 3.00000000
1.00000000 11.00000000 11.00000000
3.00000000 13.00000000 13.00000000
1.00000000 21.00000000 21.00000000
3.00000000 23.00000000 23.00000000
1.00000000 31.00000000 31.00000000
3.00000000 33.00000000 33.00000000
1.00000000 41.00000000 41.00000000
3.00000000 43.00000000 43.00000000

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
(
d1 DECIMAL(9, 8),
d2 DECIMAL(18, 8),
d3 DECIMAL(38, 8)
)
ENGINE = MergeTree
PARTITION BY toInt32(d1)
ORDER BY (d2, d3);
INSERT INTO test.decimal (d1, d2, d3) VALUES (4.2, 4.2, 4.2);
SELECT count() FROM test.decimal WHERE d1 = toDecimal32('4.2', 1);
SELECT count() FROM test.decimal WHERE d1 != toDecimal32('4.2', 2);
SELECT count() FROM test.decimal WHERE d1 < toDecimal32('4.2', 3);
SELECT count() FROM test.decimal WHERE d1 > toDecimal32('4.2', 4);
SELECT count() FROM test.decimal WHERE d1 <= toDecimal32('4.2', 5);
SELECT count() FROM test.decimal WHERE d1 >= toDecimal32('4.2', 6);
INSERT INTO test.decimal (d1, d2, d3)
SELECT toDecimal32(number % 10, 8), toDecimal64(number, 8), toDecimal128(number, 8) FROM system.numbers LIMIT 50;
SELECT count() FROM test.decimal WHERE d1 = 1;
SELECT * FROM test.decimal WHERE d1 > 5 AND d2 < 30 ORDER BY d2 DESC;
SELECT * FROM test.decimal WHERE d1 IN(1, 3) ORDER BY d2;
DROP TABLE test.decimal;

View File

@ -0,0 +1,97 @@
#pragma once
namespace common
{
template <typename T>
inline bool addOverflow(T x, T y, T & res)
{
return __builtin_add_overflow(x, y, &res);
}
template <>
inline bool addOverflow(int x, int y, int & res)
{
return __builtin_sadd_overflow(x, y, &res);
}
template <>
inline bool addOverflow(long x, long y, long & res)
{
return __builtin_saddl_overflow(x, y, &res);
}
template <>
inline bool addOverflow(long long x, long long y, long long & res)
{
return __builtin_saddll_overflow(x, y, &res);
}
template <>
inline bool addOverflow(__int128 x, __int128 y, __int128 & res)
{
res = x + y;
return (res - y) != x;
}
template <typename T>
inline bool subOverflow(T x, T y, T & res)
{
return __builtin_sub_overflow(x, y, &res);
}
template <>
inline bool subOverflow(int x, int y, int & res)
{
return __builtin_ssub_overflow(x, y, &res);
}
template <>
inline bool subOverflow(long x, long y, long & res)
{
return __builtin_ssubl_overflow(x, y, &res);
}
template <>
inline bool subOverflow(long long x, long long y, long long & res)
{
return __builtin_ssubll_overflow(x, y, &res);
}
template <>
inline bool subOverflow(__int128 x, __int128 y, __int128 & res)
{
res = x - y;
return (res + y) != x;
}
template <typename T>
inline bool mulOverflow(T x, T y, T & res)
{
return __builtin_mul_overflow(x, y, &res);
}
template <>
inline bool mulOverflow(int x, int y, int & res)
{
return __builtin_smul_overflow(x, y, &res);
}
template <>
inline bool mulOverflow(long x, long y, long & res)
{
return __builtin_smull_overflow(x, y, &res);
}
template <>
inline bool mulOverflow(long long x, long long y, long long & res)
{
return __builtin_smulll_overflow(x, y, &res);
}
template <>
inline bool mulOverflow(__int128 x, __int128 y, __int128 & res)
{
res = x * y;
return (res / y) != x;
}
}