提交 d6a85cc1 编写于 作者: R robot-clickhouse

Backport #19672 to 20.12: Remove old code for BigInt.

上级 1ec12aaa
......@@ -104,8 +104,3 @@ template <> struct is_big_int<wUInt256> { static constexpr bool value = true; };
template <typename T>
inline constexpr bool is_big_int_v = is_big_int<T>::value;
template <typename To, typename From>
inline To bigint_cast(const From & x [[maybe_unused]])
{
return static_cast<To>(x);
}
......@@ -4,7 +4,6 @@
#include <Common/assert_cast.h>
#include <Common/WeakHash.h>
#include <Common/HashTable/Hash.h>
#include <Core/BigInt.h>
#include <common/unaligned.h>
#include <common/sort.h>
......
......@@ -37,33 +37,16 @@ namespace ErrorCodes
template <typename T>
StringRef ColumnVector<T>::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
{
if constexpr (is_big_int_v<T>)
{
static constexpr size_t bytesize = BigInt<T>::size;
char * pos = arena.allocContinue(bytesize, begin);
return BigInt<T>::serialize(data[n], pos);
}
else
{
auto * pos = arena.allocContinue(sizeof(T), begin);
unalignedStore<T>(pos, data[n]);
return StringRef(pos, sizeof(T));
}
auto * pos = arena.allocContinue(sizeof(T), begin);
unalignedStore<T>(pos, data[n]);
return StringRef(pos, sizeof(T));
}
template <typename T>
const char * ColumnVector<T>::deserializeAndInsertFromArena(const char * pos)
{
if constexpr (is_big_int_v<T>)
{
data.emplace_back(BigInt<T>::deserialize(pos));
return pos + BigInt<T>::size;
}
else
{
data.emplace_back(unalignedLoad<T>(pos));
return pos + sizeof(T);
}
data.emplace_back(unalignedLoad<T>(pos));
return pos + sizeof(T);
}
template <typename T>
......@@ -299,18 +282,10 @@ MutableColumnPtr ColumnVector<T>::cloneResized(size_t size) const
new_col.data.resize(size);
size_t count = std::min(this->size(), size);
if constexpr (is_POD)
{
memcpy(new_col.data.data(), data.data(), count * sizeof(data[0]));
memcpy(new_col.data.data(), data.data(), count * sizeof(data[0]));
if (size > count)
memset(static_cast<void *>(&new_col.data[count]), static_cast<int>(ValueType()), (size - count) * sizeof(ValueType));
}
else
{
for (size_t i = 0; i < count; i++)
new_col.data[i] = data[i];
}
if (size > count)
memset(static_cast<void *>(&new_col.data[count]), static_cast<int>(ValueType()), (size - count) * sizeof(ValueType));
}
return res;
......@@ -348,15 +323,7 @@ void ColumnVector<T>::insertRangeFrom(const IColumn & src, size_t start, size_t
size_t old_size = data.size();
data.resize(old_size + length);
if constexpr (is_POD)
{
memcpy(data.data() + old_size, &src_vec.data[start], length * sizeof(data[0]));
}
else
{
for (size_t i = 0; i < length; i++)
data[old_size + i] = src_vec.data[start + i];
}
memcpy(data.data() + old_size, &src_vec.data[start], length * sizeof(data[0]));
}
template <typename T>
......@@ -372,70 +339,52 @@ ColumnPtr ColumnVector<T>::filter(const IColumn::Filter & filt, ssize_t result_s
if (result_size_hint)
res_data.reserve(result_size_hint > 0 ? result_size_hint : size);
if constexpr (is_POD)
{
const UInt8 * filt_pos = filt.data();
const UInt8 * filt_end = filt_pos + size;
const T * data_pos = data.data();
const UInt8 * filt_pos = filt.data();
const UInt8 * filt_end = filt_pos + size;
const T * data_pos = data.data();
#ifdef __SSE2__
/** A slightly more optimized version.
* Based on the assumption that often pieces of consecutive values
* completely pass or do not pass the filter.
* Therefore, we will optimistically check the parts of `SIMD_BYTES` values.
*/
/** A slightly more optimized version.
* Based on the assumption that often pieces of consecutive values
* completely pass or do not pass the filter.
* Therefore, we will optimistically check the parts of `SIMD_BYTES` values.
*/
static constexpr size_t SIMD_BYTES = 16;
const __m128i zero16 = _mm_setzero_si128();
const UInt8 * filt_end_sse = filt_pos + size / SIMD_BYTES * SIMD_BYTES;
static constexpr size_t SIMD_BYTES = 16;
const __m128i zero16 = _mm_setzero_si128();
const UInt8 * filt_end_sse = filt_pos + size / SIMD_BYTES * SIMD_BYTES;
while (filt_pos < filt_end_sse)
{
int mask = _mm_movemask_epi8(_mm_cmpgt_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i *>(filt_pos)), zero16));
if (0 == mask)
{
/// Nothing is inserted.
}
else if (0xFFFF == mask)
{
res_data.insert(data_pos, data_pos + SIMD_BYTES);
}
else
{
for (size_t i = 0; i < SIMD_BYTES; ++i)
if (filt_pos[i])
res_data.push_back(data_pos[i]);
}
while (filt_pos < filt_end_sse)
{
int mask = _mm_movemask_epi8(_mm_cmpgt_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i *>(filt_pos)), zero16));
filt_pos += SIMD_BYTES;
data_pos += SIMD_BYTES;
if (0 == mask)
{
/// Nothing is inserted.
}
#endif
while (filt_pos < filt_end)
else if (0xFFFF == mask)
{
if (*filt_pos)
res_data.push_back(*data_pos);
++filt_pos;
++data_pos;
res_data.insert(data_pos, data_pos + SIMD_BYTES);
}
else
{
for (size_t i = 0; i < SIMD_BYTES; ++i)
if (filt_pos[i])
res_data.push_back(data_pos[i]);
}
filt_pos += SIMD_BYTES;
data_pos += SIMD_BYTES;
}
else
{
const auto * filt_pos = filt.begin();
const auto * filt_end = filt.end();
auto data_pos = data.begin();
#endif
while (filt_pos < filt_end)
{
if (*filt_pos)
res_data.push_back(*data_pos);
while (filt_pos < filt_end)
{
if (*filt_pos)
res_data.push_back(*data_pos);
++filt_pos;
++data_pos;
}
++filt_pos;
++data_pos;
}
return res;
......
......@@ -6,7 +6,6 @@
#include <Columns/ColumnVectorHelper.h>
#include <common/unaligned.h>
#include <Core/Field.h>
#include <Core/BigInt.h>
#include <Common/assert_cast.h>
......@@ -107,10 +106,7 @@ private:
public:
using ValueType = T;
static constexpr bool is_POD = !is_big_int_v<T>;
using Container = std::conditional_t<is_POD,
PaddedPODArray<ValueType>,
std::vector<ValueType>>;
using Container = PaddedPODArray<ValueType>;
private:
ColumnVector() {}
......@@ -136,10 +132,7 @@ public:
void insertData(const char * pos, size_t) override
{
if constexpr (is_POD)
data.emplace_back(unalignedLoad<T>(pos));
else
data.emplace_back(BigInt<T>::deserialize(pos));
data.emplace_back(unalignedLoad<T>(pos));
}
void insertDefault() override
......@@ -149,18 +142,12 @@ public:
void insertManyDefaults(size_t length) override
{
if constexpr (is_POD)
data.resize_fill(data.size() + length, T());
else
data.resize(data.size() + length, T());
data.resize_fill(data.size() + length, T());
}
void popBack(size_t n) override
{
if constexpr (is_POD)
data.resize_assume_reserved(data.size() - n);
else
data.resize(data.size() - n);
data.resize_assume_reserved(data.size() - n);
}
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
......@@ -180,16 +167,12 @@ public:
size_t allocatedBytes() const override
{
if constexpr (is_POD)
return data.allocated_bytes();
else
return data.capacity() * sizeof(data[0]);
return data.allocated_bytes();
}
void protect() override
{
if constexpr (is_POD)
data.protect();
data.protect();
}
void insertValue(const T value)
......
......@@ -204,7 +204,7 @@ public:
else if constexpr (std::is_same_v<T, UInt128>)
throw Exception("No conversion to old UInt128 from " + demangle(typeid(U).name()), ErrorCodes::NOT_IMPLEMENTED);
else
return bigint_cast<T>(x);
return static_cast<T>(x);
}
};
......
#pragma once
#include <common/types.h>
#include <Core/BigInt.h>
#include <Common/UInt128.h>
#include <common/unaligned.h>
......
......@@ -18,7 +18,7 @@
#include <string>
#include <type_traits>
#include <Core/Defines.h>
#include <Core/BigInt.h>
#define ROTL(x, b) static_cast<UInt64>(((x) << (b)) | ((x) >> (64 - (b))))
......@@ -136,23 +136,11 @@ public:
}
template <typename T>
std::enable_if_t<std::has_unique_object_representations_v<T>, void> update(const T & x)
{
update(reinterpret_cast<const char *>(&x), sizeof(x));
}
template <typename T>
std::enable_if_t<(std::is_floating_point_v<T> || std::is_same_v<T, CityHash_v1_0_2::uint128>), void> update(const T & x)
void update(const T & x)
{
update(reinterpret_cast<const char *>(&x), sizeof(x));
}
template <typename T>
std::enable_if_t<is_big_int_v<T> && !std::has_unique_object_representations_v<T>, void> update(const T & x)
{
update(DB::BigInt<T>::serialize(x));
}
void update(const std::string & x)
{
update(x.data(), x.length());
......@@ -205,27 +193,13 @@ inline UInt64 sipHash64(const char * data, const size_t size)
}
template <typename T>
std::enable_if_t<std::has_unique_object_representations_v<T>, UInt64> sipHash64(const T & x)
UInt64 sipHash64(const T & x)
{
SipHash hash;
hash.update(x);
return hash.get64();
}
template <typename T>
std::enable_if_t<(std::is_floating_point_v<T> || (is_big_int_v<T> && !std::has_unique_object_representations_v<T>)), UInt64> sipHash64(const T & x)
{
SipHash hash;
hash.update(x);
return hash.get64();
}
template <typename T>
std::enable_if_t<DB::IsDecimalNumber<T>, UInt64> sipHash64(const T & x)
{
return sipHash64(x.value);
}
inline UInt64 sipHash64(const std::string & s)
{
return sipHash64(s.data(), s.size());
......
#pragma once
#include <common/types.h>
#include <Core/Types.h>
#include <Compression/ICompressionCodec.h>
......
......@@ -93,7 +93,7 @@ using bool_if_gt_int_vs_uint = std::enable_if_t<is_gt_int_vs_uint<TInt, TUInt>,
template <typename TInt, typename TUInt>
inline bool_if_gt_int_vs_uint<TInt, TUInt> greaterOpTmpl(TInt a, TUInt b)
{
return bigint_cast<TInt>(a) > bigint_cast<TInt>(b);
return static_cast<TInt>(a) > static_cast<TInt>(b);
}
template <typename TInt, typename TUInt>
......@@ -101,19 +101,19 @@ inline bool_if_gt_int_vs_uint<TInt, TUInt> greaterOpTmpl(TUInt a, TInt b)
{
using CastA = std::conditional_t<is_big_int_v<TInt> && std::is_same_v<TUInt, DB::UInt128>, DB::UInt256, TInt>;
return bigint_cast<CastA>(a) > b;
return static_cast<CastA>(a) > b;
}
template <typename TInt, typename TUInt>
inline bool_if_gt_int_vs_uint<TInt, TUInt> equalsOpTmpl(TInt a, TUInt b)
{
return bigint_cast<TInt>(a) == bigint_cast<TInt>(b);
return static_cast<TInt>(a) == static_cast<TInt>(b);
}
template <typename TInt, typename TUInt>
inline bool_if_gt_int_vs_uint<TInt, TUInt> equalsOpTmpl(TUInt a, TInt b)
{
return bigint_cast<TInt>(a) == bigint_cast<TInt>(b);
return static_cast<TInt>(a) == static_cast<TInt>(b);
}
......@@ -196,7 +196,7 @@ inline bool_if_safe_conversion<A, B> greaterOp(A a, B b)
using CastB = std::conditional_t<is_big_int_v<A> && std::is_same_v<B, DB::UInt128>, A, CastB1>;
if constexpr (is_big_int_v<A> || is_big_int_v<B>)
return bigint_cast<CastA>(a) > bigint_cast<CastB>(b);
return static_cast<CastA>(a) > static_cast<CastB>(b);
else
return a > b;
}
......@@ -306,7 +306,7 @@ inline bool_if_safe_conversion<A, B> equalsOp(A a, B b)
{
using LargestType = std::conditional_t<(sizeof(A) > sizeof(B)) || ((sizeof(A) == sizeof(B)) && !std::is_same_v<A, DB::UInt128>), A, B>;
return bigint_cast<LargestType>(a) == bigint_cast<LargestType>(b);
return static_cast<LargestType>(a) == static_cast<LargestType>(b);
}
template <>
......@@ -429,7 +429,7 @@ inline bool_if_safe_conversion<A, B> notEqualsOp(A a, B b)
using CastB = std::conditional_t<is_big_int_v<A> && std::is_same_v<B, DB::UInt128>, A, CastB1>;
if constexpr (is_big_int_v<A> || is_big_int_v<B>)
return bigint_cast<CastA>(a) != bigint_cast<CastB>(b);
return static_cast<CastA>(a) != static_cast<CastB>(b);
else
return a != b;
}
......@@ -451,7 +451,7 @@ inline bool_if_safe_conversion<A, B> lessOp(A a, B b)
using CastB = std::conditional_t<is_big_int_v<A> && std::is_same_v<B, DB::UInt128>, A, CastB1>;
if constexpr (is_big_int_v<A> || is_big_int_v<B>)
return bigint_cast<CastA>(a) < bigint_cast<CastB>(b);
return static_cast<CastA>(a) < static_cast<CastB>(b);
else
return a < b;
}
......@@ -475,7 +475,7 @@ inline bool_if_safe_conversion<A, B> lessOrEqualsOp(A a, B b)
using CastB = std::conditional_t<is_big_int_v<A> && std::is_same_v<B, DB::UInt128>, A, CastB1>;
if constexpr (is_big_int_v<A> || is_big_int_v<B>)
return bigint_cast<CastA>(a) <= bigint_cast<CastB>(b);
return static_cast<CastA>(a) <= static_cast<CastB>(b);
else
return a <= b;
}
......@@ -499,7 +499,7 @@ inline bool_if_safe_conversion<A, B> greaterOrEqualsOp(A a, B b)
using CastB = std::conditional_t<is_big_int_v<A> && std::is_same_v<B, DB::UInt128>, A, CastB1>;
if constexpr (is_big_int_v<A> || is_big_int_v<B>)
return bigint_cast<CastA>(a) >= bigint_cast<CastB>(b);
return static_cast<CastA>(a) >= static_cast<CastB>(b);
else
return a >= b;
}
......
#pragma once
#include <common/StringRef.h>
#include <common/unaligned.h>
#include <Core/Types.h>
namespace DB
{
template <typename T>
struct BigInt
{
static_assert(sizeof(T) == 32);
static constexpr size_t size = 32;
static StringRef serialize(const T & x, char * pos)
{
unalignedStore<T>(pos, x);
return StringRef(pos, size);
}
static String serialize(const T & x)
{
String str(size, '\0');
serialize(x, str.data());
return str;
}
static T deserialize(const char * pos)
{
return unalignedLoad<T>(pos);
}
};
}
......@@ -233,9 +233,9 @@ private:
bool overflow = false;
if constexpr (sizeof(A) > sizeof(CompareInt))
overflow |= (bigint_cast<A>(x) != a);
overflow |= (static_cast<A>(x) != a);
if constexpr (sizeof(B) > sizeof(CompareInt))
overflow |= (bigint_cast<B>(y) != b);
overflow |= (static_cast<B>(y) != b);
if constexpr (is_unsigned_v<A>)
overflow |= (x < 0);
if constexpr (is_unsigned_v<B>)
......
......@@ -156,7 +156,7 @@ struct Decimal
return convertTo<typename U::NativeType>();
}
else
return bigint_cast<U>(value);
return static_cast<U>(value);
}
const Decimal<T> & operator += (const T & x) { value += x; return *this; }
......
......@@ -218,7 +218,7 @@ using ResultOfGreatest = std::conditional_t<LeastGreatestSpecialCase<A, B>,
template <typename T>
static inline auto littleBits(const T & x)
{
return bigint_cast<UInt8>(x);
return static_cast<UInt8>(x);
}
}
......@@ -51,9 +51,9 @@ inline auto checkedDivision(A a, B b)
throwIfDivisionLeadsToFPE(a, b);
if constexpr (is_big_int_v<A> && std::is_floating_point_v<B>)
return bigint_cast<B>(a) / b;
return static_cast<B>(a) / b;
else if constexpr (is_big_int_v<B> && std::is_floating_point_v<A>)
return a / bigint_cast<A>(b);
return a / static_cast<A>(b);
else if constexpr (is_big_int_v<A> && is_big_int_v<B>)
return static_cast<A>(a / b);
else if constexpr (!is_big_int_v<A> && is_big_int_v<B>)
......@@ -84,10 +84,10 @@ struct DivideIntegralImpl
using SignedCastA = make_signed_t<CastA>;
using SignedCastB = std::conditional_t<sizeof(A) <= sizeof(B), make_signed_t<CastB>, SignedCastA>;
return bigint_cast<Result>(checkedDivision(bigint_cast<SignedCastA>(a), bigint_cast<SignedCastB>(b)));
return static_cast<Result>(checkedDivision(static_cast<SignedCastA>(a), static_cast<SignedCastB>(b)));
}
else
return bigint_cast<Result>(checkedDivision(CastA(a), CastB(b)));
return static_cast<Result>(checkedDivision(CastA(a), CastB(b)));
}
#if USE_EMBEDDED_COMPILER
......@@ -110,7 +110,7 @@ struct ModuloImpl
if constexpr (std::is_floating_point_v<ResultType>)
{
/// This computation is similar to `fmod` but the latter is not inlined and has 40 times worse performance.
return bigint_cast<ResultType>(a) - trunc(bigint_cast<ResultType>(a) / bigint_cast<ResultType>(b)) * bigint_cast<ResultType>(b);
return static_cast<ResultType>(a) - trunc(static_cast<ResultType>(a) / static_cast<ResultType>(b)) * static_cast<ResultType>(b);
}
else
{
......@@ -125,9 +125,9 @@ struct ModuloImpl
CastB int_b(b);
if constexpr (is_big_int_v<IntegerBType> && sizeof(IntegerAType) <= sizeof(IntegerBType))
return bigint_cast<Result>(bigint_cast<CastB>(int_a) % int_b);
return static_cast<Result>(static_cast<CastB>(int_a) % int_b);
else
return bigint_cast<Result>(int_a % bigint_cast<CastA>(int_b));
return static_cast<Result>(int_a % static_cast<CastA>(int_b));
}
else
return IntegerAType(a) % IntegerBType(b);
......
......@@ -81,8 +81,8 @@ namespace ErrorCodes
template <typename A, typename B, typename Op>
struct NumComparisonImpl
{
using ContainerA = std::conditional_t<!is_big_int_v<A>, PaddedPODArray<A>, std::vector<A>>;
using ContainerB = std::conditional_t<!is_big_int_v<B>, PaddedPODArray<B>, std::vector<B>>;
using ContainerA = PaddedPODArray<A>;
using ContainerB = PaddedPODArray<B>;
/// If you don't specify NO_INLINE, the compiler will inline this function, but we don't need this as this function contains tight loop inside.
static void NO_INLINE vectorVector(const ContainerA & a, const ContainerB & b, PaddedPODArray<UInt8> & c)
......
......@@ -806,16 +806,7 @@ private:
size_t size = vec_from.size();
for (size_t i = 0; i < size; ++i)
{
ToType h;
if constexpr (OverBigInt<FromType>)
{
using NativeT = typename NativeType<FromType>::Type;
std::string buffer = BigInt<NativeT>::serialize(vec_from[i]);
h = Impl::apply(buffer.data(), buffer.size());
}
else
h = Impl::apply(reinterpret_cast<const char *>(&vec_from[i]), sizeof(vec_from[i]));
ToType h = Impl::apply(reinterpret_cast<const char *>(&vec_from[i]), sizeof(vec_from[i]));
if constexpr (first)
vec_to[i] = h;
......@@ -827,16 +818,7 @@ private:
{
auto value = col_from_const->template getValue<FromType>();
ToType h;
if constexpr (OverBigInt<FromType>)
{
using NativeT = typename NativeType<FromType>::Type;
std::string buffer = BigInt<NativeT>::serialize(value);
h = Impl::apply(buffer.data(), buffer.size());
}
else
h = Impl::apply(reinterpret_cast<const char *>(&value), sizeof(value));
ToType h = Impl::apply(reinterpret_cast<const char *>(&value), sizeof(value));
size_t size = vec_to.size();
if constexpr (first)
......
......@@ -50,9 +50,9 @@ void writeSlice(const NumericArraySlice<T> & slice, NumericArraySink<U> & sink)
throw Exception("No conversion between UInt128 and " + demangle(typeid(T).name()), ErrorCodes::NOT_IMPLEMENTED);
}
else if constexpr (IsDecimalNumber<T>)
dst = bigint_cast<NativeU>(src.value);
dst = static_cast<NativeU>(src.value);
else
dst = bigint_cast<NativeU>(src);
dst = static_cast<NativeU>(src);
}
else
dst = static_cast<NativeU>(src);
......
......@@ -21,7 +21,7 @@ struct BitAndImpl
template <typename Result = ResultType>
static inline Result apply(A a, B b)
{
return bigint_cast<Result>(a) & bigint_cast<Result>(b);
return static_cast<Result>(a) & static_cast<Result>(b);
}
#if USE_EMBEDDED_COMPILER
......
......@@ -20,7 +20,7 @@ struct BitOrImpl
template <typename Result = ResultType>
static inline Result apply(A a, B b)
{
return bigint_cast<Result>(a) | bigint_cast<Result>(b);
return static_cast<Result>(a) | static_cast<Result>(b);
}
#if USE_EMBEDDED_COMPILER
......
......@@ -24,7 +24,7 @@ struct BitShiftLeftImpl
if constexpr (is_big_int_v<B>)
throw Exception("BitShiftLeft is not implemented for big integers as second argument", ErrorCodes::NOT_IMPLEMENTED);
else if constexpr (is_big_int_v<A>)
return bigint_cast<Result>(a) << bigint_cast<UInt32>(b);
return static_cast<Result>(a) << static_cast<UInt32>(b);
else
return static_cast<Result>(a) << static_cast<Result>(b);
}
......
......@@ -24,7 +24,7 @@ struct BitShiftRightImpl
if constexpr (is_big_int_v<B>)
throw Exception("BitShiftRight is not implemented for big integers as second argument", ErrorCodes::NOT_IMPLEMENTED);
else if constexpr (is_big_int_v<A>)
return bigint_cast<Result>(a) >> bigint_cast<UInt32>(b);
return static_cast<Result>(a) >> static_cast<UInt32>(b);
else
return static_cast<Result>(a) >> static_cast<Result>(b);
}
......
......@@ -20,7 +20,7 @@ struct BitXorImpl
template <typename Result = ResultType>
static inline Result apply(A a, B b)
{
return bigint_cast<Result>(a) ^ bigint_cast<Result>(b);
return static_cast<Result>(a) ^ static_cast<Result>(b);
}
#if USE_EMBEDDED_COMPILER
......
......@@ -16,8 +16,8 @@ struct GreatestBaseImpl
template <typename Result = ResultType>
static inline Result apply(A a, B b)
{
return bigint_cast<Result>(a) > bigint_cast<Result>(b) ?
bigint_cast<Result>(a) : bigint_cast<Result>(b);
return static_cast<Result>(a) > static_cast<Result>(b) ?
static_cast<Result>(a) : static_cast<Result>(b);
}
#if USE_EMBEDDED_COMPILER
......
......@@ -60,7 +60,7 @@ struct NumIfImpl
typename ColVecResult::Container & res = col_res->getData();
for (size_t i = 0; i < size; ++i)
res[i] = cond[i] ? bigint_cast<ResultType>(a[i]) : bigint_cast<ResultType>(b[i]);
res[i] = cond[i] ? static_cast<ResultType>(a[i]) : static_cast<ResultType>(b[i]);
return col_res;
}
......@@ -71,7 +71,7 @@ struct NumIfImpl
typename ColVecResult::Container & res = col_res->getData();
for (size_t i = 0; i < size; ++i)
res[i] = cond[i] ? bigint_cast<ResultType>(a[i]) : bigint_cast<ResultType>(b);
res[i] = cond[i] ? static_cast<ResultType>(a[i]) : static_cast<ResultType>(b);
return col_res;
}
......@@ -82,7 +82,7 @@ struct NumIfImpl
typename ColVecResult::Container & res = col_res->getData();
for (size_t i = 0; i < size; ++i)
res[i] = cond[i] ? bigint_cast<ResultType>(a) : bigint_cast<ResultType>(b[i]);
res[i] = cond[i] ? static_cast<ResultType>(a) : static_cast<ResultType>(b[i]);
return col_res;
}
......@@ -93,7 +93,7 @@ struct NumIfImpl
typename ColVecResult::Container & res = col_res->getData();
for (size_t i = 0; i < size; ++i)
res[i] = cond[i] ? bigint_cast<ResultType>(a) : bigint_cast<ResultType>(b);
res[i] = cond[i] ? static_cast<ResultType>(a) : static_cast<ResultType>(b);
return col_res;
}
};
......
......@@ -17,7 +17,7 @@ struct LeastBaseImpl
static inline Result apply(A a, B b)
{
/** gcc 4.9.2 successfully vectorizes a loop from this function. */
return bigint_cast<Result>(a) < bigint_cast<Result>(b) ? bigint_cast<Result>(a) : bigint_cast<Result>(b);
return static_cast<Result>(a) < static_cast<Result>(b) ? static_cast<Result>(a) : static_cast<Result>(b);
}
#if USE_EMBEDDED_COMPILER
......
......@@ -19,7 +19,7 @@ struct MinusImpl
using CastA = std::conditional_t<std::is_floating_point_v<B>, B, A>;
using CastB = std::conditional_t<std::is_floating_point_v<A>, A, B>;
return bigint_cast<Result>(bigint_cast<CastA>(a)) - bigint_cast<Result>(bigint_cast<CastB>(b));
return static_cast<Result>(static_cast<CastA>(a)) - static_cast<Result>(static_cast<CastB>(b));
}
else
return static_cast<Result>(a) - b;
......
......@@ -19,7 +19,7 @@ struct MultiplyImpl
using CastA = std::conditional_t<std::is_floating_point_v<B>, B, A>;
using CastB = std::conditional_t<std::is_floating_point_v<A>, A, B>;
return bigint_cast<Result>(bigint_cast<CastA>(a)) * bigint_cast<Result>(bigint_cast<CastB>(b));
return static_cast<Result>(static_cast<CastA>(a)) * static_cast<Result>(static_cast<CastB>(b));
}
else
return static_cast<Result>(a) * b;
......
......@@ -21,7 +21,7 @@ struct PlusImpl
using CastA = std::conditional_t<std::is_floating_point_v<B>, B, A>;
using CastB = std::conditional_t<std::is_floating_point_v<A>, A, B>;
return bigint_cast<Result>(bigint_cast<CastA>(a)) + bigint_cast<Result>(bigint_cast<CastB>(b));
return static_cast<Result>(static_cast<CastA>(a)) + static_cast<Result>(static_cast<CastB>(b));
}
else
return static_cast<Result>(a) + b;
......
......@@ -17,7 +17,6 @@
#include <Core/Types.h>
#include <Core/DecimalFunctions.h>
#include <Core/UUID.h>
#include <Core/BigInt.h>
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>
......@@ -121,17 +120,6 @@ inline void readFloatBinary(T & x, ReadBuffer & buf)
readPODBinary(x, buf);
}
template <typename T>
void readBigIntBinary(T & x, ReadBuffer & buf)
{
static const constexpr size_t bytesize = BigInt<T>::size;
char bytes[bytesize];
buf.readStrict(bytes, bytesize);
x = BigInt<T>::deserialize(bytes);
}
inline void readStringBinary(std::string & s, ReadBuffer & buf, size_t MAX_STRING_SIZE = DEFAULT_MAX_STRING_SIZE)
{
size_t size = 0;
......@@ -841,11 +829,11 @@ inline void readBinary(DummyUInt256 & x, ReadBuffer & buf) { readPODBinary(x, bu
inline void readBinary(Decimal32 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(Decimal64 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(Decimal128 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(Decimal256 & x, ReadBuffer & buf) { readBigIntBinary(x.value, buf); }
inline void readBinary(Decimal256 & x, ReadBuffer & buf) { readPODBinary(x.value, buf); }
inline void readBinary(LocalDate & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(UInt256 & x, ReadBuffer & buf) { readBigIntBinary(x, buf); }
inline void readBinary(Int256 & x, ReadBuffer & buf) { readBigIntBinary(x, buf); }
inline void readBinary(UInt256 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(Int256 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
template <typename T>
inline std::enable_if_t<is_arithmetic_v<T> && (sizeof(T) <= 8), void>
......
......@@ -16,7 +16,6 @@
#include <Core/DecimalFunctions.h>
#include <Core/Types.h>
#include <Core/UUID.h>
#include <Core/BigInt.h>
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>
......@@ -113,17 +112,6 @@ inline void writeStringBinary(const std::string_view & s, WriteBuffer & buf)
writeStringBinary(StringRef{s}, buf);
}
template <typename T>
void writeBigIntBinary(const T & x, WriteBuffer & buf)
{
static const constexpr size_t bytesize = BigInt<T>::size;
char bytes[bytesize];
BigInt<T>::serialize(x, bytes);
buf.write(bytes, bytesize);
}
template <typename T>
void writeVectorBinary(const std::vector<T> & v, WriteBuffer & buf)
{
......@@ -843,12 +831,12 @@ inline void writeBinary(const DummyUInt256 & x, WriteBuffer & buf) { writePODBin
inline void writeBinary(const Decimal32 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
inline void writeBinary(const Decimal64 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
inline void writeBinary(const Decimal128 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
inline void writeBinary(const Decimal256 & x, WriteBuffer & buf) { writeBigIntBinary(x.value, buf); }
inline void writeBinary(const Decimal256 & x, WriteBuffer & buf) { writePODBinary(x.value, buf); }
inline void writeBinary(const LocalDate & x, WriteBuffer & buf) { writePODBinary(x, buf); }
inline void writeBinary(const LocalDateTime & x, WriteBuffer & buf) { writePODBinary(x, buf); }
inline void writeBinary(const UInt256 & x, WriteBuffer & buf) { writeBigIntBinary(x, buf); }
inline void writeBinary(const Int256 & x, WriteBuffer & buf) { writeBigIntBinary(x, buf); }
inline void writeBinary(const UInt256 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
inline void writeBinary(const Int256 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
/// Methods for outputting the value in text form for a tab-separated format.
template <typename T>
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册