mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
Merge pull request #34052 from kitaisreal/remove-decimal-padded-pod-array
Remove DecimalPaddedPODArray
This commit is contained in:
commit
60129aaecb
@ -226,7 +226,7 @@ public:
|
|||||||
{
|
{
|
||||||
// FIXME why is storing NearestFieldType not enough, and we
|
// FIXME why is storing NearestFieldType not enough, and we
|
||||||
// have to check for decimals again here?
|
// have to check for decimals again here?
|
||||||
UInt32 scale = static_cast<const ColumnDecimal<T> &>(key_column).getData().getScale();
|
UInt32 scale = static_cast<const ColumnDecimal<T> &>(key_column).getScale();
|
||||||
it = merged_maps.find(DecimalField<T>(key, scale));
|
it = merged_maps.find(DecimalField<T>(key, scale));
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
@ -251,7 +251,7 @@ public:
|
|||||||
|
|
||||||
if constexpr (is_decimal<T>)
|
if constexpr (is_decimal<T>)
|
||||||
{
|
{
|
||||||
UInt32 scale = static_cast<const ColumnDecimal<T> &>(key_column).getData().getScale();
|
UInt32 scale = static_cast<const ColumnDecimal<T> &>(key_column).getScale();
|
||||||
merged_maps.emplace(DecimalField<T>(key, scale), std::move(new_values));
|
merged_maps.emplace(DecimalField<T>(key, scale), std::move(new_values));
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
|
@ -32,12 +32,6 @@ namespace ErrorCodes
|
|||||||
extern const int LOGICAL_ERROR;
|
extern const int LOGICAL_ERROR;
|
||||||
}
|
}
|
||||||
|
|
||||||
template class DecimalPaddedPODArray<Decimal32>;
|
|
||||||
template class DecimalPaddedPODArray<Decimal64>;
|
|
||||||
template class DecimalPaddedPODArray<Decimal128>;
|
|
||||||
template class DecimalPaddedPODArray<Decimal256>;
|
|
||||||
template class DecimalPaddedPODArray<DateTime64>;
|
|
||||||
|
|
||||||
template <is_decimal T>
|
template <is_decimal T>
|
||||||
int ColumnDecimal<T>::compareAt(size_t n, size_t m, const IColumn & rhs_, int) const
|
int ColumnDecimal<T>::compareAt(size_t n, size_t m, const IColumn & rhs_, int) const
|
||||||
{
|
{
|
||||||
@ -131,19 +125,6 @@ void ColumnDecimal<T>::updateHashFast(SipHash & hash) const
|
|||||||
template <is_decimal T>
|
template <is_decimal T>
|
||||||
void ColumnDecimal<T>::getPermutation(bool reverse, size_t limit, int , IColumn::Permutation & res) const
|
void ColumnDecimal<T>::getPermutation(bool reverse, size_t limit, int , IColumn::Permutation & res) const
|
||||||
{
|
{
|
||||||
#if 1 /// TODO: perf test
|
|
||||||
if (data.size() <= std::numeric_limits<UInt32>::max())
|
|
||||||
{
|
|
||||||
PaddedPODArray<UInt32> tmp_res;
|
|
||||||
permutation(reverse, limit, tmp_res);
|
|
||||||
|
|
||||||
res.resize(tmp_res.size());
|
|
||||||
for (size_t i = 0; i < tmp_res.size(); ++i)
|
|
||||||
res[i] = tmp_res[i];
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
#endif
|
|
||||||
|
|
||||||
permutation(reverse, limit, res);
|
permutation(reverse, limit, res);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,66 +1,21 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <cmath>
|
||||||
|
|
||||||
|
#include <base/sort.h>
|
||||||
|
#include <base/TypeName.h>
|
||||||
|
#include <Core/Field.h>
|
||||||
|
#include <Core/DecimalFunctions.h>
|
||||||
|
#include <Core/TypeId.h>
|
||||||
|
#include <Common/typeid_cast.h>
|
||||||
#include <Columns/ColumnVectorHelper.h>
|
#include <Columns/ColumnVectorHelper.h>
|
||||||
#include <Columns/IColumn.h>
|
#include <Columns/IColumn.h>
|
||||||
#include <Columns/IColumnImpl.h>
|
#include <Columns/IColumnImpl.h>
|
||||||
#include <Core/Field.h>
|
|
||||||
#include <Core/DecimalFunctions.h>
|
|
||||||
#include <Common/typeid_cast.h>
|
|
||||||
#include <base/sort.h>
|
|
||||||
#include <Core/TypeId.h>
|
|
||||||
#include <base/TypeName.h>
|
|
||||||
|
|
||||||
#include <cmath>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
/// PaddedPODArray extended by Decimal scale
|
|
||||||
template <typename T>
|
|
||||||
class DecimalPaddedPODArray : public PaddedPODArray<T>
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
using Base = PaddedPODArray<T>;
|
|
||||||
using Base::operator[];
|
|
||||||
|
|
||||||
DecimalPaddedPODArray(size_t size, UInt32 scale_)
|
|
||||||
: Base(size),
|
|
||||||
scale(scale_)
|
|
||||||
{}
|
|
||||||
|
|
||||||
DecimalPaddedPODArray(const DecimalPaddedPODArray & other)
|
|
||||||
: Base(other.begin(), other.end()),
|
|
||||||
scale(other.scale)
|
|
||||||
{}
|
|
||||||
|
|
||||||
DecimalPaddedPODArray(DecimalPaddedPODArray && other)
|
|
||||||
{
|
|
||||||
this->swap(other);
|
|
||||||
std::swap(scale, other.scale);
|
|
||||||
}
|
|
||||||
|
|
||||||
DecimalPaddedPODArray & operator=(DecimalPaddedPODArray && other)
|
|
||||||
{
|
|
||||||
this->swap(other);
|
|
||||||
std::swap(scale, other.scale);
|
|
||||||
return *this;
|
|
||||||
}
|
|
||||||
|
|
||||||
UInt32 getScale() const { return scale; }
|
|
||||||
|
|
||||||
private:
|
|
||||||
UInt32 scale;
|
|
||||||
};
|
|
||||||
|
|
||||||
/// Prevent implicit template instantiation of DecimalPaddedPODArray for common decimal types
|
|
||||||
|
|
||||||
extern template class DecimalPaddedPODArray<Decimal32>;
|
|
||||||
extern template class DecimalPaddedPODArray<Decimal64>;
|
|
||||||
extern template class DecimalPaddedPODArray<Decimal128>;
|
|
||||||
extern template class DecimalPaddedPODArray<Decimal256>;
|
|
||||||
extern template class DecimalPaddedPODArray<DateTime64>;
|
|
||||||
|
|
||||||
/// A ColumnVector for Decimals
|
/// A ColumnVector for Decimals
|
||||||
template <is_decimal T>
|
template <is_decimal T>
|
||||||
class ColumnDecimal final : public COWHelper<ColumnVectorHelper, ColumnDecimal<T>>
|
class ColumnDecimal final : public COWHelper<ColumnVectorHelper, ColumnDecimal<T>>
|
||||||
@ -72,16 +27,16 @@ private:
|
|||||||
public:
|
public:
|
||||||
using ValueType = T;
|
using ValueType = T;
|
||||||
using NativeT = typename T::NativeType;
|
using NativeT = typename T::NativeType;
|
||||||
using Container = DecimalPaddedPODArray<T>;
|
using Container = PaddedPODArray<T>;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
ColumnDecimal(const size_t n, UInt32 scale_)
|
ColumnDecimal(const size_t n, UInt32 scale_)
|
||||||
: data(n, scale_),
|
: data(n),
|
||||||
scale(scale_)
|
scale(scale_)
|
||||||
{}
|
{}
|
||||||
|
|
||||||
ColumnDecimal(const ColumnDecimal & src)
|
ColumnDecimal(const ColumnDecimal & src)
|
||||||
: data(src.data),
|
: data(src.data.begin(), src.data.end()),
|
||||||
scale(src.scale)
|
scale(src.scale)
|
||||||
{}
|
{}
|
||||||
|
|
||||||
@ -195,7 +150,7 @@ public:
|
|||||||
const T & getElement(size_t n) const { return data[n]; }
|
const T & getElement(size_t n) const { return data[n]; }
|
||||||
T & getElement(size_t n) { return data[n]; }
|
T & getElement(size_t n) { return data[n]; }
|
||||||
|
|
||||||
UInt32 getScale() const {return scale;}
|
UInt32 getScale() const { return scale; }
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
Container data;
|
Container data;
|
||||||
@ -206,8 +161,8 @@ protected:
|
|||||||
{
|
{
|
||||||
size_t s = data.size();
|
size_t s = data.size();
|
||||||
res.resize(s);
|
res.resize(s);
|
||||||
for (U i = 0; i < s; ++i)
|
for (size_t i = 0; i < s; ++i)
|
||||||
res[i] = i;
|
res[i] = static_cast<U>(i);
|
||||||
|
|
||||||
auto sort_end = res.end();
|
auto sort_end = res.end();
|
||||||
if (limit && limit < s)
|
if (limit && limit < s)
|
||||||
|
@ -125,7 +125,7 @@ private:
|
|||||||
{
|
{
|
||||||
const auto & src_data = col->getData();
|
const auto & src_data = col->getData();
|
||||||
const size_t size = src_data.size();
|
const size_t size = src_data.size();
|
||||||
UInt32 scale = src_data.getScale();
|
UInt32 scale = col->getScale();
|
||||||
|
|
||||||
auto dst = ColumnVector<ReturnType>::create();
|
auto dst = ColumnVector<ReturnType>::create();
|
||||||
auto & dst_data = dst->getData();
|
auto & dst_data = dst->getData();
|
||||||
|
@ -152,9 +152,11 @@ struct ConvertImpl
|
|||||||
if (const ColVecFrom * col_from = checkAndGetColumn<ColVecFrom>(named_from.column.get()))
|
if (const ColVecFrom * col_from = checkAndGetColumn<ColVecFrom>(named_from.column.get()))
|
||||||
{
|
{
|
||||||
typename ColVecTo::MutablePtr col_to = nullptr;
|
typename ColVecTo::MutablePtr col_to = nullptr;
|
||||||
|
|
||||||
if constexpr (IsDataTypeDecimal<ToDataType>)
|
if constexpr (IsDataTypeDecimal<ToDataType>)
|
||||||
{
|
{
|
||||||
UInt32 scale;
|
UInt32 scale;
|
||||||
|
|
||||||
if constexpr (std::is_same_v<Additions, AccurateConvertStrategyAdditions>
|
if constexpr (std::is_same_v<Additions, AccurateConvertStrategyAdditions>
|
||||||
|| std::is_same_v<Additions, AccurateOrNullConvertStrategyAdditions>)
|
|| std::is_same_v<Additions, AccurateOrNullConvertStrategyAdditions>)
|
||||||
{
|
{
|
||||||
@ -208,11 +210,11 @@ struct ConvertImpl
|
|||||||
bool convert_result = false;
|
bool convert_result = false;
|
||||||
|
|
||||||
if constexpr (IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>)
|
if constexpr (IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>)
|
||||||
convert_result = tryConvertDecimals<FromDataType, ToDataType>(vec_from[i], vec_from.getScale(), vec_to.getScale(), result);
|
convert_result = tryConvertDecimals<FromDataType, ToDataType>(vec_from[i], col_from->getScale(), col_to->getScale(), result);
|
||||||
else if constexpr (IsDataTypeDecimal<FromDataType> && IsDataTypeNumber<ToDataType>)
|
else if constexpr (IsDataTypeDecimal<FromDataType> && IsDataTypeNumber<ToDataType>)
|
||||||
convert_result = tryConvertFromDecimal<FromDataType, ToDataType>(vec_from[i], vec_from.getScale(), result);
|
convert_result = tryConvertFromDecimal<FromDataType, ToDataType>(vec_from[i], col_from->getScale(), result);
|
||||||
else if constexpr (IsDataTypeNumber<FromDataType> && IsDataTypeDecimal<ToDataType>)
|
else if constexpr (IsDataTypeNumber<FromDataType> && IsDataTypeDecimal<ToDataType>)
|
||||||
convert_result = tryConvertToDecimal<FromDataType, ToDataType>(vec_from[i], vec_to.getScale(), result);
|
convert_result = tryConvertToDecimal<FromDataType, ToDataType>(vec_from[i], col_to->getScale(), result);
|
||||||
|
|
||||||
if (convert_result)
|
if (convert_result)
|
||||||
vec_to[i] = result;
|
vec_to[i] = result;
|
||||||
@ -225,11 +227,11 @@ struct ConvertImpl
|
|||||||
else
|
else
|
||||||
{
|
{
|
||||||
if constexpr (IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>)
|
if constexpr (IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>)
|
||||||
vec_to[i] = convertDecimals<FromDataType, ToDataType>(vec_from[i], vec_from.getScale(), vec_to.getScale());
|
vec_to[i] = convertDecimals<FromDataType, ToDataType>(vec_from[i], col_from->getScale(), col_to->getScale());
|
||||||
else if constexpr (IsDataTypeDecimal<FromDataType> && IsDataTypeNumber<ToDataType>)
|
else if constexpr (IsDataTypeDecimal<FromDataType> && IsDataTypeNumber<ToDataType>)
|
||||||
vec_to[i] = convertFromDecimal<FromDataType, ToDataType>(vec_from[i], vec_from.getScale());
|
vec_to[i] = convertFromDecimal<FromDataType, ToDataType>(vec_from[i], col_from->getScale());
|
||||||
else if constexpr (IsDataTypeNumber<FromDataType> && IsDataTypeDecimal<ToDataType>)
|
else if constexpr (IsDataTypeNumber<FromDataType> && IsDataTypeDecimal<ToDataType>)
|
||||||
vec_to[i] = convertToDecimal<FromDataType, ToDataType>(vec_from[i], vec_to.getScale());
|
vec_to[i] = convertToDecimal<FromDataType, ToDataType>(vec_from[i], col_to->getScale());
|
||||||
else
|
else
|
||||||
throw Exception("Unsupported data type in conversion function", ErrorCodes::CANNOT_CONVERT_TYPE);
|
throw Exception("Unsupported data type in conversion function", ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||||
}
|
}
|
||||||
@ -820,7 +822,7 @@ struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType,
|
|||||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime>)
|
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime>)
|
||||||
data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss") + 1));
|
data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss") + 1));
|
||||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime64>)
|
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime64>)
|
||||||
data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss.") + vec_from.getScale() + 1));
|
data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss.") + col_from->getScale() + 1));
|
||||||
else
|
else
|
||||||
data_to.resize(size * 3); /// Arbitrary
|
data_to.resize(size * 3); /// Arbitrary
|
||||||
|
|
||||||
@ -1169,7 +1171,7 @@ struct ConvertThroughParsing
|
|||||||
if constexpr (to_datetime64)
|
if constexpr (to_datetime64)
|
||||||
{
|
{
|
||||||
DateTime64 res = 0;
|
DateTime64 res = 0;
|
||||||
parseDateTime64BestEffort(res, vec_to.getScale(), read_buffer, *local_time_zone, *utc_time_zone);
|
parseDateTime64BestEffort(res, col_to->getScale(), read_buffer, *local_time_zone, *utc_time_zone);
|
||||||
vec_to[i] = res;
|
vec_to[i] = res;
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
@ -1184,7 +1186,7 @@ struct ConvertThroughParsing
|
|||||||
if constexpr (to_datetime64)
|
if constexpr (to_datetime64)
|
||||||
{
|
{
|
||||||
DateTime64 res = 0;
|
DateTime64 res = 0;
|
||||||
parseDateTime64BestEffortUS(res, vec_to.getScale(), read_buffer, *local_time_zone, *utc_time_zone);
|
parseDateTime64BestEffortUS(res, col_to->getScale(), read_buffer, *local_time_zone, *utc_time_zone);
|
||||||
vec_to[i] = res;
|
vec_to[i] = res;
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
@ -1199,12 +1201,12 @@ struct ConvertThroughParsing
|
|||||||
if constexpr (to_datetime64)
|
if constexpr (to_datetime64)
|
||||||
{
|
{
|
||||||
DateTime64 value = 0;
|
DateTime64 value = 0;
|
||||||
readDateTime64Text(value, vec_to.getScale(), read_buffer, *local_time_zone);
|
readDateTime64Text(value, col_to->getScale(), read_buffer, *local_time_zone);
|
||||||
vec_to[i] = value;
|
vec_to[i] = value;
|
||||||
}
|
}
|
||||||
else if constexpr (IsDataTypeDecimal<ToDataType>)
|
else if constexpr (IsDataTypeDecimal<ToDataType>)
|
||||||
SerializationDecimal<typename ToDataType::FieldType>::readText(
|
SerializationDecimal<typename ToDataType::FieldType>::readText(
|
||||||
vec_to[i], read_buffer, ToDataType::maxPrecision(), vec_to.getScale());
|
vec_to[i], read_buffer, ToDataType::maxPrecision(), col_to->getScale());
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
parseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone);
|
parseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone);
|
||||||
@ -1223,7 +1225,7 @@ struct ConvertThroughParsing
|
|||||||
if constexpr (to_datetime64)
|
if constexpr (to_datetime64)
|
||||||
{
|
{
|
||||||
DateTime64 res = 0;
|
DateTime64 res = 0;
|
||||||
parsed = tryParseDateTime64BestEffort(res, vec_to.getScale(), read_buffer, *local_time_zone, *utc_time_zone);
|
parsed = tryParseDateTime64BestEffort(res, col_to->getScale(), read_buffer, *local_time_zone, *utc_time_zone);
|
||||||
vec_to[i] = res;
|
vec_to[i] = res;
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
@ -1244,12 +1246,12 @@ struct ConvertThroughParsing
|
|||||||
if constexpr (to_datetime64)
|
if constexpr (to_datetime64)
|
||||||
{
|
{
|
||||||
DateTime64 value = 0;
|
DateTime64 value = 0;
|
||||||
parsed = tryReadDateTime64Text(value, vec_to.getScale(), read_buffer, *local_time_zone);
|
parsed = tryReadDateTime64Text(value, col_to->getScale(), read_buffer, *local_time_zone);
|
||||||
vec_to[i] = value;
|
vec_to[i] = value;
|
||||||
}
|
}
|
||||||
else if constexpr (IsDataTypeDecimal<ToDataType>)
|
else if constexpr (IsDataTypeDecimal<ToDataType>)
|
||||||
parsed = SerializationDecimal<typename ToDataType::FieldType>::tryReadText(
|
parsed = SerializationDecimal<typename ToDataType::FieldType>::tryReadText(
|
||||||
vec_to[i], read_buffer, ToDataType::maxPrecision(), vec_to.getScale());
|
vec_to[i], read_buffer, ToDataType::maxPrecision(), col_to->getScale());
|
||||||
else
|
else
|
||||||
parsed = tryParseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone);
|
parsed = tryParseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone);
|
||||||
}
|
}
|
||||||
|
@ -422,9 +422,9 @@ private:
|
|||||||
using Container = typename ColumnDecimal<T>::Container;
|
using Container = typename ColumnDecimal<T>::Container;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static NO_INLINE void apply(const Container & in, Container & out, Scale scale_arg)
|
static NO_INLINE void apply(const Container & in, UInt32 in_scale, Container & out, Scale scale_arg)
|
||||||
{
|
{
|
||||||
scale_arg = in.getScale() - scale_arg;
|
scale_arg = in_scale - scale_arg;
|
||||||
if (scale_arg > 0)
|
if (scale_arg > 0)
|
||||||
{
|
{
|
||||||
size_t scale = intExp10(scale_arg);
|
size_t scale = intExp10(scale_arg);
|
||||||
@ -498,11 +498,11 @@ public:
|
|||||||
const auto * const col = checkAndGetColumn<ColumnDecimal<T>>(col_general);
|
const auto * const col = checkAndGetColumn<ColumnDecimal<T>>(col_general);
|
||||||
const typename ColumnDecimal<T>::Container & vec_src = col->getData();
|
const typename ColumnDecimal<T>::Container & vec_src = col->getData();
|
||||||
|
|
||||||
auto col_res = ColumnDecimal<T>::create(vec_src.size(), vec_src.getScale());
|
auto col_res = ColumnDecimal<T>::create(vec_src.size(), col->getScale());
|
||||||
auto & vec_res = col_res->getData();
|
auto & vec_res = col_res->getData();
|
||||||
|
|
||||||
if (!vec_res.empty())
|
if (!vec_res.empty())
|
||||||
DecimalRoundingImpl<T, rounding_mode, tie_breaking_mode>::apply(col->getData(), vec_res, scale_arg);
|
DecimalRoundingImpl<T, rounding_mode, tie_breaking_mode>::apply(col->getData(), col->getScale(), vec_res, scale_arg);
|
||||||
|
|
||||||
return col_res;
|
return col_res;
|
||||||
}
|
}
|
||||||
|
@ -157,11 +157,11 @@ struct ArrayAggregateImpl
|
|||||||
return false;
|
return false;
|
||||||
|
|
||||||
const AggregationType x = column_const->template getValue<Element>(); // NOLINT
|
const AggregationType x = column_const->template getValue<Element>(); // NOLINT
|
||||||
const auto & data = checkAndGetColumn<ColVecType>(&column_const->getDataColumn())->getData();
|
const ColVecType * column_typed = checkAndGetColumn<ColVecType>(&column_const->getDataColumn());
|
||||||
|
|
||||||
typename ColVecResultType::MutablePtr res_column;
|
typename ColVecResultType::MutablePtr res_column;
|
||||||
if constexpr (is_decimal<Element>)
|
if constexpr (is_decimal<Element>)
|
||||||
res_column = ColVecResultType::create(offsets.size(), data.getScale());
|
res_column = ColVecResultType::create(offsets.size(), column_typed->getScale());
|
||||||
else
|
else
|
||||||
res_column = ColVecResultType::create(offsets.size());
|
res_column = ColVecResultType::create(offsets.size());
|
||||||
|
|
||||||
@ -185,7 +185,7 @@ struct ArrayAggregateImpl
|
|||||||
{
|
{
|
||||||
if constexpr (is_decimal<Element>)
|
if constexpr (is_decimal<Element>)
|
||||||
{
|
{
|
||||||
res[i] = DecimalUtils::convertTo<ResultType>(x, data.getScale());
|
res[i] = DecimalUtils::convertTo<ResultType>(x, column_typed->getScale());
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
@ -210,11 +210,11 @@ struct ArrayAggregateImpl
|
|||||||
throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Decimal math overflow");
|
throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Decimal math overflow");
|
||||||
}
|
}
|
||||||
|
|
||||||
auto result_scale = data.getScale() * array_size;
|
auto result_scale = column_typed->getScale() * array_size;
|
||||||
if (unlikely(result_scale > DecimalUtils::max_precision<AggregationType>))
|
if (unlikely(result_scale > DecimalUtils::max_precision<AggregationType>))
|
||||||
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds", result_scale);
|
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds", result_scale);
|
||||||
|
|
||||||
res[i] = DecimalUtils::convertTo<ResultType>(product, data.getScale() * array_size);
|
res[i] = DecimalUtils::convertTo<ResultType>(product, result_scale);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
@ -236,7 +236,7 @@ struct ArrayAggregateImpl
|
|||||||
|
|
||||||
typename ColVecResultType::MutablePtr res_column;
|
typename ColVecResultType::MutablePtr res_column;
|
||||||
if constexpr (is_decimal<Element>)
|
if constexpr (is_decimal<Element>)
|
||||||
res_column = ColVecResultType::create(offsets.size(), data.getScale());
|
res_column = ColVecResultType::create(offsets.size(), column->getScale());
|
||||||
else
|
else
|
||||||
res_column = ColVecResultType::create(offsets.size());
|
res_column = ColVecResultType::create(offsets.size());
|
||||||
|
|
||||||
@ -309,7 +309,7 @@ struct ArrayAggregateImpl
|
|||||||
if constexpr (is_decimal<Element>)
|
if constexpr (is_decimal<Element>)
|
||||||
{
|
{
|
||||||
aggregate_value = aggregate_value / AggregationType(count);
|
aggregate_value = aggregate_value / AggregationType(count);
|
||||||
res[i] = DecimalUtils::convertTo<ResultType>(aggregate_value, data.getScale());
|
res[i] = DecimalUtils::convertTo<ResultType>(aggregate_value, column->getScale());
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
@ -318,7 +318,7 @@ struct ArrayAggregateImpl
|
|||||||
}
|
}
|
||||||
else if constexpr (aggregate_operation == AggregateOperation::product && is_decimal<Element>)
|
else if constexpr (aggregate_operation == AggregateOperation::product && is_decimal<Element>)
|
||||||
{
|
{
|
||||||
auto result_scale = data.getScale() * count;
|
auto result_scale = column->getScale() * count;
|
||||||
|
|
||||||
if (unlikely(result_scale > DecimalUtils::max_precision<AggregationType>))
|
if (unlikely(result_scale > DecimalUtils::max_precision<AggregationType>))
|
||||||
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds", result_scale);
|
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale {} is out of bounds", result_scale);
|
||||||
|
@ -40,7 +40,7 @@ struct ArrayCompactImpl
|
|||||||
|
|
||||||
typename ColVecType::MutablePtr res_values_column;
|
typename ColVecType::MutablePtr res_values_column;
|
||||||
if constexpr (is_decimal<T>)
|
if constexpr (is_decimal<T>)
|
||||||
res_values_column = ColVecType::create(src_values.size(), src_values.getScale());
|
res_values_column = ColVecType::create(src_values.size(), src_values_column->getScale());
|
||||||
else
|
else
|
||||||
res_values_column = ColVecType::create(src_values.size());
|
res_values_column = ColVecType::create(src_values.size());
|
||||||
|
|
||||||
|
@ -101,9 +101,8 @@ struct ArrayCumSumImpl
|
|||||||
typename ColVecResult::MutablePtr res_nested;
|
typename ColVecResult::MutablePtr res_nested;
|
||||||
if constexpr (is_decimal<Element>)
|
if constexpr (is_decimal<Element>)
|
||||||
{
|
{
|
||||||
const typename ColVecType::Container & data =
|
const ColVecType * column_typed = checkAndGetColumn<ColVecType>(&column_const->getDataColumn());
|
||||||
checkAndGetColumn<ColVecType>(&column_const->getDataColumn())->getData();
|
res_nested = ColVecResult::create(0, column_typed->getScale());
|
||||||
res_nested = ColVecResult::create(0, data.getScale());
|
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
res_nested = ColVecResult::create();
|
res_nested = ColVecResult::create();
|
||||||
@ -120,7 +119,7 @@ struct ArrayCumSumImpl
|
|||||||
|
|
||||||
typename ColVecResult::MutablePtr res_nested;
|
typename ColVecResult::MutablePtr res_nested;
|
||||||
if constexpr (is_decimal<Element>)
|
if constexpr (is_decimal<Element>)
|
||||||
res_nested = ColVecResult::create(0, data.getScale());
|
res_nested = ColVecResult::create(0, column->getScale());
|
||||||
else
|
else
|
||||||
res_nested = ColVecResult::create();
|
res_nested = ColVecResult::create();
|
||||||
|
|
||||||
|
@ -83,7 +83,7 @@ struct ArrayCumSumNonNegativeImpl
|
|||||||
|
|
||||||
typename ColVecResult::MutablePtr res_nested;
|
typename ColVecResult::MutablePtr res_nested;
|
||||||
if constexpr (is_decimal<Element>)
|
if constexpr (is_decimal<Element>)
|
||||||
res_nested = ColVecResult::create(0, data.getScale());
|
res_nested = ColVecResult::create(0, column->getScale());
|
||||||
else
|
else
|
||||||
res_nested = ColVecResult::create();
|
res_nested = ColVecResult::create();
|
||||||
|
|
||||||
|
@ -105,7 +105,7 @@ struct ArrayDifferenceImpl
|
|||||||
|
|
||||||
typename ColVecResult::MutablePtr res_nested;
|
typename ColVecResult::MutablePtr res_nested;
|
||||||
if constexpr (is_decimal<Element>)
|
if constexpr (is_decimal<Element>)
|
||||||
res_nested = ColVecResult::create(0, data.getScale());
|
res_nested = ColVecResult::create(0, column->getScale());
|
||||||
else
|
else
|
||||||
res_nested = ColVecResult::create();
|
res_nested = ColVecResult::create();
|
||||||
|
|
||||||
|
@ -148,7 +148,7 @@ public:
|
|||||||
UInt32 scale [[maybe_unused]] = 0;
|
UInt32 scale [[maybe_unused]] = 0;
|
||||||
if constexpr (std::is_same_v<DataType, DataTypeDateTime64>)
|
if constexpr (std::is_same_v<DataType, DataTypeDateTime64>)
|
||||||
{
|
{
|
||||||
scale = times_data.getScale();
|
scale = times->getScale();
|
||||||
}
|
}
|
||||||
|
|
||||||
auto result_column = ColumnString::create();
|
auto result_column = ColumnString::create();
|
||||||
|
@ -440,7 +440,7 @@ public:
|
|||||||
UInt32 scale [[maybe_unused]] = 0;
|
UInt32 scale [[maybe_unused]] = 0;
|
||||||
if constexpr (std::is_same_v<DataType, DataTypeDateTime64>)
|
if constexpr (std::is_same_v<DataType, DataTypeDateTime64>)
|
||||||
{
|
{
|
||||||
scale = vec.getScale();
|
scale = times->getScale();
|
||||||
}
|
}
|
||||||
|
|
||||||
auto col_res = ColumnString::create();
|
auto col_res = ColumnString::create();
|
||||||
|
Loading…
Reference in New Issue
Block a user