Replacing IsDecimalNumber

This commit is contained in:
Mike Kot 2021-09-10 13:49:22 +02:00
parent c4385a2c40
commit bce011cb33
76 changed files with 573 additions and 515 deletions

View File

@ -1,17 +0,0 @@
#pragma once
#include <type_traits>
#include <concepts>
#include "extended_types.h"
namespace DB
{
// Concept for enums can't be named just "enum", so they all start with is_ for consistency
/// C++ integral types as stated in standard + wide integer types [U]Int[128, 256]
template <class T> concept is_integer = is_integer_v<T>;
template <class T> concept is_floating_point = std::is_floating_point_v<T>;
template <class T> concept is_enum = std::is_enum_v<T>;
}

161
base/common/Decimal.h Normal file
View File

@ -0,0 +1,161 @@
#pragma once
#include <concepts>
#include "common/extended_types.h"
#include "common/types.h"
#if !defined(NO_SANITIZE_UNDEFINED)
#if defined(__clang__)
#define NO_SANITIZE_UNDEFINED __attribute__((__no_sanitize__("undefined")))
#else
#define NO_SANITIZE_UNDEFINED
#endif
#endif
namespace DB
{
template <class> struct Decimal;
class DateTime64;
using Decimal32 = Decimal<Int32>;
using Decimal64 = Decimal<Int64>;
using Decimal128 = Decimal<Int128>;
using Decimal256 = Decimal<Int256>;
template <class T>
concept is_decimal =
std::same_as<T, Decimal32>
|| std::same_as<T, Decimal64>
|| std::same_as<T, Decimal128>
|| std::same_as<T, Decimal256>
|| std::same_as<T, DateTime64>;
template <class T>
concept is_over_big_int =
std::same_as<T, Int128>
|| std::same_as<T, UInt128>
|| std::same_as<T, Int256>
|| std::same_as<T, UInt256>
|| std::same_as<T, Decimal128>
|| std::same_as<T, Decimal256>;
template <class T> struct NativeTypeT { using Type = T; };
template <is_decimal T> struct NativeTypeT<T> { using Type = typename T::NativeType; };
template <class T> using NativeType = typename NativeTypeT<T>::Type;
/// Own FieldType for Decimal.
/// It is only a "storage" for decimal.
/// To perform operations, you also have to provide a scale (number of digits after point).
template <typename T>
struct Decimal
{
using NativeType = T;
constexpr Decimal() = default;
constexpr Decimal(Decimal<T> &&) = default;
constexpr Decimal(const Decimal<T> &) = default;
constexpr Decimal(const T & value_): value(value_) {}
template <typename U>
constexpr Decimal(const Decimal<U> & x): value(x.value) {}
constexpr Decimal<T> & operator = (Decimal<T> &&) = default;
constexpr Decimal<T> & operator = (const Decimal<T> &) = default;
constexpr operator T () const { return value; }
template <typename U>
constexpr U convertTo() const
{
if constexpr(is_decimal<U>)
return convertTo<typename U::NativeType>();
else
return static_cast<U>(value);
}
const Decimal<T> & operator += (const T & x) { value += x; return *this; }
const Decimal<T> & operator -= (const T & x) { value -= x; return *this; }
const Decimal<T> & operator *= (const T & x) { value *= x; return *this; }
const Decimal<T> & operator /= (const T & x) { value /= x; return *this; }
const Decimal<T> & operator %= (const T & x) { value %= x; return *this; }
template <typename U> const Decimal<T> & operator += (const Decimal<U> & x) { value += x.value; return *this; }
template <typename U> const Decimal<T> & operator -= (const Decimal<U> & x) { value -= x.value; return *this; }
template <typename U> const Decimal<T> & operator *= (const Decimal<U> & x) { value *= x.value; return *this; }
template <typename U> const Decimal<T> & operator /= (const Decimal<U> & x) { value /= x.value; return *this; }
template <typename U> const Decimal<T> & operator %= (const Decimal<U> & x) { value %= x.value; return *this; }
/// This is to avoid UB for sumWithOverflow()
void NO_SANITIZE_UNDEFINED addOverflow(const T & x) { value += x; }
T value;
};
template <typename T> inline bool operator< (const Decimal<T> & x, const Decimal<T> & y) { return x.value < y.value; }
template <typename T> inline bool operator> (const Decimal<T> & x, const Decimal<T> & y) { return x.value > y.value; }
template <typename T> inline bool operator<= (const Decimal<T> & x, const Decimal<T> & y) { return x.value <= y.value; }
template <typename T> inline bool operator>= (const Decimal<T> & x, const Decimal<T> & y) { return x.value >= y.value; }
template <typename T> inline bool operator== (const Decimal<T> & x, const Decimal<T> & y) { return x.value == y.value; }
template <typename T> inline bool operator!= (const Decimal<T> & x, const Decimal<T> & y) { return x.value != y.value; }
template <typename T> inline Decimal<T> operator+ (const Decimal<T> & x, const Decimal<T> & y) { return x.value + y.value; }
template <typename T> inline Decimal<T> operator- (const Decimal<T> & x, const Decimal<T> & y) { return x.value - y.value; }
template <typename T> inline Decimal<T> operator* (const Decimal<T> & x, const Decimal<T> & y) { return x.value * y.value; }
template <typename T> inline Decimal<T> operator/ (const Decimal<T> & x, const Decimal<T> & y) { return x.value / y.value; }
template <typename T> inline Decimal<T> operator- (const Decimal<T> & x) { return -x.value; }
// Distinguishable type to allow function resolution/deduction based on value type,
// but also relatively easy to convert to/from Decimal64.
class DateTime64 : public Decimal64
{
public:
using Base = Decimal64;
using Base::Base;
using NativeType = Base::NativeType;
constexpr DateTime64(const Base & v): Base(v) {}
};
}
constexpr DB::UInt64 max_uint_mask = std::numeric_limits<DB::UInt64>::max();
namespace std
{
template <typename T>
struct hash<DB::Decimal<T>>
{
size_t operator()(const DB::Decimal<T> & x) const { return hash<T>()(x.value); }
};
template <>
struct hash<DB::Decimal128>
{
size_t operator()(const DB::Decimal128 & x) const
{
return std::hash<DB::Int64>()(x.value >> 64)
^ std::hash<DB::Int64>()(x.value & max_uint_mask);
}
};
template <>
struct hash<DB::DateTime64>
{
size_t operator()(const DB::DateTime64 & x) const
{
return std::hash<DB::DateTime64::NativeType>()(x);
}
};
template <>
struct hash<DB::Decimal256>
{
size_t operator()(const DB::Decimal256 & x) const
{
// FIXME temp solution
return std::hash<DB::Int64>()(static_cast<DB::Int64>(x.value >> 64 & max_uint_mask))
^ std::hash<DB::Int64>()(static_cast<DB::Int64>(x.value & max_uint_mask));
}
};
}

View File

@ -1,8 +1,45 @@
#pragma once
#include <type_traits>
#include <magic_enum.hpp>
#include <fmt/format.h>
#include "Concepts.h"
namespace DB { template <class T> concept is_enum = std::is_enum_v<T>; }
//template <DB::is_enum E, class F, size_t I1, size_t ...I>
//constexpr void static_for(F && f, std::index_sequence<I...>)
//{
// constexpr auto constant = std::integral_constant<E, magic_enum::enum_value<E>(I1)>();
//
// if (!std::forward<F>(f)(constant))
// return;
//
// if constexpr (sizeof...(I) > 0)
// static_for<E>(std::forward<F>(f), std::index_sequence<I...>());
//}
//}
namespace detail
{
template <DB::is_enum E, class F, size_t ...I>
constexpr void static_for(F && f, std::index_sequence<I...>)
{
(std::forward<F>(f)(std::integral_constant<E, magic_enum::enum_value<E>(I)>()) , ...);
}
}
/**
* Iterate over enum values in compile-time (compile-time switch/case, loop unrolling).
*
* @example static_for<E>([](auto enum_value) { return template_func<enum_value>(); }
* ^ enum_value can be used as a template parameter
*/
template <DB::is_enum E, class F>
constexpr void static_for(F && f)
{
constexpr size_t count = magic_enum::enum_count<E>();
detail::static_for<E>(std::forward<F>(f), std::make_index_sequence<count>());
}
/// Enable printing enum values as strings via fmt + magic_enum
template <DB::is_enum T>

View File

@ -41,22 +41,17 @@ template <> struct is_unsigned<UInt256> { static constexpr bool value = true; };
template <typename T>
inline constexpr bool is_unsigned_v = is_unsigned<T>::value;
/// TODO: is_integral includes char, char8_t and wchar_t.
template <typename T>
struct is_integer
namespace DB
{
static constexpr bool value = std::is_integral_v<T>;
};
template <> struct is_integer<Int128> { static constexpr bool value = true; };
template <> struct is_integer<UInt128> { static constexpr bool value = true; };
template <> struct is_integer<Int256> { static constexpr bool value = true; };
template <> struct is_integer<UInt256> { static constexpr bool value = true; };
template <typename T>
inline constexpr bool is_integer_v = is_integer<T>::value;
template <class T> concept is_integer =
std::is_integral_v<T>
|| std::is_same_v<T, Int128>
|| std::is_same_v<T, UInt128>
|| std::is_same_v<T, Int256>
|| std::is_same_v<T, UInt256>;
template <class T> concept is_floating_point = std::is_floating_point_v<T>;
}
template <typename T>
struct is_arithmetic

View File

@ -14,16 +14,18 @@ private:
public:
using UnderlyingType = T;
template <class Enable = typename std::is_copy_constructible<T>::type>
explicit StrongTypedef(const T & t_) : t(t_) {}
constexpr explicit StrongTypedef(const T & t_) : t(t_) {}
template <class Enable = typename std::is_move_constructible<T>::type>
explicit StrongTypedef(T && t_) : t(std::move(t_)) {}
constexpr explicit StrongTypedef(T && t_) : t(std::move(t_)) {}
template <class Enable = typename std::is_default_constructible<T>::type>
StrongTypedef(): t() {}
constexpr StrongTypedef(): t() {}
StrongTypedef(const Self &) = default;
StrongTypedef(Self &&) = default;
constexpr StrongTypedef(const Self &) = default;
constexpr StrongTypedef(Self &&) = default;
Self & operator=(const Self &) = default;
Self & operator=(Self &&) = default;

View File

@ -21,11 +21,9 @@
namespace DB
{
struct Settings;
template <typename T>
using DecimalOrVectorCol = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
template <typename T> constexpr bool DecimalOrExtendedInt =
IsDecimalNumber<T>
is_decimal<T>
|| std::is_same_v<T, Int128>
|| std::is_same_v<T, Int256>
|| std::is_same_v<T, UInt128>
@ -44,7 +42,7 @@ struct AvgFraction
/// Invoked only is either Numerator or Denominator are Decimal.
Float64 NO_SANITIZE_UNDEFINED divideIfAnyDecimal(UInt32 num_scale, UInt32 denom_scale [[maybe_unused]]) const
{
if constexpr (IsDecimalNumber<Numerator> && IsDecimalNumber<Denominator>)
if constexpr (is_decimal<Numerator> && is_decimal<Denominator>)
{
// According to the docs, num(S1) / denom(S2) would have scale S1
@ -60,7 +58,7 @@ struct AvgFraction
/// Numerator is always casted to Float64 to divide correctly if the denominator is not Float64.
Float64 num_converted;
if constexpr (IsDecimalNumber<Numerator>)
if constexpr (is_decimal<Numerator>)
num_converted = DecimalUtils::convertTo<Float64>(numerator, num_scale);
else
num_converted = static_cast<Float64>(numerator); /// all other types, including extended integral.
@ -68,7 +66,7 @@ struct AvgFraction
std::conditional_t<DecimalOrExtendedInt<Denominator>,
Float64, Denominator> denom_converted;
if constexpr (IsDecimalNumber<Denominator>)
if constexpr (is_decimal<Denominator>)
denom_converted = DecimalUtils::convertTo<Float64>(denominator, denom_scale);
else if constexpr (DecimalOrExtendedInt<Denominator>)
/// no way to divide Float64 and extended integral type without an explicit cast.
@ -139,7 +137,7 @@ public:
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
if constexpr (IsDecimalNumber<Numerator> || IsDecimalNumber<Denominator>)
if constexpr (is_decimal<Numerator> || is_decimal<Denominator>)
assert_cast<ColumnVector<Float64> &>(to).getData().push_back(
this->data(place).divideIfAnyDecimal(num_scale, denom_scale));
else
@ -222,7 +220,7 @@ private:
};
template <typename T>
using AvgFieldType = std::conditional_t<IsDecimalNumber<T>,
using AvgFieldType = std::conditional_t<is_decimal<T>,
std::conditional_t<std::is_same_v<T, Decimal256>, Decimal256, Decimal128>,
NearestFieldType<T>>;
@ -239,7 +237,7 @@ public:
void NO_SANITIZE_UNDEFINED add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const final
{
this->data(place).numerator += static_cast<const DecimalOrVectorCol<T> &>(*columns[0]).getData()[row_num];
this->data(place).numerator += static_cast<const ColumnVectorOrDecimal<T> &>(*columns[0]).getData()[row_num];
++this->data(place).denominator;
}

View File

@ -8,7 +8,7 @@ namespace DB
struct Settings;
template <typename T>
using AvgWeightedFieldType = std::conditional_t<IsDecimalNumber<T>,
using AvgWeightedFieldType = std::conditional_t<is_decimal<T>,
std::conditional_t<std::is_same_v<T, Decimal256>, Decimal256, Decimal128>,
std::conditional_t<DecimalOrExtendedInt<T>,
Float64, // no way to do UInt128 * UInt128, better cast to Float64

View File

@ -25,14 +25,14 @@ namespace
template <typename T, typename LimitNumberOfElements>
struct MovingSum
{
using Data = MovingSumData<std::conditional_t<IsDecimalNumber<T>, Decimal128, NearestFieldType<T>>>;
using Data = MovingSumData<std::conditional_t<is_decimal<T>, Decimal128, NearestFieldType<T>>>;
using Function = MovingImpl<T, LimitNumberOfElements, Data>;
};
template <typename T, typename LimitNumberOfElements>
struct MovingAvg
{
using Data = MovingAvgData<std::conditional_t<IsDecimalNumber<T>, Decimal128, Float64>>;
using Data = MovingAvgData<std::conditional_t<is_decimal<T>, Decimal128, Float64>>;
using Function = MovingImpl<T, LimitNumberOfElements, Data>;
};

View File

@ -87,16 +87,12 @@ class MovingImpl final
public:
using ResultT = typename Data::Accumulator;
using ColumnSource = std::conditional_t<IsDecimalNumber<T>,
ColumnDecimal<T>,
ColumnVector<T>>;
using ColumnSource = ColumnVectorOrDecimal<T>;
/// Probably for overflow function in the future.
using ColumnResult = std::conditional_t<IsDecimalNumber<ResultT>,
ColumnDecimal<ResultT>,
ColumnVector<ResultT>>;
using ColumnResult = ColumnVectorOrDecimal<ResultT>;
using DataTypeResult = std::conditional_t<IsDecimalNumber<ResultT>,
using DataTypeResult = std::conditional_t<is_decimal<ResultT>,
DataTypeDecimal<ResultT>,
DataTypeNumber<ResultT>>;
@ -108,7 +104,7 @@ public:
DataTypePtr getReturnType() const override
{
if constexpr (IsDecimalNumber<ResultT>)
if constexpr (is_decimal<ResultT>)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeResult>(
DataTypeResult::maxPrecision(), getDecimalScale(*this->argument_types.at(0))));
else

View File

@ -44,7 +44,7 @@ struct SingleValueDataFixed
{
private:
using Self = SingleValueDataFixed;
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
using ColVecType = ColumnVectorOrDecimal<T>;
bool has_value = false; /// We need to remember if at least one value has been passed. This is necessary for AggregateFunctionIf.
T value;

View File

@ -67,10 +67,10 @@ class AggregateFunctionQuantile final : public IAggregateFunctionDataHelper<Data
AggregateFunctionQuantile<Value, Data, Name, has_second_arg, FloatReturnType, returns_many>>
{
private:
using ColVecType = std::conditional_t<IsDecimalNumber<Value>, ColumnDecimal<Value>, ColumnVector<Value>>;
using ColVecType = ColumnVectorOrDecimal<Value>;
static constexpr bool returns_float = !(std::is_same_v<FloatReturnType, void>);
static_assert(!IsDecimalNumber<Value> || !returns_float);
static_assert(!is_decimal<Value> || !returns_float);
QuantileLevels<Float64> levels;

View File

@ -49,7 +49,7 @@ struct StatFuncOneArg
using Type1 = T;
using Type2 = T;
using ResultType = std::conditional_t<std::is_same_v<T, Float32>, Float32, Float64>;
using Data = std::conditional_t<IsDecimalNumber<T>, VarMomentsDecimal<Decimal128, _level>, VarMoments<ResultType, _level>>;
using Data = std::conditional_t<is_decimal<T>, VarMomentsDecimal<Decimal128, _level>, VarMoments<ResultType, _level>>;
static constexpr StatisticsFunctionKind kind = _kind;
static constexpr UInt32 num_args = 1;
@ -75,8 +75,8 @@ class AggregateFunctionVarianceSimple final
public:
using T1 = typename StatFunc::Type1;
using T2 = typename StatFunc::Type2;
using ColVecT1 = std::conditional_t<IsDecimalNumber<T1>, ColumnDecimal<T1>, ColumnVector<T1>>;
using ColVecT2 = std::conditional_t<IsDecimalNumber<T2>, ColumnDecimal<T2>, ColumnVector<T2>>;
using ColVecT1 = ColumnVectorOrDecimal<T1>;
using ColVecT2 = ColumnVectorOrDecimal<T2>;
using ResultType = typename StatFunc::ResultType;
using ColVecResult = ColumnVector<ResultType>;
@ -132,7 +132,7 @@ public:
static_cast<ResultType>(static_cast<const ColVecT2 &>(*columns[1]).getData()[row_num]));
else
{
if constexpr (IsDecimalNumber<T1>)
if constexpr (is_decimal<T1>)
{
this->data(place).add(static_cast<ResultType>(
static_cast<const ColVecT1 &>(*columns[0]).getData()[row_num].value));
@ -163,7 +163,7 @@ public:
const auto & data = this->data(place);
auto & dst = static_cast<ColVecResult &>(to).getData();
if constexpr (IsDecimalNumber<T1>)
if constexpr (is_decimal<T1>)
{
if constexpr (StatFunc::kind == StatisticsFunctionKind::varPop)
dst.push_back(data.getPopulation(src_scale * 2));

View File

@ -20,10 +20,9 @@ template <typename T>
struct SumSimple
{
/// @note It uses slow Decimal128 (cause we need such a variant). sumWithOverflow is faster for Decimal32/64
using ResultType = std::conditional_t<IsDecimalNumber<T>,
using ResultType = std::conditional_t<is_decimal<T>,
std::conditional_t<std::is_same_v<T, Decimal256>, Decimal256, Decimal128>,
NearestFieldType<T>>;
// using ResultType = std::conditional_t<IsDecimalNumber<T>, Decimal128, NearestFieldType<T>>;
using AggregateDataType = AggregateFunctionSumData<ResultType>;
using Function = AggregateFunctionSum<T, ResultType, AggregateDataType, AggregateFunctionTypeSum>;
};
@ -47,7 +46,7 @@ struct SumKahan
template <typename T> using AggregateFunctionSumSimple = typename SumSimple<T>::Function;
template <typename T> using AggregateFunctionSumWithOverflow = typename SumSameType<T>::Function;
template <typename T> using AggregateFunctionSumKahan =
std::conditional_t<IsDecimalNumber<T>, typename SumSimple<T>::Function, typename SumKahan<T>::Function>;
std::conditional_t<is_decimal<T>, typename SumSimple<T>::Function, typename SumKahan<T>::Function>;
template <template <typename> class Function>

View File

@ -105,7 +105,7 @@ struct AggregateFunctionSumData
if constexpr (
(is_integer_v<T> && !is_big_int_v<T>)
|| (IsDecimalNumber<T> && !std::is_same_v<T, Decimal256> && !std::is_same_v<T, Decimal128>))
|| (is_decimal<T> && !std::is_same_v<T, Decimal256> && !std::is_same_v<T, Decimal128>))
{
/// For integers we can vectorize the operation if we replace the null check using a multiplication (by 0 for null, 1 for not null)
/// https://quick-bench.com/q/MLTnfTvwC2qZFVeWHfOBR3U7a8I
@ -334,9 +334,9 @@ class AggregateFunctionSum final : public IAggregateFunctionDataHelper<Data, Agg
public:
static constexpr bool DateTime64Supported = false;
using ResultDataType = std::conditional_t<IsDecimalNumber<T>, DataTypeDecimal<TResult>, DataTypeNumber<TResult>>;
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
using ColVecResult = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<TResult>, ColumnVector<TResult>>;
using ResultDataType = std::conditional_t<is_decimal<T>, DataTypeDecimal<TResult>, DataTypeNumber<TResult>>;
using ColVecType = ColumnVectorOrDecimal<T>;
using ColVecResult = std::conditional_t<is_decimal<T>, ColumnDecimal<TResult>, ColumnVector<TResult>>;
String getName() const override
{
@ -361,7 +361,7 @@ public:
DataTypePtr getReturnType() const override
{
if constexpr (IsDecimalNumber<T>)
if constexpr (is_decimal<T>)
return std::make_shared<ResultDataType>(ResultDataType::maxPrecision(), scale);
else
return std::make_shared<ResultDataType>();

View File

@ -8,7 +8,10 @@
namespace DB
{
template <typename T>
using DecimalOrNumberDataType = std::conditional_t<IsDecimalNumber<T>, DataTypeDecimal<AvgFieldType<T>>, DataTypeNumber<AvgFieldType<T>>>;
using DecimalOrNumberDataType = std::conditional_t<is_decimal<T>,
DataTypeDecimal<AvgFieldType<T>>,
DataTypeNumber<AvgFieldType<T>>>;
template <typename T>
class AggregateFunctionSumCount final : public AggregateFunctionAvgBase<AvgFieldType<T>, UInt64, AggregateFunctionSumCount<T>>
{
@ -21,7 +24,7 @@ public:
DataTypePtr getReturnType() const override
{
DataTypes types;
if constexpr (IsDecimalNumber<T>)
if constexpr (is_decimal<T>)
types.emplace_back(std::make_shared<DecimalOrNumberDataType<T>>(DecimalOrNumberDataType<T>::maxPrecision(), scale));
else
types.emplace_back(std::make_shared<DecimalOrNumberDataType<T>>());

View File

@ -190,7 +190,7 @@ public:
continue;
decltype(merged_maps.begin()) it;
if constexpr (IsDecimalNumber<T>)
if constexpr (is_decimal<T>)
{
// FIXME why is storing NearestFieldType not enough, and we
// have to check for decimals again here?
@ -217,7 +217,7 @@ public:
new_values.resize(size);
new_values[col] = value;
if constexpr (IsDecimalNumber<T>)
if constexpr (is_decimal<T>)
{
UInt32 scale = static_cast<const ColumnDecimal<T> &>(key_column).getData().getScale();
merged_maps.emplace(DecimalField<T>(key, scale), std::move(new_values));
@ -280,7 +280,7 @@ public:
for (size_t col = 0; col < values_types.size(); ++col)
values_serializations[col]->deserializeBinary(values[col], buf);
if constexpr (IsDecimalNumber<T>)
if constexpr (is_decimal<T>)
merged_maps[key.get<DecimalField<T>>()] = values;
else
merged_maps[key.get<T>()] = values;
@ -396,7 +396,7 @@ private:
using Base = AggregateFunctionMapBase<T, Self, FieldVisitorSum, overflow, tuple_argument, true>;
/// ARCADIA_BUILD disallow unordered_set for big ints for some reason
static constexpr const bool allow_hash = !OverBigInt<T>;
static constexpr const bool allow_hash = !is_over_big_int<T>;
using ContainerT = std::conditional_t<allow_hash, std::unordered_set<T>, std::set<T>>;
ContainerT keys_to_keep;

View File

@ -30,7 +30,7 @@ struct QuantileExactWeighted
};
using Weight = UInt64;
using UnderlyingType = typename NativeType<Value>::Type;
using UnderlyingType = NativeType<Value>;
using Hasher = std::conditional_t<std::is_same_v<Value, Decimal128>, Int128Hash, HashCRC32<UnderlyingType>>;
/// When creating, the hash table must be small.

View File

@ -121,7 +121,7 @@ public:
{
if (samples.empty())
{
if (DB::IsDecimalNumber<T>)
if (DB::is_decimal<T>)
return 0;
return onEmpty<double>();
}
@ -134,7 +134,7 @@ public:
size_t right_index = left_index + 1;
if (right_index == samples.size())
{
if constexpr (DB::IsDecimalNumber<T>)
if constexpr (DB::is_decimal<T>)
return static_cast<double>(samples[left_index].value);
else
return static_cast<double>(samples[left_index]);
@ -143,7 +143,7 @@ public:
double left_coef = right_index - index;
double right_coef = index - left_index;
if constexpr (DB::IsDecimalNumber<T>)
if constexpr (DB::is_decimal<T>)
return static_cast<double>(samples[left_index].value) * left_coef + static_cast<double>(samples[right_index].value) * right_coef;
else
return static_cast<double>(samples[left_index]) * left_coef + static_cast<double>(samples[right_index]) * right_coef;

View File

@ -19,7 +19,7 @@
#include <DataStreams/ColumnGathererStream.h>
template <typename T> bool decimalLess(T x, T y, UInt32 x_scale, UInt32 y_scale);
template <class T> bool decimalLess(T x, T y, UInt32 x_scale, UInt32 y_scale);
namespace DB
{
@ -38,7 +38,7 @@ template class DecimalPaddedPODArray<Decimal128>;
template class DecimalPaddedPODArray<Decimal256>;
template class DecimalPaddedPODArray<DateTime64>;
template <typename T>
template <is_decimal T>
int ColumnDecimal<T>::compareAt(size_t n, size_t m, const IColumn & rhs_, int) const
{
auto & other = static_cast<const Self &>(rhs_);
@ -50,7 +50,7 @@ int ColumnDecimal<T>::compareAt(size_t n, size_t m, const IColumn & rhs_, int) c
return decimalLess<T>(b, a, other.scale, scale) ? 1 : (decimalLess<T>(a, b, scale, other.scale) ? -1 : 0);
}
template <typename T>
template <is_decimal T>
void ColumnDecimal<T>::compareColumn(const IColumn & rhs, size_t rhs_row_num,
PaddedPODArray<UInt64> * row_indexes, PaddedPODArray<Int8> & compare_results,
int direction, int nan_direction_hint) const
@ -59,13 +59,13 @@ void ColumnDecimal<T>::compareColumn(const IColumn & rhs, size_t rhs_row_num,
compare_results, direction, nan_direction_hint);
}
template <typename T>
template <is_decimal T>
bool ColumnDecimal<T>::hasEqualValues() const
{
return this->template hasEqualValuesImpl<ColumnDecimal<T>>();
}
template <typename T>
template <is_decimal T>
StringRef ColumnDecimal<T>::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
{
auto * pos = arena.allocContinue(sizeof(T), begin);
@ -73,20 +73,20 @@ StringRef ColumnDecimal<T>::serializeValueIntoArena(size_t n, Arena & arena, cha
return StringRef(pos, sizeof(T));
}
template <typename T>
template <is_decimal T>
const char * ColumnDecimal<T>::deserializeAndInsertFromArena(const char * pos)
{
data.push_back(unalignedLoad<T>(pos));
return pos + sizeof(T);
}
template <typename T>
template <is_decimal T>
const char * ColumnDecimal<T>::skipSerializedInArena(const char * pos) const
{
return pos + sizeof(T);
}
template <typename T>
template <is_decimal T>
UInt64 ColumnDecimal<T>::get64([[maybe_unused]] size_t n) const
{
if constexpr (sizeof(T) > sizeof(UInt64))
@ -95,13 +95,13 @@ UInt64 ColumnDecimal<T>::get64([[maybe_unused]] size_t n) const
return static_cast<NativeT>(data[n]);
}
template <typename T>
template <is_decimal T>
void ColumnDecimal<T>::updateHashWithValue(size_t n, SipHash & hash) const
{
hash.update(data[n].value);
}
template <typename T>
template <is_decimal T>
void ColumnDecimal<T>::updateWeakHash32(WeakHash32 & hash) const
{
auto s = data.size();
@ -122,13 +122,13 @@ void ColumnDecimal<T>::updateWeakHash32(WeakHash32 & hash) const
}
}
template <typename T>
template <is_decimal T>
void ColumnDecimal<T>::updateHashFast(SipHash & hash) const
{
hash.update(reinterpret_cast<const char *>(data.data()), size() * sizeof(data[0]));
}
template <typename T>
template <is_decimal T>
void ColumnDecimal<T>::getPermutation(bool reverse, size_t limit, int , IColumn::Permutation & res) const
{
#if 1 /// TODO: perf test
@ -147,7 +147,7 @@ void ColumnDecimal<T>::getPermutation(bool reverse, size_t limit, int , IColumn:
permutation(reverse, limit, res);
}
template <typename T>
template <is_decimal T>
void ColumnDecimal<T>::updatePermutation(bool reverse, size_t limit, int, IColumn::Permutation & res, EqualRanges & equal_ranges) const
{
if (equal_ranges.empty())
@ -228,7 +228,7 @@ void ColumnDecimal<T>::updatePermutation(bool reverse, size_t limit, int, IColum
}
}
template <typename T>
template <is_decimal T>
ColumnPtr ColumnDecimal<T>::permute(const IColumn::Permutation & perm, size_t limit) const
{
size_t size = limit ? std::min(data.size(), limit) : data.size();
@ -244,7 +244,7 @@ ColumnPtr ColumnDecimal<T>::permute(const IColumn::Permutation & perm, size_t li
return res;
}
template <typename T>
template <is_decimal T>
MutableColumnPtr ColumnDecimal<T>::cloneResized(size_t size) const
{
auto res = this->create(0, scale);
@ -268,7 +268,7 @@ MutableColumnPtr ColumnDecimal<T>::cloneResized(size_t size) const
return res;
}
template <typename T>
template <is_decimal T>
void ColumnDecimal<T>::insertData(const char * src, size_t /*length*/)
{
T tmp;
@ -276,7 +276,7 @@ void ColumnDecimal<T>::insertData(const char * src, size_t /*length*/)
data.emplace_back(tmp);
}
template <typename T>
template <is_decimal T>
void ColumnDecimal<T>::insertRangeFrom(const IColumn & src, size_t start, size_t length)
{
const ColumnDecimal & src_vec = assert_cast<const ColumnDecimal &>(src);
@ -292,7 +292,7 @@ void ColumnDecimal<T>::insertRangeFrom(const IColumn & src, size_t start, size_t
memcpy(data.data() + old_size, &src_vec.data[start], length * sizeof(data[0]));
}
template <typename T>
template <is_decimal T>
ColumnPtr ColumnDecimal<T>::filter(const IColumn::Filter & filt, ssize_t result_size_hint) const
{
size_t size = data.size();
@ -321,19 +321,19 @@ ColumnPtr ColumnDecimal<T>::filter(const IColumn::Filter & filt, ssize_t result_
return res;
}
template <typename T>
template <is_decimal T>
void ColumnDecimal<T>::expand(const IColumn::Filter & mask, bool inverted)
{
expandDataByMask<T>(data, mask, inverted);
}
template <typename T>
template <is_decimal T>
ColumnPtr ColumnDecimal<T>::index(const IColumn & indexes, size_t limit) const
{
return selectIndexImpl(*this, indexes, limit);
}
template <typename T>
template <is_decimal T>
ColumnPtr ColumnDecimal<T>::replicate(const IColumn::Offsets & offsets) const
{
size_t size = data.size();
@ -360,13 +360,13 @@ ColumnPtr ColumnDecimal<T>::replicate(const IColumn::Offsets & offsets) const
return res;
}
template <typename T>
template <is_decimal T>
void ColumnDecimal<T>::gather(ColumnGathererStream & gatherer)
{
gatherer.gather(*this);
}
template <typename T>
template <is_decimal T>
ColumnPtr ColumnDecimal<T>::compress() const
{
size_t source_size = data.size() * sizeof(T);
@ -390,7 +390,7 @@ ColumnPtr ColumnDecimal<T>::compress() const
});
}
template <typename T>
template <is_decimal T>
void ColumnDecimal<T>::getExtremes(Field & min, Field & max) const
{
if (data.empty())

View File

@ -7,6 +7,8 @@
#include <Core/DecimalFunctions.h>
#include <Common/typeid_cast.h>
#include <common/sort.h>
#include <Core/TypeId.h>
#include <Core/TypeName.h>
#include <cmath>
@ -59,11 +61,9 @@ extern template class DecimalPaddedPODArray<Decimal256>;
extern template class DecimalPaddedPODArray<DateTime64>;
/// A ColumnVector for Decimals
template <typename T>
template <is_decimal T>
class ColumnDecimal final : public COWHelper<ColumnVectorHelper, ColumnDecimal<T>>
{
static_assert(IsDecimalNumber<T>);
private:
using Self = ColumnDecimal;
friend class COWHelper<ColumnVectorHelper, Self>;
@ -210,7 +210,12 @@ protected:
}
};
template <typename T>
template <class> class ColumnVector;
template <class T> struct ColumnVectorOrDecimalT { using Col = ColumnVector<T>; };
template <is_decimal T> struct ColumnVectorOrDecimalT<T> { using Col = ColumnDecimal<T>; };
template <class T> using ColumnVectorOrDecimal = typename ColumnVectorOrDecimalT<T>::Col;
template <is_decimal T>
template <typename Type>
ColumnPtr ColumnDecimal<T>::indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const
{

View File

@ -7,6 +7,8 @@
#include <common/unaligned.h>
#include <Core/Field.h>
#include <Common/assert_cast.h>
#include <Core/TypeId.h>
#include <Core/TypeName.h>
namespace DB
@ -102,7 +104,7 @@ template <class U> struct CompareHelper<Float64, U> : public FloatCompareHelper<
template <typename T>
class ColumnVector final : public COWHelper<ColumnVectorHelper, ColumnVector<T>>
{
static_assert(!IsDecimalNumber<T>);
static_assert(!is_decimal<T>);
private:
using Self = ColumnVector;

View File

@ -116,7 +116,7 @@ public:
template <typename U, typename = std::enable_if_t<is_big_int_v<U>> >
T operator() (const U & x) const
{
if constexpr (IsDecimalNumber<T>)
if constexpr (is_decimal<T>)
return static_cast<T>(static_cast<typename T::NativeType>(x));
else if constexpr (std::is_same_v<T, UInt128>)
throw Exception("No conversion to old UInt128 from " + demangle(typeid(U).name()), ErrorCodes::NOT_IMPLEMENTED);

View File

@ -197,12 +197,8 @@ inline size_t DefaultHash64(std::enable_if_t<(sizeof(T) > sizeof(UInt64)), T> ke
__builtin_unreachable();
}
template <typename T, typename Enable = void>
struct DefaultHash;
template <typename T>
struct DefaultHash<T, std::enable_if_t<!DB::IsDecimalNumber<T>>>
struct DefaultHash
{
size_t operator() (T key) const
{
@ -210,8 +206,8 @@ struct DefaultHash<T, std::enable_if_t<!DB::IsDecimalNumber<T>>>
}
};
template <typename T>
struct DefaultHash<T, std::enable_if_t<DB::IsDecimalNumber<T>>>
template <DB::is_decimal T>
struct DefaultHash<T>
{
size_t operator() (T key) const
{

View File

@ -187,7 +187,7 @@ struct RadixSortIntTraits
template <typename T>
using RadixSortNumTraits = std::conditional_t<
is_integer_v<T>,
DB::is_integer<T>,
std::conditional_t<is_unsigned_v<T>, RadixSortUIntTraits<T>, RadixSortIntTraits<T>>,
RadixSortFloatTraits<T>>;

View File

@ -33,7 +33,7 @@ bool lessOp(A a, B b)
return false;
/// int vs int
if constexpr (is_integer_v<A> && is_integer_v<B>)
if constexpr (is_integer<A> && is_integer<B>)
{
/// same signedness
if constexpr (is_signed_v<A> == is_signed_v<B>)
@ -49,7 +49,7 @@ bool lessOp(A a, B b)
}
/// int vs float
if constexpr (is_integer_v<A> && std::is_floating_point_v<B>)
if constexpr (is_integer<A> && std::is_floating_point_v<B>)
{
if constexpr (sizeof(A) <= 4)
return static_cast<double>(a) < static_cast<double>(b);
@ -57,7 +57,7 @@ bool lessOp(A a, B b)
return DecomposedFloat<B>(b).greater(a);
}
if constexpr (std::is_floating_point_v<A> && is_integer_v<B>)
if constexpr (std::is_floating_point_v<A> && is_integer<B>)
{
if constexpr (sizeof(B) <= 4)
return static_cast<double>(a) < static_cast<double>(b);
@ -65,8 +65,8 @@ bool lessOp(A a, B b)
return DecomposedFloat<A>(a).less(b);
}
static_assert(is_integer_v<A> || std::is_floating_point_v<A>);
static_assert(is_integer_v<B> || std::is_floating_point_v<B>);
static_assert(is_integer<A> || std::is_floating_point_v<A>);
static_assert(is_integer<B> || std::is_floating_point_v<B>);
__builtin_unreachable();
}
@ -109,7 +109,7 @@ bool equalsOp(A a, B b)
return false;
/// int vs int
if constexpr (is_integer_v<A> && is_integer_v<B>)
if constexpr (is_integer<A> && is_integer<B>)
{
/// same signedness
if constexpr (is_signed_v<A> == is_signed_v<B>)
@ -125,7 +125,7 @@ bool equalsOp(A a, B b)
}
/// int vs float
if constexpr (is_integer_v<A> && std::is_floating_point_v<B>)
if constexpr (is_integer<A> && std::is_floating_point_v<B>)
{
if constexpr (sizeof(A) <= 4)
return static_cast<double>(a) == static_cast<double>(b);
@ -133,7 +133,7 @@ bool equalsOp(A a, B b)
return DecomposedFloat<B>(b).equals(a);
}
if constexpr (std::is_floating_point_v<A> && is_integer_v<B>)
if constexpr (std::is_floating_point_v<A> && is_integer<B>)
{
if constexpr (sizeof(B) <= 4)
return static_cast<double>(a) == static_cast<double>(b);

View File

@ -47,21 +47,21 @@ template <> struct ConstructDecInt<32> { using Type = Int256; };
template <typename T, typename U>
struct DecCompareInt
{
using Type = typename ConstructDecInt<(!IsDecimalNumber<U> || sizeof(T) > sizeof(U)) ? sizeof(T) : sizeof(U)>::Type;
using Type = typename ConstructDecInt<(!is_decimal<U> || sizeof(T) > sizeof(U)) ? sizeof(T) : sizeof(U)>::Type;
using TypeA = Type;
using TypeB = Type;
};
///
template <typename A, typename B, template <typename, typename> typename Operation, bool _check_overflow = true,
bool _actual = IsDecimalNumber<A> || IsDecimalNumber<B>>
bool _actual = is_decimal<A> || is_decimal<B>>
class DecimalComparison
{
public:
using CompareInt = typename DecCompareInt<A, B>::Type;
using Op = Operation<CompareInt, CompareInt>;
using ColVecA = std::conditional_t<IsDecimalNumber<A>, ColumnDecimal<A>, ColumnVector<A>>;
using ColVecB = std::conditional_t<IsDecimalNumber<B>, ColumnDecimal<B>, ColumnVector<B>>;
using ColVecA = ColumnVectorOrDecimal<A>;
using ColVecB = ColumnVectorOrDecimal<B>;
using ArrayA = typename ColVecA::Container;
using ArrayB = typename ColVecB::Container;
@ -116,7 +116,7 @@ private:
}
template <typename T, typename U>
static std::enable_if_t<IsDecimalNumber<T> && IsDecimalNumber<U>, Shift>
static std::enable_if_t<is_decimal<T> && is_decimal<U>, Shift>
getScales(const DataTypePtr & left_type, const DataTypePtr & right_type)
{
const DataTypeDecimalBase<T> * decimal0 = checkDecimalBase<T>(*left_type);
@ -138,7 +138,7 @@ private:
}
template <typename T, typename U>
static std::enable_if_t<IsDecimalNumber<T> && !IsDecimalNumber<U>, Shift>
static std::enable_if_t<is_decimal<T> && !is_decimal<U>, Shift>
getScales(const DataTypePtr & left_type, const DataTypePtr &)
{
Shift shift;
@ -149,7 +149,7 @@ private:
}
template <typename T, typename U>
static std::enable_if_t<!IsDecimalNumber<T> && IsDecimalNumber<U>, Shift>
static std::enable_if_t<!is_decimal<T> && is_decimal<U>, Shift>
getScales(const DataTypePtr &, const DataTypePtr & right_type)
{
Shift shift;
@ -222,13 +222,13 @@ private:
static NO_INLINE UInt8 apply(A a, B b, CompareInt scale [[maybe_unused]])
{
CompareInt x;
if constexpr (IsDecimalNumber<A>)
if constexpr (is_decimal<A>)
x = a.value;
else
x = a;
CompareInt y;
if constexpr (IsDecimalNumber<B>)
if constexpr (is_decimal<B>)
y = b.value;
else
y = b;

View File

@ -6,7 +6,6 @@
#include <common/arithmeticOverflow.h>
#include <limits>
#include <type_traits>
namespace DB
@ -230,7 +229,7 @@ ReturnType convertToImpl(const DecimalType & decimal, size_t scale, To & result)
{
result = static_cast<To>(decimal.value) / static_cast<To>(scaleMultiplier<NativeT>(scale));
}
else if constexpr (is_integer_v<To> && (sizeof(To) >= sizeof(NativeT)))
else if constexpr (is_integer<To> && (sizeof(To) >= sizeof(NativeT)))
{
NativeT whole = getWholePart(decimal, scale);
@ -247,9 +246,9 @@ ReturnType convertToImpl(const DecimalType & decimal, size_t scale, To & result)
result = static_cast<To>(whole);
}
else if constexpr (is_integer_v<To>)
else if constexpr (is_integer<To>)
{
using ToNativeT = typename NativeType<To>::Type;
using ToNativeT = NativeType<To>;
using CastTo = std::conditional_t<(is_big_int_v<NativeT> && std::is_same_v<ToNativeT, UInt8>), uint8_t, ToNativeT>;
const NativeT whole = getWholePart(decimal, scale);

View File

@ -837,12 +837,6 @@ T safeGet(Field & field)
return field.template safeGet<T>();
}
template <> inline constexpr const char * TypeName<Array> = "Array";
template <> inline constexpr const char * TypeName<Tuple> = "Tuple";
template <> inline constexpr const char * TypeName<Map> = "Map";
template <> inline constexpr const char * TypeName<AggregateFunctionStateData> = "AggregateFunctionState";
template <typename T>
Field::Field(T && rhs, enable_if_not_field_or_bool_or_stringlike_t<T>) //-V730
{

61
src/Core/TypeId.h Normal file
View File

@ -0,0 +1,61 @@
#pragma once
#include <Core/Types.h>
namespace DB
{
/**
* Obtain TypeIndex value from real type if possible.
*
* Returns TypeIndex::Nothing if type was not present in TypeIndex;
* Returns TypeIndex element otherwise.
*
* @example TypeId<UInt8> == TypeIndex::UInt8
* @example TypeId<MySuperType> == TypeIndex::Nothing
*/
template <class T> inline constexpr TypeIndex TypeId = TypeIndex::Nothing;
template <TypeIndex index> struct ReverseTypeIdT : std::false_type {};
/**
* Obtain real type from TypeIndex if possible.
*
* Returns a type alias if is corresponds to TypeIndex value.
* Yields a compiler error otherwise.
*
* @example ReverseTypeId<TypeIndex::UInt8> == UInt8
*/
template <TypeIndex index> using ReverseTypeId = typename ReverseTypeIdT<index>::T;
template <TypeIndex index> constexpr bool HasReverseTypeId = ReverseTypeIdT<index>::value;
#define TYPEID_MAP(_A) \
template <> inline constexpr TypeIndex TypeId<_A> = TypeIndex::_A; \
template <> struct ReverseTypeIdT<TypeIndex::_A> : std::true_type { using T = _A; };
TYPEID_MAP(UInt8)
TYPEID_MAP(UInt16)
TYPEID_MAP(UInt32)
TYPEID_MAP(UInt64)
TYPEID_MAP(UInt128)
TYPEID_MAP(UInt256)
TYPEID_MAP(Int8)
TYPEID_MAP(Int16)
TYPEID_MAP(Int32)
TYPEID_MAP(Int64)
TYPEID_MAP(Int128)
TYPEID_MAP(Int256)
TYPEID_MAP(Float32)
TYPEID_MAP(Float64)
TYPEID_MAP(UUID)
TYPEID_MAP(Decimal32)
TYPEID_MAP(Decimal64)
TYPEID_MAP(Decimal128)
TYPEID_MAP(Decimal256)
TYPEID_MAP(DateTime64)
TYPEID_MAP(String)
struct Array;
TYPEID_MAP(Array)
}

39
src/Core/TypeName.h Normal file
View File

@ -0,0 +1,39 @@
#pragma once
#include <Core/Types.h>
namespace DB
{
struct Array;
struct Tuple;
struct Map;
struct AggregateFunctionStateData;
template <class T> constexpr const char * TypeName = "";
template <> inline constexpr const char * TypeName<UInt8> = "UInt8";
template <> inline constexpr const char * TypeName<UInt16> = "UInt16";
template <> inline constexpr const char * TypeName<UInt32> = "UInt32";
template <> inline constexpr const char * TypeName<UInt64> = "UInt64";
template <> inline constexpr const char * TypeName<UInt128> = "UInt128";
template <> inline constexpr const char * TypeName<UInt256> = "UInt256";
template <> inline constexpr const char * TypeName<Int8> = "Int8";
template <> inline constexpr const char * TypeName<Int16> = "Int16";
template <> inline constexpr const char * TypeName<Int32> = "Int32";
template <> inline constexpr const char * TypeName<Int64> = "Int64";
template <> inline constexpr const char * TypeName<Int128> = "Int128";
template <> inline constexpr const char * TypeName<Int256> = "Int256";
template <> inline constexpr const char * TypeName<Float32> = "Float32";
template <> inline constexpr const char * TypeName<Float64> = "Float64";
template <> inline constexpr const char * TypeName<String> = "String";
template <> inline constexpr const char * TypeName<UUID> = "UUID";
template <> inline constexpr const char * TypeName<Decimal32> = "Decimal32";
template <> inline constexpr const char * TypeName<Decimal64> = "Decimal64";
template <> inline constexpr const char * TypeName<Decimal128> = "Decimal128";
template <> inline constexpr const char * TypeName<Decimal256> = "Decimal256";
template <> inline constexpr const char * TypeName<DateTime64> = "DateTime64";
template <> inline constexpr const char * TypeName<Array> = "Array";
template <> inline constexpr const char * TypeName<Tuple> = "Tuple";
template <> inline constexpr const char * TypeName<Map> = "Map";
template <> inline constexpr const char * TypeName<AggregateFunctionStateData> = "AggregateFunctionState";
}

View File

@ -4,7 +4,7 @@
#include <string>
#include <vector>
#include <common/strong_typedef.h>
#include <common/extended_types.h>
#include <common/Decimal.h>
#include <common/defines.h>
@ -46,6 +46,7 @@ struct Null
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wshadow"
#endif
/// @note Except explicitly described you should not assume on TypeIndex numbers and/or their orders in this enum.
enum class TypeIndex
{
@ -87,11 +88,11 @@ enum class TypeIndex
LowCardinality,
Map,
};
#if !defined(__clang__)
#pragma GCC diagnostic pop
#endif
using UInt128 = ::UInt128;
using UInt256 = ::UInt256;
using Int128 = ::Int128;
@ -99,207 +100,6 @@ using Int256 = ::Int256;
STRONG_TYPEDEF(UInt128, UUID)
template <typename T> constexpr const char * TypeName = "";
template <> inline constexpr const char * TypeName<UInt8> = "UInt8";
template <> inline constexpr const char * TypeName<UInt16> = "UInt16";
template <> inline constexpr const char * TypeName<UInt32> = "UInt32";
template <> inline constexpr const char * TypeName<UInt64> = "UInt64";
template <> inline constexpr const char * TypeName<UInt128> = "UInt128";
template <> inline constexpr const char * TypeName<UInt256> = "UInt256";
template <> inline constexpr const char * TypeName<Int8> = "Int8";
template <> inline constexpr const char * TypeName<Int16> = "Int16";
template <> inline constexpr const char * TypeName<Int32> = "Int32";
template <> inline constexpr const char * TypeName<Int64> = "Int64";
template <> inline constexpr const char * TypeName<Int128> = "Int128";
template <> inline constexpr const char * TypeName<Int256> = "Int256";
template <> inline constexpr const char * TypeName<Float32> = "Float32";
template <> inline constexpr const char * TypeName<Float64> = "Float64";
template <> inline constexpr const char * TypeName<String> = "String";
template <> inline constexpr const char * TypeName<UUID> = "UUID";
/// TODO Try to remove it.
template <typename T> constexpr TypeIndex TypeId = TypeIndex::Nothing;
template <> inline constexpr TypeIndex TypeId<UInt8> = TypeIndex::UInt8;
template <> inline constexpr TypeIndex TypeId<UInt16> = TypeIndex::UInt16;
template <> inline constexpr TypeIndex TypeId<UInt32> = TypeIndex::UInt32;
template <> inline constexpr TypeIndex TypeId<UInt64> = TypeIndex::UInt64;
template <> inline constexpr TypeIndex TypeId<UInt128> = TypeIndex::UInt128;
template <> inline constexpr TypeIndex TypeId<UInt256> = TypeIndex::UInt256;
template <> inline constexpr TypeIndex TypeId<Int8> = TypeIndex::Int8;
template <> inline constexpr TypeIndex TypeId<Int16> = TypeIndex::Int16;
template <> inline constexpr TypeIndex TypeId<Int32> = TypeIndex::Int32;
template <> inline constexpr TypeIndex TypeId<Int64> = TypeIndex::Int64;
template <> inline constexpr TypeIndex TypeId<Int128> = TypeIndex::Int128;
template <> inline constexpr TypeIndex TypeId<Int256> = TypeIndex::Int256;
template <> inline constexpr TypeIndex TypeId<Float32> = TypeIndex::Float32;
template <> inline constexpr TypeIndex TypeId<Float64> = TypeIndex::Float64;
template <> inline constexpr TypeIndex TypeId<UUID> = TypeIndex::UUID;
/// Not a data type in database, defined just for convenience.
using Strings = std::vector<String>;
/// Own FieldType for Decimal.
/// It is only a "storage" for decimal. To perform operations, you also have to provide a scale (number of digits after point).
template <typename T>
struct Decimal
{
using NativeType = T;
Decimal() = default;
Decimal(Decimal<T> &&) = default;
Decimal(const Decimal<T> &) = default;
Decimal(const T & value_)
: value(value_)
{}
template <typename U>
Decimal(const Decimal<U> & x)
: value(x.value)
{}
constexpr Decimal<T> & operator = (Decimal<T> &&) = default;
constexpr Decimal<T> & operator = (const Decimal<T> &) = default;
operator T () const { return value; }
template <typename U>
U convertTo() const
{
/// no IsDecimalNumber defined yet
if constexpr (std::is_same_v<U, Decimal<Int32>> ||
std::is_same_v<U, Decimal<Int64>> ||
std::is_same_v<U, Decimal<Int128>> ||
std::is_same_v<U, Decimal<Int256>>)
{
return convertTo<typename U::NativeType>();
}
else
return static_cast<U>(value);
}
const Decimal<T> & operator += (const T & x) { value += x; return *this; }
const Decimal<T> & operator -= (const T & x) { value -= x; return *this; }
const Decimal<T> & operator *= (const T & x) { value *= x; return *this; }
const Decimal<T> & operator /= (const T & x) { value /= x; return *this; }
const Decimal<T> & operator %= (const T & x) { value %= x; return *this; }
template <typename U> const Decimal<T> & operator += (const Decimal<U> & x) { value += x.value; return *this; }
template <typename U> const Decimal<T> & operator -= (const Decimal<U> & x) { value -= x.value; return *this; }
template <typename U> const Decimal<T> & operator *= (const Decimal<U> & x) { value *= x.value; return *this; }
template <typename U> const Decimal<T> & operator /= (const Decimal<U> & x) { value /= x.value; return *this; }
template <typename U> const Decimal<T> & operator %= (const Decimal<U> & x) { value %= x.value; return *this; }
/// This is to avoid UB for sumWithOverflow()
void NO_SANITIZE_UNDEFINED addOverflow(const T & x) { value += x; }
T value;
};
template <typename T> inline bool operator< (const Decimal<T> & x, const Decimal<T> & y) { return x.value < y.value; }
template <typename T> inline bool operator> (const Decimal<T> & x, const Decimal<T> & y) { return x.value > y.value; }
template <typename T> inline bool operator<= (const Decimal<T> & x, const Decimal<T> & y) { return x.value <= y.value; }
template <typename T> inline bool operator>= (const Decimal<T> & x, const Decimal<T> & y) { return x.value >= y.value; }
template <typename T> inline bool operator== (const Decimal<T> & x, const Decimal<T> & y) { return x.value == y.value; }
template <typename T> inline bool operator!= (const Decimal<T> & x, const Decimal<T> & y) { return x.value != y.value; }
template <typename T> inline Decimal<T> operator+ (const Decimal<T> & x, const Decimal<T> & y) { return x.value + y.value; }
template <typename T> inline Decimal<T> operator- (const Decimal<T> & x, const Decimal<T> & y) { return x.value - y.value; }
template <typename T> inline Decimal<T> operator* (const Decimal<T> & x, const Decimal<T> & y) { return x.value * y.value; }
template <typename T> inline Decimal<T> operator/ (const Decimal<T> & x, const Decimal<T> & y) { return x.value / y.value; }
template <typename T> inline Decimal<T> operator- (const Decimal<T> & x) { return -x.value; }
using Decimal32 = Decimal<Int32>;
using Decimal64 = Decimal<Int64>;
using Decimal128 = Decimal<Int128>;
using Decimal256 = Decimal<Int256>;
// Distinguishable type to allow function resolution/deduction based on value type,
// but also relatively easy to convert to/from Decimal64.
class DateTime64 : public Decimal64
{
public:
using Base = Decimal64;
using Base::Base;
DateTime64(const Base & v)
: Base(v)
{}
};
template <> inline constexpr const char * TypeName<Decimal32> = "Decimal32";
template <> inline constexpr const char * TypeName<Decimal64> = "Decimal64";
template <> inline constexpr const char * TypeName<Decimal128> = "Decimal128";
template <> inline constexpr const char * TypeName<Decimal256> = "Decimal256";
template <> inline constexpr const char * TypeName<DateTime64> = "DateTime64";
template <> inline constexpr TypeIndex TypeId<Decimal32> = TypeIndex::Decimal32;
template <> inline constexpr TypeIndex TypeId<Decimal64> = TypeIndex::Decimal64;
template <> inline constexpr TypeIndex TypeId<Decimal128> = TypeIndex::Decimal128;
template <> inline constexpr TypeIndex TypeId<Decimal256> = TypeIndex::Decimal256;
template <> inline constexpr TypeIndex TypeId<DateTime64> = TypeIndex::DateTime64;
template <typename T> constexpr bool IsDecimalNumber = false;
template <> inline constexpr bool IsDecimalNumber<Decimal32> = true;
template <> inline constexpr bool IsDecimalNumber<Decimal64> = true;
template <> inline constexpr bool IsDecimalNumber<Decimal128> = true;
template <> inline constexpr bool IsDecimalNumber<Decimal256> = true;
template <> inline constexpr bool IsDecimalNumber<DateTime64> = true;
template <typename T> struct NativeType { using Type = T; };
template <> struct NativeType<Decimal32> { using Type = Int32; };
template <> struct NativeType<Decimal64> { using Type = Int64; };
template <> struct NativeType<Decimal128> { using Type = Int128; };
template <> struct NativeType<Decimal256> { using Type = Int256; };
template <> struct NativeType<DateTime64> { using Type = Int64; };
template <typename T> constexpr bool OverBigInt = false;
template <> inline constexpr bool OverBigInt<Int128> = true;
template <> inline constexpr bool OverBigInt<UInt128> = true;
template <> inline constexpr bool OverBigInt<Int256> = true;
template <> inline constexpr bool OverBigInt<UInt256> = true;
template <> inline constexpr bool OverBigInt<Decimal128> = true;
template <> inline constexpr bool OverBigInt<Decimal256> = true;
}
/// Specialization of `std::hash` for the Decimal<T> types.
namespace std
{
template <typename T>
struct hash<DB::Decimal<T>> { size_t operator()(const DB::Decimal<T> & x) const { return hash<T>()(x.value); } };
template <>
struct hash<DB::Decimal128>
{
size_t operator()(const DB::Decimal128 & x) const
{
return std::hash<DB::Int64>()(x.value >> 64)
^ std::hash<DB::Int64>()(x.value & std::numeric_limits<DB::UInt64>::max());
}
};
template <>
struct hash<DB::DateTime64>
{
size_t operator()(const DB::DateTime64 & x) const
{
return std::hash<std::decay_t<decltype(x)>::NativeType>()(x);
}
};
template <>
struct hash<DB::Decimal256>
{
size_t operator()(const DB::Decimal256 & x) const
{
// temp solution
static constexpr DB::UInt64 max_uint_mask = std::numeric_limits<DB::UInt64>::max();
return std::hash<DB::Int64>()(static_cast<DB::Int64>(x.value >> 64 & max_uint_mask))
^ std::hash<DB::Int64>()(static_cast<DB::Int64>(x.value & max_uint_mask));
}
};
}

View File

@ -164,7 +164,7 @@ class DataTypeDateTime;
class DataTypeDateTime64;
template <typename T> class DataTypeEnum;
template <typename T> class DataTypeNumber;
template <typename T> class DataTypeDecimal;
template <is_decimal T> class DataTypeDecimal;
template <typename T, typename F, typename... ExtraArgs>

View File

@ -28,19 +28,19 @@ bool decimalCheckArithmeticOverflow(ContextPtr context)
return context->getSettingsRef().decimal_check_overflow;
}
template <typename T>
template <is_decimal T>
Field DataTypeDecimalBase<T>::getDefault() const
{
return DecimalField(T(0), scale);
}
template <typename T>
template <is_decimal T>
MutableColumnPtr DataTypeDecimalBase<T>::createColumn() const
{
return ColumnType::create(0, scale);
}
template <typename T>
template <is_decimal T>
T DataTypeDecimalBase<T>::getScaleMultiplier(UInt32 scale_)
{
return DecimalUtils::scaleMultiplier<typename T::NativeType>(scale_);

View File

@ -53,11 +53,9 @@ inline UInt32 leastDecimalPrecisionFor(TypeIndex int_type)
/// Operation between two decimals leads to Decimal(P, S), where
/// P is one of (9, 18, 38, 76); equals to the maximum precision for the biggest underlying type of operands.
/// S is maximum scale of operands. The allowed valuas are [0, precision]
template <typename T>
template <is_decimal T>
class DataTypeDecimalBase : public IDataType
{
static_assert(IsDecimalNumber<T>);
public:
using FieldType = T;
using ColumnType = ColumnDecimal<T>;

View File

@ -28,13 +28,13 @@ MutableColumnPtr DataTypeNumberBase<T>::createColumn() const
template <typename T>
bool DataTypeNumberBase<T>::isValueRepresentedByInteger() const
{
return is_integer_v<T>;
return is_integer<T>;
}
template <typename T>
bool DataTypeNumberBase<T>::isValueRepresentedByUnsignedInteger() const
{
return is_integer_v<T> && is_unsigned_v<T>;
return is_integer<T> && is_unsigned_v<T>;
}

View File

@ -25,14 +25,14 @@ namespace ErrorCodes
}
template <typename T>
template <is_decimal T>
std::string DataTypeDecimal<T>::doGetName() const
{
return fmt::format("Decimal({}, {})", this->precision, this->scale);
}
template <typename T>
template <is_decimal T>
bool DataTypeDecimal<T>::equals(const IDataType & rhs) const
{
if (auto * ptype = typeid_cast<const DataTypeDecimal<T> *>(&rhs))
@ -40,14 +40,14 @@ bool DataTypeDecimal<T>::equals(const IDataType & rhs) const
return false;
}
template <typename T>
template <is_decimal T>
DataTypePtr DataTypeDecimal<T>::promoteNumericType() const
{
using PromotedType = DataTypeDecimal<Decimal128>;
return std::make_shared<PromotedType>(PromotedType::maxPrecision(), this->scale);
}
template <typename T>
template <is_decimal T>
T DataTypeDecimal<T>::parseFromString(const String & str) const
{
ReadBufferFromMemory buf(str.data(), str.size());
@ -61,7 +61,7 @@ T DataTypeDecimal<T>::parseFromString(const String & str) const
return x;
}
template <typename T>
template <is_decimal T>
SerializationPtr DataTypeDecimal<T>::doGetDefaultSerialization() const
{
return std::make_shared<SerializationDecimal<T>>(this->precision, this->scale);

View File

@ -24,11 +24,10 @@ namespace ErrorCodes
/// Operation between two decimals leads to Decimal(P, S), where
/// P is one of (9, 18, 38, 76); equals to the maximum precision for the biggest underlying type of operands.
/// S is maximum scale of operands. The allowed valuas are [0, precision]
template <typename T>
template <is_decimal T>
class DataTypeDecimal final : public DataTypeDecimalBase<T>
{
using Base = DataTypeDecimalBase<T>;
static_assert(IsDecimalNumber<T>);
public:
using typename Base::FieldType;

View File

@ -4,7 +4,7 @@
#include <Common/COW.h>
#include <boost/noncopyable.hpp>
#include <Core/Names.h>
#include <Core/Types.h>
#include <Core/TypeId.h>
#include <DataTypes/DataTypeCustom.h>
@ -490,7 +490,7 @@ template <typename DataType> constexpr bool IsDataTypeDateOrDateTime = false;
template <typename DataType> constexpr bool IsDataTypeDecimalOrNumber = IsDataTypeDecimal<DataType> || IsDataTypeNumber<DataType>;
template <typename T>
template <is_decimal T>
class DataTypeDecimal;
template <typename T>

View File

@ -172,14 +172,14 @@ template <typename A, typename B>
struct ResultOfIf
{
static constexpr bool has_float = std::is_floating_point_v<A> || std::is_floating_point_v<B>;
static constexpr bool has_integer = is_integer_v<A> || is_integer_v<B>;
static constexpr bool has_integer = is_integer<A> || is_integer<B>;
static constexpr bool has_signed = is_signed_v<A> || is_signed_v<B>;
static constexpr bool has_unsigned = !is_signed_v<A> || !is_signed_v<B>;
static constexpr bool has_big_int = is_big_int_v<A> || is_big_int_v<B>;
static constexpr size_t max_size_of_unsigned_integer = max(is_signed_v<A> ? 0 : sizeof(A), is_signed_v<B> ? 0 : sizeof(B));
static constexpr size_t max_size_of_signed_integer = max(is_signed_v<A> ? sizeof(A) : 0, is_signed_v<B> ? sizeof(B) : 0);
static constexpr size_t max_size_of_integer = max(is_integer_v<A> ? sizeof(A) : 0, is_integer_v<B> ? sizeof(B) : 0);
static constexpr size_t max_size_of_integer = max(is_integer<A> ? sizeof(A) : 0, is_integer<B> ? sizeof(B) : 0);
static constexpr size_t max_size_of_float = max(std::is_floating_point_v<A> ? sizeof(A) : 0, std::is_floating_point_v<B> ? sizeof(B) : 0);
using ConstructedType = typename Construct<has_signed, has_float,
@ -190,9 +190,9 @@ struct ResultOfIf
using Type =
std::conditional_t<std::is_same_v<A, B>, A,
std::conditional_t<IsDecimalNumber<A> && IsDecimalNumber<B>,
std::conditional_t<is_decimal<A> && is_decimal<B>,
std::conditional_t<(sizeof(A) > sizeof(B)), A, B>,
std::conditional_t<!IsDecimalNumber<A> && !IsDecimalNumber<B>,
std::conditional_t<!is_decimal<A> && !is_decimal<B>,
ConstructedType, Error>>>;
};

View File

@ -25,7 +25,7 @@ void SerializationNumber<T>::deserializeText(IColumn & column, ReadBuffer & istr
{
T x;
if constexpr (is_integer_v<T> && is_arithmetic_v<T>)
if constexpr (is_integer<T> && is_arithmetic_v<T>)
readIntTextUnsafe(x, istr);
else
readText(x, istr);

View File

@ -240,8 +240,7 @@ public:
using ColumnType =
std::conditional_t<std::is_same_v<DictionaryAttributeType, Array>, ColumnArray,
std::conditional_t<std::is_same_v<DictionaryAttributeType, String>, ColumnString,
std::conditional_t<IsDecimalNumber<DictionaryAttributeType>, ColumnDecimal<DictionaryAttributeType>,
ColumnVector<DictionaryAttributeType>>>>;
ColumnVectorOrDecimal<DictionaryAttributeType>>>;
using ColumnPtr = typename ColumnType::MutablePtr;
@ -267,7 +266,7 @@ public:
{
return ColumnType::create(size);
}
else if constexpr (IsDecimalNumber<DictionaryAttributeType>)
else if constexpr (is_decimal<DictionaryAttributeType>)
{
auto nested_type = removeNullable(dictionary_attribute.type);
auto scale = getDecimalScale(*nested_type);

View File

@ -12,6 +12,7 @@
#include <DataTypes/IDataType.h>
#include <Interpreters/IExternalLoadable.h>
#include <common/EnumReflection.h>
#include <Core/TypeId.h>
#if defined(__GNUC__)
/// GCC mistakenly warns about the names in enum class.
@ -22,7 +23,10 @@ namespace DB
{
using TypeIndexUnderlying = magic_enum::underlying_type_t<TypeIndex>;
/// We need to be able to map TypeIndex -> AttributeUnderlyingType and AttributeUnderlyingType -> real type
// We need to be able to map TypeIndex -> AttributeUnderlyingType and AttributeUnderlyingType -> real type
// The first can be done by defining AttributeUnderlyingType enum values to TypeIndex values and then performing
// a enum_cast.
// The second can be achieved by using ReverseTypeId
#define map_item(__T) __T = static_cast<TypeIndexUnderlying>(TypeIndex::__T)
enum class AttributeUnderlyingType : TypeIndexUnderlying
@ -65,26 +69,23 @@ struct DictionaryAttribute final
const bool is_nullable;
};
struct T { using AttributeType = int; };
template <AttributeUnderlyingType type>
struct DictionaryAttributeType
{
/// Converts @c type to it underlying type e.g. AttributeUnderlyingType::UInt8 -> UInt8
using AttributeType = ReverseTypeId<
static_cast<TypeIndex>(
static_cast<TypeIndexUnderlying>(type))>;
};
template <typename F>
void callOnDictionaryAttributeType(AttributeUnderlyingType type, F && func)
constexpr void callOnDictionaryAttributeType(AttributeUnderlyingType type, F && func)
{
// for (AttributeUnderlyingType other : magic_enum::enum_values<AttributeUnderlyingType>())
// if (type == other)
// {
//
// }
//
// switch (type)
// {
//#define M(TYPE) \
// case AttributeUnderlyingType::TYPE: \
// func(DictionaryAttributeType<AttributeUnderlyingType::TYPE>()); \
// break;
// FOR_ATTRIBUTE_TYPES(M)
//#undef M
// }
static_for<AttributeUnderlyingType>([type, func = std::forward<F>(func)](auto other)
{
if (type == other)
func(DictionaryAttributeType<other>{});
});
};
struct DictionarySpecialAttribute final

View File

@ -81,7 +81,7 @@ struct DivideIntegralImpl
/// Otherwise overflow may occur due to integer promotion. Example: int8_t(-1) / uint64_t(2).
/// NOTE: overflow is still possible when dividing large signed number to large unsigned number or vice-versa. But it's less harmful.
if constexpr (is_integer_v<A> && is_integer_v<B> && (is_signed_v<A> || is_signed_v<B>))
if constexpr (is_integer<A> && is_integer<B> && (is_signed_v<A> || is_signed_v<B>))
{
using SignedCastA = make_signed_t<CastA>;
using SignedCastB = std::conditional_t<sizeof(A) <= sizeof(B), make_signed_t<CastB>, SignedCastA>;

View File

@ -186,7 +186,7 @@ enum class OpCase { Vector, LeftConstant, RightConstant };
template <class T>
inline constexpr const auto & undec(const T & x)
{
if constexpr (IsDecimalNumber<T>)
if constexpr (is_decimal<T>)
return x.value;
else
return x;
@ -301,9 +301,9 @@ struct DecimalBinaryOperation
{
private:
using ResultType = OpResultType; // e.g. Decimal32
using NativeResultType = typename NativeType<ResultType>::Type; // e.g. UInt32 for Decimal32
using NativeResultType = NativeType<ResultType>; // e.g. UInt32 for Decimal32
using ResultContainerType = typename std::conditional_t<IsDecimalNumber<ResultType>,
using ResultContainerType = typename std::conditional_t<is_decimal<ResultType>,
ColumnDecimal<ResultType>,
ColumnVector<ResultType>>::Container;
@ -312,8 +312,8 @@ public:
static void NO_INLINE process(const A & a, const B & b, ResultContainerType & c,
NativeResultType scale_a, NativeResultType scale_b)
{
if constexpr (op_case == OpCase::LeftConstant) static_assert(!IsDecimalNumber<A>);
if constexpr (op_case == OpCase::RightConstant) static_assert(!IsDecimalNumber<B>);
if constexpr (op_case == OpCase::LeftConstant) static_assert(!is_decimal<A>);
if constexpr (op_case == OpCase::RightConstant) static_assert(!is_decimal<B>);
size_t size;
@ -384,8 +384,8 @@ public:
template <bool is_decimal_a, bool is_decimal_b, class A, class B>
static ResultType process(A a, B b, NativeResultType scale_a, NativeResultType scale_b)
{
static_assert(!IsDecimalNumber<A>);
static_assert(!IsDecimalNumber<B>);
static_assert(!is_decimal<A>);
static_assert(!is_decimal<B>);
if constexpr (is_division && is_decimal_b)
return applyScaledDiv<is_decimal_a>(a, b, scale_a);
@ -791,11 +791,11 @@ class FunctionBinaryArithmetic : public IFunction
static auto helperGetOrConvert(const CC & col_const, const C & col)
{
using ResultType = typename ResultDataType::FieldType;
using NativeResultType = typename NativeType<ResultType>::Type;
using NativeResultType = NativeType<ResultType>;
if constexpr (IsFloatingPoint<ResultDataType> && IsDecimalNumber<T>)
if constexpr (is_floating_point<ResultDataType> && is_decimal<T>)
return DecimalUtils::convertTo<NativeResultType>(col_const->template getValue<T>(), col.getScale());
else if constexpr (IsDecimalNumber<T>)
else if constexpr (is_decimal<T>)
return col_const->template getValue<T>().value;
else
return col_const->template getValue<T>();
@ -823,15 +823,15 @@ class FunctionBinaryArithmetic : public IFunction
using T1 = typename RightDataType::FieldType;
using ResultType = typename ResultDataType::FieldType;
using NativeResultType = typename NativeType<ResultType>::Type;
using NativeResultType = NativeType<ResultType>;
using OpImpl = DecimalBinaryOperation<Op, ResultType, false>;
using OpImplCheck = DecimalBinaryOperation<Op, ResultType, true>;
using ColVecResult = std::conditional_t<IsDecimalNumber<ResultType>,
using ColVecResult = std::conditional_t<is_decimal<ResultType>,
ColumnDecimal<ResultType>, ColumnVector<ResultType>>;
static constexpr const bool left_is_decimal = IsDecimalNumber<T0>;
static constexpr const bool right_is_decimal = IsDecimalNumber<T1>;
static constexpr const bool left_is_decimal = is_decimal<T0>;
static constexpr const bool right_is_decimal = is_decimal<T1>;
static constexpr const bool result_is_decimal = IsDataTypeDecimal<ResultDataType>;
typename ColVecResult::MutablePtr col_res = nullptr;
@ -1178,9 +1178,9 @@ public:
using T0 = typename LeftDataType::FieldType;
using T1 = typename RightDataType::FieldType;
using ResultType = typename ResultDataType::FieldType;
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>>;
using ColVecT0 = ColumnVectorOrDecimal<T0>;
using ColVecT1 = ColumnVectorOrDecimal<T1>;
using ColVecResult = ColumnVectorOrDecimal<ResultType>;
const auto * const col_left_raw = arguments[0].column.get();
const auto * const col_right_raw = arguments[1].column.get();

View File

@ -68,13 +68,13 @@ const ColumnConst * checkAndGetColumnConstStringOrFixedString(const IColumn * co
/// Transform anything to Field.
template <typename T>
inline std::enable_if_t<!IsDecimalNumber<T>, Field> toField(const T & x)
Field toField(const T & x)
{
return Field(NearestFieldType<T>(x));
}
template <typename T>
inline std::enable_if_t<IsDecimalNumber<T>, Field> toField(const T & x, UInt32 scale)
template <is_decimal T>
Field toField(const T & x, UInt32 scale)
{
return Field(NearestFieldType<T>(x, scale));
}

View File

@ -150,7 +150,7 @@ private:
using Types = std::decay_t<decltype(types)>;
using Type = typename Types::RightType;
using ReturnType = std::conditional_t<Impl::always_returns_float64 || !std::is_floating_point_v<Type>, Float64, Type>;
using ColVecType = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
using ColVecType = ColumnVectorOrDecimal<Type>;
const auto col_vec = checkAndGetColumn<ColVecType>(col.column.get());
return (res = execute<Type, ReturnType>(col_vec)) != nullptr;

View File

@ -38,8 +38,8 @@ template <typename A, typename Op>
struct UnaryOperationImpl
{
using ResultType = typename Op::ResultType;
using ColVecA = std::conditional_t<IsDecimalNumber<A>, ColumnDecimal<A>, ColumnVector<A>>;
using ColVecC = std::conditional_t<IsDecimalNumber<ResultType>, ColumnDecimal<ResultType>, ColumnVector<ResultType>>;
using ColVecA = ColumnVectorOrDecimal<A>;
using ColVecC = ColumnVectorOrDecimal<ResultType>;
using ArrayA = typename ColVecA::Container;
using ArrayC = typename ColVecC::Container;

View File

@ -764,7 +764,7 @@ template <typename FromDataType, typename Name>
struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType, DataTypeString>, DataTypeString>, Name, ConvertDefaultBehaviorTag>
{
using FromFieldType = typename FromDataType::FieldType;
using ColVecType = std::conditional_t<IsDecimalNumber<FromFieldType>, ColumnDecimal<FromFieldType>, ColumnVector<FromFieldType>>;
using ColVecType = ColumnVectorOrDecimal<FromFieldType>;
static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/)
{
@ -1699,9 +1699,9 @@ private:
using RightT = typename RightDataType::FieldType;
static constexpr bool bad_left =
IsDecimalNumber<LeftT> || std::is_floating_point_v<LeftT> || is_big_int_v<LeftT> || is_signed_v<LeftT>;
is_decimal<LeftT> || std::is_floating_point_v<LeftT> || is_big_int_v<LeftT> || is_signed_v<LeftT>;
static constexpr bool bad_right =
IsDecimalNumber<RightT> || std::is_floating_point_v<RightT> || is_big_int_v<RightT> || is_signed_v<RightT>;
is_decimal<RightT> || std::is_floating_point_v<RightT> || is_big_int_v<RightT> || is_signed_v<RightT>;
/// Disallow int vs UUID conversion (but support int vs UInt128 conversion)
if constexpr ((bad_left && std::is_same_v<RightDataType, DataTypeUUID>) ||

View File

@ -635,7 +635,7 @@ private:
template <typename FromType>
ColumnPtr executeType(const ColumnsWithTypeAndName & arguments) const
{
using ColVecType = std::conditional_t<IsDecimalNumber<FromType>, ColumnDecimal<FromType>, ColumnVector<FromType>>;
using ColVecType = ColumnVectorOrDecimal<FromType>;
if (const ColVecType * col_from = checkAndGetColumn<ColVecType>(arguments[0].column.get()))
{
@ -762,7 +762,7 @@ private:
template <typename FromType, bool first>
void executeIntType(const IColumn * column, typename ColumnVector<ToType>::Container & vec_to) const
{
using ColVecType = std::conditional_t<IsDecimalNumber<FromType>, ColumnDecimal<FromType>, ColumnVector<FromType>>;
using ColVecType = ColumnVectorOrDecimal<FromType>;
if (const ColVecType * col_from = checkAndGetColumn<ColVecType>(column))
{
@ -819,7 +819,7 @@ private:
template <typename FromType, bool first>
void executeBigIntType(const IColumn * column, typename ColumnVector<ToType>::Container & vec_to) const
{
using ColVecType = std::conditional_t<IsDecimalNumber<FromType>, ColumnDecimal<FromType>, ColumnVector<FromType>>;
using ColVecType = ColumnVectorOrDecimal<FromType>;
if (const ColVecType * col_from = checkAndGetColumn<ColVecType>(column))
{

View File

@ -522,7 +522,7 @@ public:
else if (!accurate::convertNumeric(element.getDouble(), value))
return false;
}
else if (element.isBool() && is_integer_v<NumberType> && convert_bool_to_integer)
else if (element.isBool() && is_integer<NumberType> && convert_bool_to_integer)
{
value = static_cast<NumberType>(element.getBool());
}

View File

@ -315,11 +315,11 @@ template <typename T, RoundingMode rounding_mode, ScaleMode scale_mode>
struct FloatRoundingImpl
{
private:
static_assert(!IsDecimalNumber<T>);
static_assert(!is_decimal<T>);
using Op = FloatRoundingComputation<T, rounding_mode, scale_mode>;
using Data = std::array<T, Op::data_count>;
using ColumnType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
using ColumnType = ColumnVector<T>;
using Container = typename ColumnType::Container;
public:
@ -413,12 +413,10 @@ public:
};
template <typename T, RoundingMode rounding_mode, TieBreakingMode tie_breaking_mode>
template <is_decimal T, RoundingMode rounding_mode, TieBreakingMode tie_breaking_mode>
class DecimalRoundingImpl
{
private:
static_assert(IsDecimalNumber<T>);
using NativeType = typename T::NativeType;
using Op = IntegerRoundingComputation<NativeType, rounding_mode, ScaleMode::Negative, tie_breaking_mode>;
using Container = typename ColumnDecimal<T>::Container;
@ -507,7 +505,7 @@ public:
{
if constexpr (is_arithmetic_v<T>)
return apply(checkAndGetColumn<ColumnVector<T>>(column), scale_arg);
else if constexpr (IsDecimalNumber<T>)
else if constexpr (is_decimal<T>)
return apply(checkAndGetColumn<ColumnDecimal<T>>(column), scale_arg);
}
};

View File

@ -34,7 +34,7 @@ void writeSlice(const NumericArraySlice<T> & slice, NumericArraySink<T> & sink)
template <typename T, typename U>
void writeSlice(const NumericArraySlice<T> & slice, NumericArraySink<U> & sink)
{
using NativeU = typename NativeType<U>::Type;
using NativeU = NativeType<U>;
sink.elements.resize(sink.current_offset + slice.size);
for (size_t i = 0; i < slice.size; ++i)
@ -42,9 +42,9 @@ void writeSlice(const NumericArraySlice<T> & slice, NumericArraySink<U> & sink)
const auto & src = slice.data[i];
auto & dst = sink.elements[sink.current_offset];
if constexpr (OverBigInt<T> || OverBigInt<U>)
if constexpr (is_over_big_int<T> || is_over_big_int<U>)
{
if constexpr (IsDecimalNumber<T>)
if constexpr (is_decimal<T>)
dst = static_cast<NativeU>(src.value);
else
dst = static_cast<NativeU>(src);
@ -99,7 +99,7 @@ inline ALWAYS_INLINE void writeSlice(const NumericArraySlice<T> & slice, Generic
{
for (size_t i = 0; i < slice.size; ++i)
{
if constexpr (IsDecimalNumber<T>)
if constexpr (is_decimal<T>)
{
DecimalField field(T(slice.data[i]), 0); /// TODO: Decimal scale
sink.elements.insert(field);
@ -558,9 +558,9 @@ bool sliceEqualElements(const NumericArraySlice<T> & first [[maybe_unused]],
size_t second_ind [[maybe_unused]])
{
/// TODO: Decimal scale
if constexpr (IsDecimalNumber<T> && IsDecimalNumber<U>)
if constexpr (is_decimal<T> && is_decimal<U>)
return accurate::equalsOp(first.data[first_ind].value, second.data[second_ind].value);
else if constexpr (IsDecimalNumber<T> || IsDecimalNumber<U>)
else if constexpr (is_decimal<T> || is_decimal<U>)
return false;
else
return accurate::equalsOp(first.data[first_ind], second.data[second_ind]);
@ -588,7 +588,7 @@ bool insliceEqualElements(const NumericArraySlice<T> & first [[maybe_unused]],
size_t first_ind [[maybe_unused]],
size_t second_ind [[maybe_unused]])
{
if constexpr (IsDecimalNumber<T>)
if constexpr (is_decimal<T>)
return accurate::equalsOp(first.data[first_ind].value, first.data[second_ind].value);
else
return accurate::equalsOp(first.data[first_ind], first.data[second_ind]);

View File

@ -36,7 +36,7 @@ struct NullableValueSource;
template <typename T>
struct NumericArraySink : public ArraySinkImpl<NumericArraySink<T>>
{
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
using ColVecType = ColumnVectorOrDecimal<T>;
using CompatibleArraySource = NumericArraySource<T>;
using CompatibleValueSource = NumericValueSource<T>;

View File

@ -39,7 +39,7 @@ template <typename ArraySink> struct NullableArraySink;
template <typename T>
struct NumericArraySource : public ArraySourceImpl<NumericArraySource<T>>
{
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
using ColVecType = ColumnVectorOrDecimal<T>;
using Slice = NumericArraySlice<T>;
using Column = ColumnArray;
@ -720,7 +720,7 @@ template <typename T>
struct NumericValueSource : ValueSourceImpl<NumericValueSource<T>>
{
using Slice = NumericValueSlice<T>;
using Column = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
using Column = ColumnVectorOrDecimal<T>;
using SinkType = NumericArraySink<T>;

View File

@ -18,7 +18,7 @@ struct ArraySinkCreator<Type, Types...>
{
static std::unique_ptr<IArraySink> create(IColumn & values, ColumnArray::Offsets & offsets, size_t column_size)
{
using ColVecType = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
using ColVecType = ColumnVectorOrDecimal<Type>;
IColumn * not_null_values = &values;
bool is_nullable = false;

View File

@ -18,7 +18,7 @@ struct ArraySourceCreator<Type, Types...>
{
static std::unique_ptr<IArraySource> create(const ColumnArray & col, const NullMap * null_map, bool is_const, size_t total_rows)
{
using ColVecType = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
using ColVecType = ColumnVectorOrDecimal<Type>;
if (typeid_cast<const ColVecType *>(&col.getData()))
{

View File

@ -18,7 +18,7 @@ struct ValueSourceCreator<Type, Types...>
{
static std::unique_ptr<IValueSource> create(const IColumn & col, const NullMap * null_map, bool is_const, size_t total_rows)
{
using ColVecType = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
using ColVecType = ColumnVectorOrDecimal<Type>;
if (auto column_vector = typeid_cast<const ColVecType *>(&col))
{

View File

@ -38,14 +38,14 @@ namespace DB
* integral type which should be at least 32 bits wide, and
* should preferably signed.
*/
template <typename T, std::enable_if_t<is_integer_v<T>> * = nullptr>
template <typename T, std::enable_if_t<is_integer<T>> * = nullptr>
GregorianDate(T mjd);
/** Convert to Modified Julian Day. The type T is an integral type
* which should be at least 32 bits wide, and should preferably
* signed.
*/
template <typename T, std::enable_if_t<is_integer_v<T>> * = nullptr>
template <typename T, std::enable_if_t<is_integer<T>> * = nullptr>
T toModifiedJulianDay() const;
/** Write the date in text form 'YYYY-MM-DD' to a buffer.
@ -90,14 +90,14 @@ namespace DB
* integral type which should be at least 32 bits wide, and
* should preferably signed.
*/
template <typename T, std::enable_if_t<is_integer_v<T>> * = nullptr>
template <typename T, std::enable_if_t<is_integer<T>> * = nullptr>
OrdinalDate(T mjd);
/** Convert to Modified Julian Day. The type T is an integral
* type which should be at least 32 bits wide, and should
* preferably be signed.
*/
template <typename T, std::enable_if_t<is_integer_v<T>> * = nullptr>
template <typename T, std::enable_if_t<is_integer<T>> * = nullptr>
T toModifiedJulianDay() const noexcept;
YearT year() const noexcept
@ -259,7 +259,7 @@ namespace DB
}
template <typename YearT>
template <typename T, std::enable_if_t<is_integer_v<T>> *>
template <typename T, std::enable_if_t<is_integer<T>> *>
GregorianDate<YearT>::GregorianDate(T mjd)
{
const OrdinalDate<YearT> ord(mjd);
@ -270,7 +270,7 @@ namespace DB
}
template <typename YearT>
template <typename T, std::enable_if_t<is_integer_v<T>> *>
template <typename T, std::enable_if_t<is_integer<T>> *>
T GregorianDate<YearT>::toModifiedJulianDay() const
{
const MonthDay md(month_, day_of_month_);
@ -332,7 +332,7 @@ namespace DB
}
template <typename YearT>
template <typename T, std::enable_if_t<is_integer_v<T>> *>
template <typename T, std::enable_if_t<is_integer<T>> *>
OrdinalDate<YearT>::OrdinalDate(T mjd)
{
const auto a = mjd + 678575;
@ -348,7 +348,7 @@ namespace DB
}
template <typename YearT>
template <typename T, std::enable_if_t<is_integer_v<T>> *>
template <typename T, std::enable_if_t<is_integer<T>> *>
T OrdinalDate<YearT>::toModifiedJulianDay() const noexcept
{
const auto y = year_ - 1;

View File

@ -9,18 +9,18 @@ namespace DB
template <typename A>
struct AbsImpl
{
using ResultType = std::conditional_t<IsDecimalNumber<A>, A, typename NumberTraits::ResultOfAbs<A>::Type>;
using ResultType = std::conditional_t<is_decimal<A>, A, typename NumberTraits::ResultOfAbs<A>::Type>;
static const constexpr bool allow_fixed_string = false;
static inline NO_SANITIZE_UNDEFINED ResultType apply(A a)
{
if constexpr (IsDecimalNumber<A>)
if constexpr (is_decimal<A>)
return a < A(0) ? A(-a) : a;
else if constexpr (is_big_int_v<A> && is_signed_v<A>)
return (a < 0) ? -a : a;
else if constexpr (is_integer_v<A> && is_signed_v<A>)
else if constexpr (is_integer<A> && is_signed_v<A>)
return a < 0 ? static_cast<ResultType>(~a) + 1 : static_cast<ResultType>(a);
else if constexpr (is_integer_v<A> && is_unsigned_v<A>)
else if constexpr (is_integer<A> && is_unsigned_v<A>)
return static_cast<ResultType>(a);
else if constexpr (std::is_floating_point_v<A>)
return static_cast<ResultType>(std::abs(a));

View File

@ -71,7 +71,7 @@ struct ArrayAggregateResultImpl<ArrayElement, AggregateOperation::sum>
std::conditional_t<std::is_same_v<ArrayElement, UInt128>, UInt128,
std::conditional_t<std::is_same_v<ArrayElement, Int256>, Int256,
std::conditional_t<std::is_same_v<ArrayElement, UInt256>, UInt256,
std::conditional_t<IsDecimalNumber<ArrayElement>, Decimal128,
std::conditional_t<is_decimal<ArrayElement>, Decimal128,
std::conditional_t<std::is_floating_point_v<ArrayElement>, Float64,
std::conditional_t<std::is_signed_v<ArrayElement>, Int64,
UInt64>>>>>>>;
@ -135,8 +135,8 @@ struct ArrayAggregateImpl
static NO_SANITIZE_UNDEFINED bool executeType(const ColumnPtr & mapped, const ColumnArray::Offsets & offsets, ColumnPtr & res_ptr)
{
using ResultType = ArrayAggregateResult<Element, aggregate_operation>;
using ColVecType = std::conditional_t<IsDecimalNumber<Element>, ColumnDecimal<Element>, ColumnVector<Element>>;
using ColVecResultType = std::conditional_t<IsDecimalNumber<ResultType>, ColumnDecimal<ResultType>, ColumnVector<ResultType>>;
using ColVecType = ColumnVectorOrDecimal<Element>;
using ColVecResultType = ColumnVectorOrDecimal<ResultType>;
/// For average and product of array we return Float64 as result, but we want to keep precision
/// so we convert to Float64 as last step, but intermediate value is represented as result of sum operation
@ -160,7 +160,7 @@ struct ArrayAggregateImpl
const auto & data = checkAndGetColumn<ColVecType>(&column_const->getDataColumn())->getData();
typename ColVecResultType::MutablePtr res_column;
if constexpr (IsDecimalNumber<Element>)
if constexpr (is_decimal<Element>)
res_column = ColVecResultType::create(offsets.size(), data.getScale());
else
res_column = ColVecResultType::create(offsets.size());
@ -183,7 +183,7 @@ struct ArrayAggregateImpl
}
else if constexpr (aggregate_operation == AggregateOperation::average)
{
if constexpr (IsDecimalNumber<Element>)
if constexpr (is_decimal<Element>)
{
res[i] = DecimalUtils::convertTo<ResultType>(x, data.getScale());
}
@ -197,7 +197,7 @@ struct ArrayAggregateImpl
size_t array_size = offsets[i] - pos;
AggregationType product = x;
if constexpr (IsDecimalNumber<Element>)
if constexpr (is_decimal<Element>)
{
using T = decltype(x.value);
T x_val = x.value;
@ -235,7 +235,7 @@ struct ArrayAggregateImpl
const auto & data = column->getData();
typename ColVecResultType::MutablePtr res_column;
if constexpr (IsDecimalNumber<Element>)
if constexpr (is_decimal<Element>)
res_column = ColVecResultType::create(offsets.size(), data.getScale());
else
res_column = ColVecResultType::create(offsets.size());
@ -250,7 +250,7 @@ struct ArrayAggregateImpl
/// Array is empty
if (offsets[i] == pos)
{
if constexpr (IsDecimalNumber<AggregationType>)
if constexpr (is_decimal<AggregationType>)
res[i] = aggregate_value.value;
else
res[i] = aggregate_value;
@ -286,7 +286,7 @@ struct ArrayAggregateImpl
}
else if constexpr (aggregate_operation == AggregateOperation::product)
{
if constexpr (IsDecimalNumber<Element>)
if constexpr (is_decimal<Element>)
{
using AggregateValueDecimalUnderlyingValue = decltype(aggregate_value.value);
AggregateValueDecimalUnderlyingValue current_aggregate_value = aggregate_value.value;
@ -306,7 +306,7 @@ struct ArrayAggregateImpl
if constexpr (aggregate_operation == AggregateOperation::average)
{
if constexpr (IsDecimalNumber<Element>)
if constexpr (is_decimal<Element>)
{
aggregate_value = aggregate_value / AggregationType(count);
res[i] = DecimalUtils::convertTo<ResultType>(aggregate_value, data.getScale());
@ -316,7 +316,7 @@ struct ArrayAggregateImpl
res[i] = static_cast<ResultType>(aggregate_value) / count;
}
}
else if constexpr (aggregate_operation == AggregateOperation::product && IsDecimalNumber<Element>)
else if constexpr (aggregate_operation == AggregateOperation::product && is_decimal<Element>)
{
auto result_scale = data.getScale() * count;

View File

@ -29,7 +29,7 @@ struct ArrayCompactImpl
template <typename T>
static bool executeType(const ColumnPtr & mapped, const ColumnArray & array, ColumnPtr & res_ptr)
{
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
using ColVecType = ColumnVectorOrDecimal<T>;
const ColVecType * src_values_column = checkAndGetColumn<ColVecType>(mapped.get());
@ -40,7 +40,7 @@ struct ArrayCompactImpl
const typename ColVecType::Container & src_values = src_values_column->getData();
typename ColVecType::MutablePtr res_values_column;
if constexpr (IsDecimalNumber<T>)
if constexpr (is_decimal<T>)
res_values_column = ColVecType::create(src_values.size(), src_values.getScale());
else
res_values_column = ColVecType::create(src_values.size());

View File

@ -83,8 +83,8 @@ struct ArrayCumSumImpl
template <typename Element, typename Result>
static bool executeType(const ColumnPtr & mapped, const ColumnArray & array, ColumnPtr & res_ptr)
{
using ColVecType = std::conditional_t<IsDecimalNumber<Element>, ColumnDecimal<Element>, ColumnVector<Element>>;
using ColVecResult = std::conditional_t<IsDecimalNumber<Result>, ColumnDecimal<Result>, ColumnVector<Result>>;
using ColVecType = ColumnVectorOrDecimal<Element>;
using ColVecResult = ColumnVectorOrDecimal<Result>;
const ColVecType * column = checkAndGetColumn<ColVecType>(&*mapped);
@ -99,7 +99,7 @@ struct ArrayCumSumImpl
const IColumn::Offsets & offsets = array.getOffsets();
typename ColVecResult::MutablePtr res_nested;
if constexpr (IsDecimalNumber<Element>)
if constexpr (is_decimal<Element>)
{
const typename ColVecType::Container & data =
checkAndGetColumn<ColVecType>(&column_const->getDataColumn())->getData();
@ -119,7 +119,7 @@ struct ArrayCumSumImpl
const IColumn::Offsets & offsets = array.getOffsets();
typename ColVecResult::MutablePtr res_nested;
if constexpr (IsDecimalNumber<Element>)
if constexpr (is_decimal<Element>)
res_nested = ColVecResult::create(0, data.getScale());
else
res_nested = ColVecResult::create();

View File

@ -71,8 +71,8 @@ struct ArrayCumSumNonNegativeImpl
template <typename Element, typename Result>
static bool executeType(const ColumnPtr & mapped, const ColumnArray & array, ColumnPtr & res_ptr)
{
using ColVecType = std::conditional_t<IsDecimalNumber<Element>, ColumnDecimal<Element>, ColumnVector<Element>>;
using ColVecResult = std::conditional_t<IsDecimalNumber<Result>, ColumnDecimal<Result>, ColumnVector<Result>>;
using ColVecType = ColumnVectorOrDecimal<Element>;
using ColVecResult = ColumnVectorOrDecimal<Result>;
const ColVecType * column = checkAndGetColumn<ColVecType>(&*mapped);
@ -83,7 +83,7 @@ struct ArrayCumSumNonNegativeImpl
const typename ColVecType::Container & data = column->getData();
typename ColVecResult::MutablePtr res_nested;
if constexpr (IsDecimalNumber<Element>)
if constexpr (is_decimal<Element>)
res_nested = ColVecResult::create(0, data.getScale());
else
res_nested = ColVecResult::create();

View File

@ -65,7 +65,7 @@ struct ArrayDifferenceImpl
{
Element curr = src[pos];
if constexpr (IsDecimalNumber<Element>)
if constexpr (is_decimal<Element>)
{
using ResultNativeType = typename Result::NativeType;
@ -93,8 +93,8 @@ struct ArrayDifferenceImpl
template <typename Element, typename Result>
static bool executeType(const ColumnPtr & mapped, const ColumnArray & array, ColumnPtr & res_ptr)
{
using ColVecType = std::conditional_t<IsDecimalNumber<Element>, ColumnDecimal<Element>, ColumnVector<Element>>;
using ColVecResult = std::conditional_t<IsDecimalNumber<Result>, ColumnDecimal<Result>, ColumnVector<Result>>;
using ColVecType = ColumnVectorOrDecimal<Element>;
using ColVecResult = ColumnVectorOrDecimal<Result>;
const ColVecType * column = checkAndGetColumn<ColVecType>(&*mapped);
@ -105,7 +105,7 @@ struct ArrayDifferenceImpl
const typename ColVecType::Container & data = column->getData();
typename ColVecResult::MutablePtr res_nested;
if constexpr (IsDecimalNumber<Element>)
if constexpr (is_decimal<Element>)
res_nested = ColVecResult::create(0, data.getScale());
else
res_nested = ColVecResult::create();

View File

@ -25,7 +25,7 @@ struct BitWrapperFuncImpl
{
// Should be a logical error, but this function is callable from SQL.
// Need to investigate this.
if constexpr (!is_integer_v<A>)
if constexpr (!is_integer<A>)
throw DB::Exception("It's a bug! Only integer types are supported by __bitWrapperFunc.", ErrorCodes::BAD_ARGUMENTS);
return a == 0 ? static_cast<ResultType>(0b10) : static_cast<ResultType >(0b1);
}

View File

@ -61,7 +61,7 @@ public:
{
using Types = std::decay_t<decltype(types)>;
using Type = typename Types::RightType;
using ColVecType = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
using ColVecType = ColumnVectorOrDecimal<Type>;
if (const ColVecType * col_vec = checkAndGetColumn<ColVecType>(src_column.column.get()))
{
@ -84,7 +84,7 @@ private:
template <typename T, typename ColVecType>
static void execute(const ColVecType & col, ColumnUInt8 & result_column, size_t rows_count)
{
using NativeT = typename NativeType<T>::Type;
using NativeT = NativeType<T>;
const auto & src_data = col.getData();
auto & dst_data = result_column.getData();
@ -92,7 +92,7 @@ private:
for (size_t i = 0; i < rows_count; ++i)
{
if constexpr (IsDecimalNumber<T>)
if constexpr (is_decimal<T>)
dst_data[i] = digits<NativeT>(src_data[i].value);
else
dst_data[i] = digits<NativeT>(src_data[i]);
@ -102,7 +102,7 @@ private:
template <typename T>
static UInt32 digits(T value)
{
static_assert(!IsDecimalNumber<T>);
static_assert(!is_decimal<T>);
using DivT = std::conditional_t<is_signed_v<T>, Int32, UInt32>;
UInt32 res = 0;

View File

@ -221,7 +221,7 @@ private:
template <typename T0, typename T1>
static UInt32 decimalScale(const ColumnsWithTypeAndName & arguments [[maybe_unused]])
{
if constexpr (IsDecimalNumber<T0> && IsDecimalNumber<T1>)
if constexpr (is_decimal<T0> && is_decimal<T1>)
{
UInt32 left_scale = getDecimalScale(*arguments[1].type);
UInt32 right_scale = getDecimalScale(*arguments[2].type);
@ -416,8 +416,8 @@ private:
ColumnPtr executeTyped(
const ColumnUInt8 * cond_col, const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const
{
using ColVecT0 = std::conditional_t<IsDecimalNumber<T0>, ColumnDecimal<T0>, ColumnVector<T0>>;
using ColVecT1 = std::conditional_t<IsDecimalNumber<T1>, ColumnDecimal<T1>, ColumnVector<T1>>;
using ColVecT0 = ColumnVectorOrDecimal<T0>;
using ColVecT1 = ColumnVectorOrDecimal<T1>;
const IColumn * col_left_untyped = arguments[1].column.get();

View File

@ -128,10 +128,9 @@ private:
dst_data[i] = outOfDigits<T>(src_data[i], precision);
}
template <typename T>
template <is_decimal T>
static bool outOfDigits(T dec, UInt32 precision)
{
static_assert(IsDecimalNumber<T>);
using NativeT = typename T::NativeType;
if (precision > DecimalUtils::max_precision<T>)

View File

@ -8,7 +8,7 @@ namespace DB
template <typename A>
struct NegateImpl
{
using ResultType = std::conditional_t<IsDecimalNumber<A>, A, typename NumberTraits::ResultOfNegate<A>::Type>;
using ResultType = std::conditional_t<is_decimal<A>, A, typename NumberTraits::ResultOfNegate<A>::Type>;
static constexpr const bool allow_fixed_string = false;
static inline NO_SANITIZE_UNDEFINED ResultType apply(A a)

View File

@ -217,7 +217,7 @@ public:
using From = typename FromType::FieldType;
using To = typename ToType::FieldType;
using FromColumnType = std::conditional_t<IsDecimalNumber<From>, ColumnDecimal<From>, ColumnVector<From>>;
using FromColumnType = ColumnVectorOrDecimal<From>;
const auto * column_from = assert_cast<const FromColumnType*>(arguments[0].column.get());

View File

@ -13,7 +13,7 @@ struct SignImpl
static inline NO_SANITIZE_UNDEFINED ResultType apply(A a)
{
if constexpr (IsDecimalNumber<A> || std::is_floating_point_v<A>)
if constexpr (is_decimal<A> || std::is_floating_point_v<A>)
return a < A(0) ? -1 : a == A(0) ? 0 : 1;
else if constexpr (is_signed_v<A>)
return a < 0 ? -1 : a == 0 ? 0 : 1;

View File

@ -13,7 +13,7 @@
#include <common/LocalDateTime.h>
#include <common/StringRef.h>
#include <common/arithmeticOverflow.h>
#include <common/Concepts.h>
#include <common/unit.h>
#include <Core/Types.h>
#include <Core/DecimalFunctions.h>
@ -36,10 +36,7 @@
#include <double-conversion/double-conversion.h>
/// 1 GiB
#define DEFAULT_MAX_STRING_SIZE (1ULL << 30)
static constexpr auto DEFAULT_MAX_STRING_SIZE = 1_GiB;
namespace DB
{
@ -383,7 +380,7 @@ end:
template <ReadIntTextCheckOverflow check_overflow = ReadIntTextCheckOverflow::DO_NOT_CHECK_OVERFLOW, typename T>
void readIntText(T & x, ReadBuffer & buf)
{
if constexpr (IsDecimalNumber<T>)
if constexpr (is_decimal<T>)
{
readIntText<check_overflow>(x.value, buf);
}
@ -1154,12 +1151,10 @@ inline bool tryParse(T & res, const char * data, size_t size)
}
template <typename T>
inline std::enable_if_t<!is_integer_v<T>, void>
readTextWithSizeSuffix(T & x, ReadBuffer & buf) { readText(x, buf); }
inline void readTextWithSizeSuffix(T & x, ReadBuffer & buf) { readText(x, buf); }
template <typename T>
inline std::enable_if_t<is_integer_v<T>, void>
readTextWithSizeSuffix(T & x, ReadBuffer & buf)
template <is_integer T>
inline void readTextWithSizeSuffix(T & x, ReadBuffer & buf)
{
readIntText(x, buf);
if (buf.eof())

View File

@ -5,7 +5,6 @@
#include <limits>
#include <algorithm>
#include <iterator>
#include <concepts>
#include <pcg-random/pcg_random.hpp>
@ -15,6 +14,7 @@
#include <common/find_symbols.h>
#include <common/StringRef.h>
#include <common/DecomposedFloat.h>
#include <common/EnumReflection.h>
#include <Core/DecimalFunctions.h>
#include <Core/Types.h>
@ -31,8 +31,6 @@
#include <IO/DoubleConverter.h>
#include <IO/WriteBufferFromString.h>
#include <common/EnumReflection.h>
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wunused-parameter"
@ -45,6 +43,7 @@
#include <Formats/FormatSettings.h>
namespace DB
{
@ -378,7 +377,7 @@ void writeJSONNumber(T x, WriteBuffer & ostr, const FormatSettings & settings)
{
bool is_finite = isFinite(x);
const bool need_quote = (is_integer_v<T> && (sizeof(T) >= 8) && settings.json.quote_64bit_integers)
const bool need_quote = (is_integer<T> && (sizeof(T) >= 8) && settings.json.quote_64bit_integers)
|| (settings.json.quote_denormals && !is_finite);
if (need_quote)

View File

@ -7,6 +7,7 @@
#include <Columns/IColumn.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <common/EnumReflection.h>
namespace DB
@ -46,7 +47,6 @@ void callWithType(TypeIndex which, F && f)
__builtin_unreachable();
}
}
@ -69,7 +69,7 @@ void AsofRowRefs::insert(TypeIndex type, const IColumn & asof_column, const Bloc
using T = std::decay_t<decltype(t)>;
using LookupPtr = typename Entry<T>::LookupPtr;
using ColumnType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
using ColumnType = ColumnVectorOrDecimal<T>;
const auto & column = typeid_cast<const ColumnType &>(asof_column);
T key = column.getElement(row_num);
@ -93,7 +93,7 @@ const RowRef * AsofRowRefs::findAsof(TypeIndex type, ASOF::Inequality inequality
using EntryType = Entry<T>;
using LookupPtr = typename EntryType::LookupPtr;
using ColumnType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
using ColumnType = ColumnVectorOrDecimal<T>;
const auto & column = typeid_cast<const ColumnType &>(asof_column);
T key = column.getElement(row_num);
auto & typed_lookup = std::get<LookupPtr>(lookups);

View File

@ -23,7 +23,7 @@ struct RowRef
const Block * block = nullptr;
SizeT row_num = 0;
RowRef() {}
RowRef() = default;
RowRef(const Block * block_, size_t row_num_) : block(block_), row_num(row_num_) {}
};
@ -232,7 +232,7 @@ public:
Entry<DateTime64>::LookupPtr>;
AsofRowRefs() {}
AsofRowRefs(TypeIndex t);
AsofRowRefs(TypeIndex type);
static std::optional<TypeIndex> getTypeSize(const IColumn & asof_column, size_t & type_size);

View File

@ -64,7 +64,7 @@ static void mixNumberColumns(
if constexpr (!std::is_same_v<DataType, DataTypeString> && !std::is_same_v<DataType, DataTypeFixedString>)
{
using FieldType = typename DataType::FieldType;
using ColVecType = std::conditional_t<IsDecimalNumber<FieldType>, ColumnDecimal<FieldType>, ColumnVector<FieldType>>;
using ColVecType = ColumnVectorOrDecimal<FieldType>;
auto col_read = typeid_cast<ColVecType *>(column_mixed.get());
if (!col_read)