Merge branch 'master' into amazon-reviews-1

This commit is contained in:
Rich Raposa 2023-03-29 08:45:06 -06:00 committed by GitHub
commit f28b53db91
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
44 changed files with 461 additions and 389 deletions

View File

@ -21,18 +21,21 @@ namespace ErrorCodes
namespace
{
/// TODO Proper support for Decimal256.
template <typename T, typename LimitNumberOfElements>
struct MovingSum
{
using Data = MovingSumData<std::conditional_t<is_decimal<T>, Decimal128, NearestFieldType<T>>>;
using Data = MovingSumData<std::conditional_t<is_decimal<T>,
std::conditional_t<sizeof(T) <= sizeof(Decimal128), Decimal128, Decimal256>,
NearestFieldType<T>>>;
using Function = MovingImpl<T, LimitNumberOfElements, Data>;
};
template <typename T, typename LimitNumberOfElements>
struct MovingAvg
{
using Data = MovingAvgData<std::conditional_t<is_decimal<T>, Decimal128, Float64>>;
using Data = MovingAvgData<std::conditional_t<is_decimal<T>,
std::conditional_t<sizeof(T) <= sizeof(Decimal128), Decimal128, Decimal256>,
Float64>>;
using Function = MovingImpl<T, LimitNumberOfElements, Data>;
};

View File

@ -33,11 +33,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
struct Settings;
enum class StatisticsFunctionKind
@ -57,7 +52,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<is_decimal<T>, VarMomentsDecimal<Decimal128, _level>, VarMoments<ResultType, _level>>;
using Data = VarMoments<ResultType, _level>;
static constexpr UInt32 num_args = 1;
};
@ -89,12 +84,11 @@ public:
explicit AggregateFunctionVarianceSimple(const DataTypes & argument_types_, StatisticsFunctionKind kind_)
: IAggregateFunctionDataHelper<typename StatFunc::Data, AggregateFunctionVarianceSimple<StatFunc>>(argument_types_, {}, std::make_shared<DataTypeNumber<ResultType>>())
, src_scale(0), kind(kind_)
{}
AggregateFunctionVarianceSimple(const IDataType & data_type, const DataTypes & argument_types_, StatisticsFunctionKind kind_)
: IAggregateFunctionDataHelper<typename StatFunc::Data, AggregateFunctionVarianceSimple<StatFunc>>(argument_types_, {}, std::make_shared<DataTypeNumber<ResultType>>())
, src_scale(getDecimalScale(data_type)), kind(kind_)
{}
{
chassert(!argument_types_.empty());
if (isDecimal(argument_types_.front()))
src_scale = getDecimalScale(*argument_types_.front());
}
String getName() const override
{
@ -113,8 +107,9 @@ public:
{
if constexpr (is_decimal<T1>)
{
this->data(place).add(static_cast<ResultType>(
static_cast<const ColVecT1 &>(*columns[0]).getData()[row_num].value));
this->data(place).add(
convertFromDecimal<DataTypeDecimal<T1>, DataTypeFloat64>(
static_cast<const ColVecT1 &>(*columns[0]).getData()[row_num], src_scale));
}
else
this->data(place).add(
@ -142,161 +137,86 @@ public:
const auto & data = this->data(place);
auto & dst = static_cast<ColVecResult &>(to).getData();
if constexpr (is_decimal<T1>)
switch (kind)
{
switch (kind)
case StatisticsFunctionKind::varPop:
{
case StatisticsFunctionKind::varPop:
{
dst.push_back(data.getPopulation(src_scale * 2));
break;
}
case StatisticsFunctionKind::varSamp:
{
dst.push_back(data.getSample(src_scale * 2));
break;
}
case StatisticsFunctionKind::stddevPop:
{
dst.push_back(sqrt(data.getPopulation(src_scale * 2)));
break;
}
case StatisticsFunctionKind::stddevSamp:
{
dst.push_back(sqrt(data.getSample(src_scale * 2)));
break;
}
case StatisticsFunctionKind::skewPop:
{
Float64 var_value = data.getPopulation(src_scale * 2);
if (var_value > 0)
dst.push_back(data.getMoment3(src_scale * 3) / pow(var_value, 1.5));
else
dst.push_back(std::numeric_limits<Float64>::quiet_NaN());
break;
}
case StatisticsFunctionKind::skewSamp:
{
Float64 var_value = data.getSample(src_scale * 2);
if (var_value > 0)
dst.push_back(data.getMoment3(src_scale * 3) / pow(var_value, 1.5));
else
dst.push_back(std::numeric_limits<Float64>::quiet_NaN());
break;
}
case StatisticsFunctionKind::kurtPop:
{
Float64 var_value = data.getPopulation(src_scale * 2);
if (var_value > 0)
dst.push_back(data.getMoment4(src_scale * 4) / pow(var_value, 2));
else
dst.push_back(std::numeric_limits<Float64>::quiet_NaN());
break;
}
case StatisticsFunctionKind::kurtSamp:
{
Float64 var_value = data.getSample(src_scale * 2);
if (var_value > 0)
dst.push_back(data.getMoment4(src_scale * 4) / pow(var_value, 2));
else
dst.push_back(std::numeric_limits<Float64>::quiet_NaN());
break;
}
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected statistical function kind");
dst.push_back(data.getPopulation());
break;
}
}
else
{
switch (kind)
case StatisticsFunctionKind::varSamp:
{
case StatisticsFunctionKind::varPop:
{
dst.push_back(data.getPopulation());
break;
}
case StatisticsFunctionKind::varSamp:
{
dst.push_back(data.getSample());
break;
}
case StatisticsFunctionKind::stddevPop:
{
dst.push_back(sqrt(data.getPopulation()));
break;
}
case StatisticsFunctionKind::stddevSamp:
{
dst.push_back(sqrt(data.getSample()));
break;
}
case StatisticsFunctionKind::skewPop:
{
ResultType var_value = data.getPopulation();
dst.push_back(data.getSample());
break;
}
case StatisticsFunctionKind::stddevPop:
{
dst.push_back(sqrt(data.getPopulation()));
break;
}
case StatisticsFunctionKind::stddevSamp:
{
dst.push_back(sqrt(data.getSample()));
break;
}
case StatisticsFunctionKind::skewPop:
{
ResultType var_value = data.getPopulation();
if (var_value > 0)
dst.push_back(static_cast<ResultType>(data.getMoment3() / pow(var_value, 1.5)));
else
dst.push_back(std::numeric_limits<ResultType>::quiet_NaN());
if (var_value > 0)
dst.push_back(static_cast<ResultType>(data.getMoment3() / pow(var_value, 1.5)));
else
dst.push_back(std::numeric_limits<ResultType>::quiet_NaN());
break;
}
case StatisticsFunctionKind::skewSamp:
{
ResultType var_value = data.getSample();
break;
}
case StatisticsFunctionKind::skewSamp:
{
ResultType var_value = data.getSample();
if (var_value > 0)
dst.push_back(static_cast<ResultType>(data.getMoment3() / pow(var_value, 1.5)));
else
dst.push_back(std::numeric_limits<ResultType>::quiet_NaN());
if (var_value > 0)
dst.push_back(static_cast<ResultType>(data.getMoment3() / pow(var_value, 1.5)));
else
dst.push_back(std::numeric_limits<ResultType>::quiet_NaN());
break;
}
case StatisticsFunctionKind::kurtPop:
{
ResultType var_value = data.getPopulation();
break;
}
case StatisticsFunctionKind::kurtPop:
{
ResultType var_value = data.getPopulation();
if (var_value > 0)
dst.push_back(static_cast<ResultType>(data.getMoment4() / pow(var_value, 2)));
else
dst.push_back(std::numeric_limits<ResultType>::quiet_NaN());
if (var_value > 0)
dst.push_back(static_cast<ResultType>(data.getMoment4() / pow(var_value, 2)));
else
dst.push_back(std::numeric_limits<ResultType>::quiet_NaN());
break;
}
case StatisticsFunctionKind::kurtSamp:
{
ResultType var_value = data.getSample();
break;
}
case StatisticsFunctionKind::kurtSamp:
{
ResultType var_value = data.getSample();
if (var_value > 0)
dst.push_back(static_cast<ResultType>(data.getMoment4() / pow(var_value, 2)));
else
dst.push_back(std::numeric_limits<ResultType>::quiet_NaN());
if (var_value > 0)
dst.push_back(static_cast<ResultType>(data.getMoment4() / pow(var_value, 2)));
else
dst.push_back(std::numeric_limits<ResultType>::quiet_NaN());
break;
}
case StatisticsFunctionKind::covarPop:
{
dst.push_back(data.getPopulation());
break;
}
case StatisticsFunctionKind::covarSamp:
{
dst.push_back(data.getSample());
break;
}
case StatisticsFunctionKind::corr:
{
dst.push_back(data.get());
break;
}
break;
}
case StatisticsFunctionKind::covarPop:
{
dst.push_back(data.getPopulation());
break;
}
case StatisticsFunctionKind::covarSamp:
{
dst.push_back(data.getSample());
break;
}
case StatisticsFunctionKind::corr:
{
dst.push_back(data.get());
break;
}
}
}
@ -327,7 +247,7 @@ AggregateFunctionPtr createAggregateFunctionStatisticsUnary(
AggregateFunctionPtr res;
const DataTypePtr & data_type = argument_types[0];
if (isDecimal(data_type))
res.reset(createWithDecimalType<FunctionTemplate>(*data_type, *data_type, argument_types, kind));
res.reset(createWithDecimalType<FunctionTemplate>(*data_type, argument_types, kind));
else
res.reset(createWithNumericType<FunctionTemplate>(*data_type, argument_types, kind));

View File

@ -19,7 +19,7 @@ namespace
template <typename T>
struct SumSimple
{
/// @note It uses slow Decimal128 (cause we need such a variant). sumWithOverflow is faster for Decimal32/64
/// @note It uses slow Decimal128/256 (cause we need such a variant). sumWithOverflow is faster for Decimal32/64
using ResultType = std::conditional_t<is_decimal<T>,
std::conditional_t<std::is_same_v<T, Decimal256>, Decimal256, Decimal128>,
NearestFieldType<T>>;

View File

@ -41,6 +41,8 @@ static IAggregateFunction * createAggregateFunctionSingleValue(const String & na
return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Decimal64>>>(argument_type);
if (which.idx == TypeIndex::Decimal128)
return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Decimal128>>>(argument_type);
if (which.idx == TypeIndex::Decimal256)
return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Decimal256>>>(argument_type);
if (which.idx == TypeIndex::String)
return new AggregateFunctionTemplate<Data<SingleValueDataString>>(argument_type);
@ -72,6 +74,8 @@ static IAggregateFunction * createAggregateFunctionArgMinMaxSecond(const DataTyp
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Decimal64>>>>(res_type, val_type);
if (which.idx == TypeIndex::Decimal128)
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Decimal128>>>>(res_type, val_type);
if (which.idx == TypeIndex::Decimal256)
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Decimal256>>>>(res_type, val_type);
if (which.idx == TypeIndex::String)
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataString>>>(res_type, val_type);
@ -106,6 +110,8 @@ static IAggregateFunction * createAggregateFunctionArgMinMax(const String & name
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Decimal64>>(res_type, val_type);
if (which.idx == TypeIndex::Decimal128)
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Decimal128>>(res_type, val_type);
if (which.idx == TypeIndex::Decimal256)
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Decimal256>>(res_type, val_type);
if (which.idx == TypeIndex::String)
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataString>(res_type, val_type);

View File

@ -17,7 +17,6 @@ struct Settings;
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int DECIMAL_OVERFLOW;
extern const int LOGICAL_ERROR;
}
@ -136,114 +135,6 @@ struct VarMoments
}
};
template <typename T, size_t _level>
class VarMomentsDecimal
{
public:
using NativeType = typename T::NativeType;
void add(NativeType x)
{
++m0;
getM(1) += x;
NativeType tmp;
bool overflow = common::mulOverflow(x, x, tmp) || common::addOverflow(getM(2), tmp, getM(2));
if constexpr (_level >= 3)
overflow = overflow || common::mulOverflow(tmp, x, tmp) || common::addOverflow(getM(3), tmp, getM(3));
if constexpr (_level >= 4)
overflow = overflow || common::mulOverflow(tmp, x, tmp) || common::addOverflow(getM(4), tmp, getM(4));
if (overflow)
throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Decimal math overflow");
}
void merge(const VarMomentsDecimal & rhs)
{
m0 += rhs.m0;
getM(1) += rhs.getM(1);
bool overflow = common::addOverflow(getM(2), rhs.getM(2), getM(2));
if constexpr (_level >= 3)
overflow = overflow || common::addOverflow(getM(3), rhs.getM(3), getM(3));
if constexpr (_level >= 4)
overflow = overflow || common::addOverflow(getM(4), rhs.getM(4), getM(4));
if (overflow)
throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Decimal math overflow");
}
void write(WriteBuffer & buf) const { writePODBinary(*this, buf); }
void read(ReadBuffer & buf) { readPODBinary(*this, buf); }
Float64 get() const
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Variation moments should be obtained by either 'getSample' or 'getPopulation' method");
}
Float64 getPopulation(UInt32 scale) const
{
if (m0 == 0)
return std::numeric_limits<Float64>::infinity();
NativeType tmp;
if (common::mulOverflow(getM(1), getM(1), tmp) ||
common::subOverflow(getM(2), NativeType(tmp / m0), tmp))
throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Decimal math overflow");
return std::max(Float64{}, DecimalUtils::convertTo<Float64>(T(tmp / m0), scale));
}
Float64 getSample(UInt32 scale) const
{
if (m0 == 0)
return std::numeric_limits<Float64>::quiet_NaN();
if (m0 == 1)
return std::numeric_limits<Float64>::infinity();
NativeType tmp;
if (common::mulOverflow(getM(1), getM(1), tmp) ||
common::subOverflow(getM(2), NativeType(tmp / m0), tmp))
throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Decimal math overflow");
return std::max(Float64{}, DecimalUtils::convertTo<Float64>(T(tmp / (m0 - 1)), scale));
}
Float64 getMoment3(UInt32 scale) const
{
if (m0 == 0)
return std::numeric_limits<Float64>::infinity();
NativeType tmp;
if (common::mulOverflow(2 * getM(1), getM(1), tmp) ||
common::subOverflow(3 * getM(2), NativeType(tmp / m0), tmp) ||
common::mulOverflow(tmp, getM(1), tmp) ||
common::subOverflow(getM(3), NativeType(tmp / m0), tmp))
throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Decimal math overflow");
return DecimalUtils::convertTo<Float64>(T(tmp / m0), scale);
}
Float64 getMoment4(UInt32 scale) const
{
if (m0 == 0)
return std::numeric_limits<Float64>::infinity();
NativeType tmp;
if (common::mulOverflow(3 * getM(1), getM(1), tmp) ||
common::subOverflow(6 * getM(2), NativeType(tmp / m0), tmp) ||
common::mulOverflow(tmp, getM(1), tmp) ||
common::subOverflow(4 * getM(3), NativeType(tmp / m0), tmp) ||
common::mulOverflow(tmp, getM(1), tmp) ||
common::subOverflow(getM(4), NativeType(tmp / m0), tmp))
throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Decimal math overflow");
return DecimalUtils::convertTo<Float64>(T(tmp / m0), scale);
}
private:
UInt64 m0{};
NativeType m[_level]{};
NativeType & getM(size_t i) { return m[i - 1]; }
const NativeType & getM(size_t i) const { return m[i - 1]; }
};
/**
Calculating multivariate central moments

View File

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

View File

@ -34,7 +34,7 @@ struct QuantileInterpolatedWeighted
using Weight = UInt64;
using UnderlyingType = NativeType<Value>;
using Hasher = std::conditional_t<std::is_same_v<Value, Decimal128>, Int128Hash, HashCRC32<UnderlyingType>>;
using Hasher = HashCRC32<UnderlyingType>;
/// When creating, the hash table must be small.
using Map = HashMapWithStackMemory<UnderlyingType, Weight, Hasher, 4>;

View File

@ -107,8 +107,9 @@ Field QueryFuzzer::fuzzField(Field field)
type_index = 1;
}
else if (type == Field::Types::Decimal32
|| type == Field::Types::Decimal64
|| type == Field::Types::Decimal128)
|| type == Field::Types::Decimal64
|| type == Field::Types::Decimal128
|| type == Field::Types::Decimal256)
{
type_index = 2;
}

View File

@ -68,7 +68,8 @@ bool Span::addAttribute(const Exception & e) noexcept
if (!this->isTraceEnabled())
return false;
return addAttributeImpl("clickhouse.exception", getExceptionMessage(e, false));
return addAttributeImpl("clickhouse.exception", getExceptionMessage(e, false))
&& addAttributeImpl("clickhouse.exception_code", toString(e.code()));
}
bool Span::addAttribute(std::exception_ptr e) noexcept
@ -79,6 +80,15 @@ bool Span::addAttribute(std::exception_ptr e) noexcept
return addAttributeImpl("clickhouse.exception", getExceptionMessage(e, false));
}
bool Span::addAttribute(const ExecutionStatus & e) noexcept
{
if (!this->isTraceEnabled())
return false;
return addAttributeImpl("clickhouse.exception", e.message)
&& addAttributeImpl("clickhouse.exception_code", toString(e.code));
}
bool Span::addAttributeImpl(std::string_view name, std::string_view value) noexcept
{
try

View File

@ -9,6 +9,7 @@ struct Settings;
class OpenTelemetrySpanLog;
class WriteBuffer;
class ReadBuffer;
struct ExecutionStatus;
namespace OpenTelemetry
{
@ -57,6 +58,7 @@ struct Span
bool addAttribute(std::string_view name, std::function<String()> value_supplier) noexcept;
bool addAttribute(const Exception & e) noexcept;
bool addAttribute(std::exception_ptr e) noexcept;
bool addAttribute(const ExecutionStatus & e) noexcept;
bool isTraceEnabled() const
{

View File

@ -11,6 +11,7 @@
#include <Poco/Util/Application.h>
#include <Poco/Util/LayeredConfiguration.h>
#include <base/demangle.h>
namespace DB
{
@ -29,6 +30,7 @@ namespace CurrentMetrics
extern const Metric LocalThreadActive;
}
static constexpr auto DEFAULT_THREAD_NAME = "ThreadPool";
template <typename Thread>
ThreadPoolImpl<Thread>::ThreadPoolImpl()
@ -342,7 +344,7 @@ void ThreadPoolImpl<Thread>::worker(typename std::list<Thread>::iterator thread_
while (true)
{
/// This is inside the loop to also reset previous thread names set inside the jobs.
setThreadName("ThreadPool");
setThreadName(DEFAULT_THREAD_NAME);
/// A copy of parent trace context
DB::OpenTelemetry::TracingContextOnThread parent_thead_trace_context;
@ -389,10 +391,17 @@ void ThreadPoolImpl<Thread>::worker(typename std::list<Thread>::iterator thread_
if (thread_trace_context.root_span.isTraceEnabled())
{
/// Use the thread name as operation name so that the tracing log will be more clear.
/// The thread name is usually set in the jobs, we can only get the name after the job finishes
/// The thread name is usually set in jobs, we can only get the name after the job finishes
std::string thread_name = getThreadName();
if (!thread_name.empty())
if (!thread_name.empty() && thread_name != DEFAULT_THREAD_NAME)
{
thread_trace_context.root_span.operation_name = thread_name;
}
else
{
/// If the thread name is not set, use the type name of the job instead
thread_trace_context.root_span.operation_name = demangle(job.target_type().name());
}
}
/// job should be reset before decrementing scheduled_jobs to

View File

@ -37,6 +37,7 @@ ColumnTypeSpec convertTypeIndexToPostgresColumnTypeSpec(TypeIndex type_index)
case TypeIndex::Decimal32:
case TypeIndex::Decimal64:
case TypeIndex::Decimal128:
case TypeIndex::Decimal256:
return {ColumnType::NUMERIC, -1};
case TypeIndex::UUID:

View File

@ -40,8 +40,10 @@ bool DataTypeDecimal<T>::equals(const IDataType & rhs) const
template <is_decimal T>
DataTypePtr DataTypeDecimal<T>::promoteNumericType() const
{
using PromotedType = DataTypeDecimal<Decimal128>;
return std::make_shared<PromotedType>(PromotedType::maxPrecision(), this->scale);
if (sizeof(T) <= sizeof(Decimal128))
return std::make_shared<DataTypeDecimal<Decimal128>>(DataTypeDecimal<Decimal128>::maxPrecision(), this->scale);
else
return std::make_shared<DataTypeDecimal<Decimal256>>(DataTypeDecimal<Decimal256>::maxPrecision(), this->scale);
}
template <is_decimal T>

View File

@ -115,6 +115,8 @@ DataTypePtr convertMySQLDataType(MultiEnum<MySQLDataTypesSupport> type_support,
res = std::make_shared<DataTypeDecimal<Decimal64>>(precision, scale);
else if (precision <= DecimalUtils::max_precision<Decimal128>)
res = std::make_shared<DataTypeDecimal<Decimal128>>(precision, scale);
else if (precision <= DecimalUtils::max_precision<Decimal256>)
res = std::make_shared<DataTypeDecimal<Decimal256>>(precision, scale);
}
/// Also String is fallback for all unknown types.

View File

@ -15,11 +15,9 @@
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeFactory.h>
#include <base/EnumReflection.h>
namespace DB
@ -528,13 +526,14 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
size_t have_decimal32 = type_ids.count(TypeIndex::Decimal32);
size_t have_decimal64 = type_ids.count(TypeIndex::Decimal64);
size_t have_decimal128 = type_ids.count(TypeIndex::Decimal128);
size_t have_decimal256 = type_ids.count(TypeIndex::Decimal256);
if (have_decimal32 || have_decimal64 || have_decimal128)
if (have_decimal32 || have_decimal64 || have_decimal128 || have_decimal256)
{
size_t num_supported = have_decimal32 + have_decimal64 + have_decimal128;
size_t num_supported = have_decimal32 + have_decimal64 + have_decimal128 + have_decimal256;
std::vector<TypeIndex> int_ids = {TypeIndex::Int8, TypeIndex::UInt8, TypeIndex::Int16, TypeIndex::UInt16,
TypeIndex::Int32, TypeIndex::UInt32, TypeIndex::Int64, TypeIndex::UInt64};
std::array<TypeIndex, 8> int_ids = {TypeIndex::Int8, TypeIndex::UInt8, TypeIndex::Int16, TypeIndex::UInt16,
TypeIndex::Int32, TypeIndex::UInt32, TypeIndex::Int64, TypeIndex::UInt64};
TypeIndex max_int = TypeIndex::Nothing;
for (auto int_id : int_ids)
@ -552,8 +551,13 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
for (const auto & type : types)
{
auto type_id = type->getTypeId();
if (type_id != TypeIndex::Decimal32 && type_id != TypeIndex::Decimal64 && type_id != TypeIndex::Decimal128)
if (type_id != TypeIndex::Decimal32
&& type_id != TypeIndex::Decimal64
&& type_id != TypeIndex::Decimal128
&& type_id != TypeIndex::Decimal256)
{
continue;
}
UInt32 scale = getDecimalScale(*type);
if (scale > max_scale)
@ -571,11 +575,13 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
min_precision = DataTypeDecimal<Decimal64>::maxPrecision();
}
if (min_precision > DataTypeDecimal<Decimal128>::maxPrecision())
if (min_precision > DataTypeDecimal<Decimal256>::maxPrecision())
return throwOrReturn<on_error>(types, "because the least supertype is Decimal("
+ toString(min_precision) + ',' + toString(max_scale) + ')',
ErrorCodes::NO_COMMON_TYPE);
if (have_decimal256 || min_precision > DataTypeDecimal<Decimal128>::maxPrecision())
return std::make_shared<DataTypeDecimal<Decimal256>>(DataTypeDecimal<Decimal256>::maxPrecision(), max_scale);
if (have_decimal128 || min_precision > DataTypeDecimal<Decimal64>::maxPrecision())
return std::make_shared<DataTypeDecimal<Decimal128>>(DataTypeDecimal<Decimal128>::maxPrecision(), max_scale);
if (have_decimal64 || min_precision > DataTypeDecimal<Decimal32>::maxPrecision())

View File

@ -11,6 +11,7 @@
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
namespace DB
@ -280,6 +281,10 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
minimize(min_bits_of_unsigned_integer, 32);
else if (typeid_cast<const DataTypeUInt64 *>(type.get()))
minimize(min_bits_of_unsigned_integer, 64);
else if (typeid_cast<const DataTypeUInt128 *>(type.get()))
minimize(min_bits_of_unsigned_integer, 128);
else if (typeid_cast<const DataTypeUInt256 *>(type.get()))
minimize(min_bits_of_unsigned_integer, 256);
else if (typeid_cast<const DataTypeInt8 *>(type.get()))
minimize(min_bits_of_signed_integer, 8);
else if (typeid_cast<const DataTypeInt16 *>(type.get()))
@ -288,6 +293,10 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
minimize(min_bits_of_signed_integer, 32);
else if (typeid_cast<const DataTypeInt64 *>(type.get()))
minimize(min_bits_of_signed_integer, 64);
else if (typeid_cast<const DataTypeInt128 *>(type.get()))
minimize(min_bits_of_signed_integer, 128);
else if (typeid_cast<const DataTypeInt256 *>(type.get()))
minimize(min_bits_of_signed_integer, 256);
else if (typeid_cast<const DataTypeFloat32 *>(type.get()))
minimize(min_mantissa_bits_of_floating, 24);
else if (typeid_cast<const DataTypeFloat64 *>(type.get()))
@ -326,6 +335,10 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
return std::make_shared<DataTypeUInt32>();
else if (min_bits_of_unsigned_integer <= 64)
return std::make_shared<DataTypeUInt64>();
else if (min_bits_of_unsigned_integer <= 128)
return std::make_shared<DataTypeUInt128>();
else if (min_bits_of_unsigned_integer <= 256)
return std::make_shared<DataTypeUInt256>();
else
throw Exception(ErrorCodes::NO_COMMON_TYPE,
"Logical error: {} but as all data types are integers, "
@ -343,6 +356,10 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
return std::make_shared<DataTypeInt32>();
else if (min_bits_of_signed_integer <= 64)
return std::make_shared<DataTypeInt64>();
else if (min_bits_of_signed_integer <= 128)
return std::make_shared<DataTypeInt128>();
else if (min_bits_of_signed_integer <= 256)
return std::make_shared<DataTypeInt256>();
else
throw Exception(ErrorCodes::NO_COMMON_TYPE,
"Logical error: {} but as all data types are integers, "
@ -351,6 +368,29 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
}
}
/// Decimals
{
bool all_decimals = true;
UInt32 min_scale = std::numeric_limits<UInt32>::max();
UInt32 min_precision = std::numeric_limits<UInt32>::max();
for (const auto & type : types)
{
if (isDecimal(type))
{
min_scale = std::min(min_scale, getDecimalScale(*type));
min_precision = std::min(min_precision, getDecimalPrecision(*type));
}
else
{
all_decimals = false;
break;
}
}
if (all_decimals)
return createDecimal<DataTypeDecimal>(min_precision, min_scale);
}
/// All other data types (UUID, AggregateFunction, Enum...) are compatible only if they are the same (checked in trivial cases).
return get_nothing_or_throw("");
}

View File

@ -299,6 +299,7 @@ public:
tryExecuteDecimal<Decimal32>(column, res_column) ||
tryExecuteDecimal<Decimal64>(column, res_column) ||
tryExecuteDecimal<Decimal128>(column, res_column) ||
tryExecuteDecimal<Decimal256>(column, res_column) ||
tryExecuteUUID(column, res_column) ||
tryExecuteIPv4(column, res_column) ||
tryExecuteIPv6(column, res_column))

View File

@ -721,9 +721,6 @@ struct NameDictGetDateTime { static constexpr auto name = "dictGetDateTime"; };
struct NameDictGetUUID { static constexpr auto name = "dictGetUUID"; };
struct NameDictGetIPv4 { static constexpr auto name = "dictGetIPv4"; };
struct NameDictGetIPv6 { static constexpr auto name = "dictGetIPv6"; };
struct NameDictGetDecimal32 { static constexpr auto name = "dictGetDecimal32"; };
struct NameDictGetDecimal64 { static constexpr auto name = "dictGetDecimal64"; };
struct NameDictGetDecimal128 { static constexpr auto name = "dictGetDecimal128"; };
struct NameDictGetString { static constexpr auto name = "dictGetString"; };
using FunctionDictGetUInt8 = FunctionDictGet<DataTypeUInt8, NameDictGetUInt8>;
@ -741,9 +738,6 @@ using FunctionDictGetDateTime = FunctionDictGet<DataTypeDateTime, NameDictGetDat
using FunctionDictGetUUID = FunctionDictGet<DataTypeUUID, NameDictGetUUID>;
using FunctionDictGetIPv4 = FunctionDictGet<DataTypeIPv4, NameDictGetIPv4>;
using FunctionDictGetIPv6 = FunctionDictGet<DataTypeIPv6, NameDictGetIPv6>;
using FunctionDictGetDecimal32 = FunctionDictGet<DataTypeDecimal<Decimal32>, NameDictGetDecimal32>;
using FunctionDictGetDecimal64 = FunctionDictGet<DataTypeDecimal<Decimal64>, NameDictGetDecimal64>;
using FunctionDictGetDecimal128 = FunctionDictGet<DataTypeDecimal<Decimal128>, NameDictGetDecimal128>;
using FunctionDictGetString = FunctionDictGet<DataTypeString, NameDictGetString>;
template<typename DataType, typename Name>
@ -764,9 +758,6 @@ struct NameDictGetDateTimeOrDefault { static constexpr auto name = "dictGetDateT
struct NameDictGetUUIDOrDefault { static constexpr auto name = "dictGetUUIDOrDefault"; };
struct NameDictGetIPv4OrDefault { static constexpr auto name = "dictGetIPv4OrDefault"; };
struct NameDictGetIPv6OrDefault { static constexpr auto name = "dictGetIPv6OrDefault"; };
struct NameDictGetDecimal32OrDefault { static constexpr auto name = "dictGetDecimal32OrDefault"; };
struct NameDictGetDecimal64OrDefault { static constexpr auto name = "dictGetDecimal64OrDefault"; };
struct NameDictGetDecimal128OrDefault { static constexpr auto name = "dictGetDecimal128OrDefault"; };
struct NameDictGetStringOrDefault { static constexpr auto name = "dictGetStringOrDefault"; };
using FunctionDictGetUInt8OrDefault = FunctionDictGetOrDefault<DataTypeUInt8, NameDictGetUInt8OrDefault>;
@ -784,9 +775,6 @@ using FunctionDictGetDateTimeOrDefault = FunctionDictGetOrDefault<DataTypeDateTi
using FunctionDictGetUUIDOrDefault = FunctionDictGetOrDefault<DataTypeUUID, NameDictGetUUIDOrDefault>;
using FunctionDictGetIPv4OrDefault = FunctionDictGetOrDefault<DataTypeIPv4, NameDictGetIPv4OrDefault>;
using FunctionDictGetIPv6OrDefault = FunctionDictGetOrDefault<DataTypeIPv6, NameDictGetIPv6OrDefault>;
using FunctionDictGetDecimal32OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal32>, NameDictGetDecimal32OrDefault>;
using FunctionDictGetDecimal64OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal64>, NameDictGetDecimal64OrDefault>;
using FunctionDictGetDecimal128OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal128>, NameDictGetDecimal128OrDefault>;
using FunctionDictGetStringOrDefault = FunctionDictGetOrDefault<DataTypeString, NameDictGetStringOrDefault>;
class FunctionDictGetOrNull final : public IFunction

View File

@ -750,42 +750,12 @@ public:
};
template <typename JSONParser>
using JSONExtractInt8Impl = JSONExtractNumericImpl<JSONParser, Int8>;
template <typename JSONParser>
using JSONExtractUInt8Impl = JSONExtractNumericImpl<JSONParser, UInt8>;
template <typename JSONParser>
using JSONExtractInt16Impl = JSONExtractNumericImpl<JSONParser, Int16>;
template <typename JSONParser>
using JSONExtractUInt16Impl = JSONExtractNumericImpl<JSONParser, UInt16>;
template <typename JSONParser>
using JSONExtractInt32Impl = JSONExtractNumericImpl<JSONParser, Int32>;
template <typename JSONParser>
using JSONExtractUInt32Impl = JSONExtractNumericImpl<JSONParser, UInt32>;
template <typename JSONParser>
using JSONExtractInt64Impl = JSONExtractNumericImpl<JSONParser, Int64>;
template <typename JSONParser>
using JSONExtractUInt64Impl = JSONExtractNumericImpl<JSONParser, UInt64>;
template <typename JSONParser>
using JSONExtractInt128Impl = JSONExtractNumericImpl<JSONParser, Int128>;
template <typename JSONParser>
using JSONExtractUInt128Impl = JSONExtractNumericImpl<JSONParser, UInt128>;
template <typename JSONParser>
using JSONExtractInt256Impl = JSONExtractNumericImpl<JSONParser, Int256>;
template <typename JSONParser>
using JSONExtractUInt256Impl = JSONExtractNumericImpl<JSONParser, UInt256>;
template <typename JSONParser>
using JSONExtractFloat32Impl = JSONExtractNumericImpl<JSONParser, Float32>;
template <typename JSONParser>
using JSONExtractFloat64Impl = JSONExtractNumericImpl<JSONParser, Float64>;
template <typename JSONParser>
using JSONExtractDecimal32Impl = JSONExtractNumericImpl<JSONParser, Decimal32>;
template <typename JSONParser>
using JSONExtractDecimal64Impl = JSONExtractNumericImpl<JSONParser, Decimal64>;
template <typename JSONParser>
using JSONExtractDecimal128Impl = JSONExtractNumericImpl<JSONParser, Decimal128>;
template <typename JSONParser>
using JSONExtractDecimal256Impl = JSONExtractNumericImpl<JSONParser, Decimal256>;
template <typename JSONParser>

View File

@ -703,7 +703,8 @@ public:
&& !executeNum<Float64>(in, out, boundaries)
&& !executeDecimal<Decimal32>(in, out, boundaries)
&& !executeDecimal<Decimal64>(in, out, boundaries)
&& !executeDecimal<Decimal128>(in, out, boundaries))
&& !executeDecimal<Decimal128>(in, out, boundaries)
&& !executeDecimal<Decimal256>(in, out, boundaries))
{
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName());
}

View File

@ -38,7 +38,7 @@ enum class AggregateOperation
* During array aggregation we derive result type from operation.
* For array min or array max we use array element as result type.
* For array average we use Float64.
* For array sum for for big integers, we use same type representation, decimal numbers we use Decimal128,
* For array sum for big integers, we use same type representation, decimal numbers up to 128-bit will use Decimal128, then Decimal256.
* for floating point numbers Float64, for numeric unsigned Int64, and for numeric signed UInt64.
*/
@ -77,10 +77,13 @@ 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<is_decimal<ArrayElement>, Decimal128,
std::conditional_t<std::is_same_v<ArrayElement, Decimal32>, Decimal128,
std::conditional_t<std::is_same_v<ArrayElement, Decimal64>, Decimal128,
std::conditional_t<std::is_same_v<ArrayElement, Decimal128>, Decimal128,
std::conditional_t<std::is_same_v<ArrayElement, Decimal256>, Decimal256,
std::conditional_t<std::is_floating_point_v<ArrayElement>, Float64,
std::conditional_t<std::is_signed_v<ArrayElement>, Int64,
UInt64>>>>>>>;
UInt64>>>>>>>>>>;
};
template <typename ArrayElement, AggregateOperation operation>
@ -364,8 +367,11 @@ struct ArrayAggregateImpl
executeType<Float64>(mapped, offsets, res) ||
executeType<Decimal32>(mapped, offsets, res) ||
executeType<Decimal64>(mapped, offsets, res) ||
executeType<Decimal128>(mapped, offsets, res))
executeType<Decimal128>(mapped, offsets, res) ||
executeType<Decimal256>(mapped, offsets, res))
{
return res;
}
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Unexpected column for arraySum: {}", mapped->getName());
}
@ -396,4 +402,3 @@ REGISTER_FUNCTION(ArrayAggregation)
}
}

View File

@ -151,7 +151,8 @@ struct ArrayCompactImpl
executeType<Float64>(mapped, array, res)) ||
executeType<Decimal32>(mapped, array, res) ||
executeType<Decimal64>(mapped, array, res) ||
executeType<Decimal128>(mapped, array, res))
executeType<Decimal128>(mapped, array, res) ||
executeType<Decimal256>(mapped, array, res))
{
executeGeneric(mapped, array, res);
}
@ -168,4 +169,3 @@ REGISTER_FUNCTION(ArrayCompact)
}
}

View File

@ -41,7 +41,11 @@ struct ArrayCumSumImpl
if (which.isDecimal())
{
UInt32 scale = getDecimalScale(*expression_return);
DataTypePtr nested = std::make_shared<DataTypeDecimal<Decimal128>>(DecimalUtils::max_precision<Decimal128>, scale);
DataTypePtr nested;
if (which.isDecimal256())
nested = std::make_shared<DataTypeDecimal<Decimal256>>(DecimalUtils::max_precision<Decimal256>, scale);
else
nested = std::make_shared<DataTypeDecimal<Decimal128>>(DecimalUtils::max_precision<Decimal128>, scale);
return std::make_shared<DataTypeArray>(nested);
}
@ -151,7 +155,8 @@ struct ArrayCumSumImpl
executeType<Float64,Float64>(mapped, array, res) ||
executeType<Decimal32, Decimal128>(mapped, array, res) ||
executeType<Decimal64, Decimal128>(mapped, array, res) ||
executeType<Decimal128, Decimal128>(mapped, array, res))
executeType<Decimal128, Decimal128>(mapped, array, res) ||
executeType<Decimal256, Decimal256>(mapped, array, res))
return res;
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Unexpected column for arrayCumSum: {}", mapped->getName());
@ -168,4 +173,3 @@ REGISTER_FUNCTION(ArrayCumSum)
}
}

View File

@ -42,7 +42,11 @@ struct ArrayCumSumNonNegativeImpl
if (which.isDecimal())
{
UInt32 scale = getDecimalScale(*expression_return);
DataTypePtr nested = std::make_shared<DataTypeDecimal<Decimal128>>(DecimalUtils::max_precision<Decimal128>, scale);
DataTypePtr nested;
if (which.isDecimal256())
nested = std::make_shared<DataTypeDecimal<Decimal256>>(DecimalUtils::max_precision<Decimal256>, scale);
else
nested = std::make_shared<DataTypeDecimal<Decimal128>>(DecimalUtils::max_precision<Decimal128>, scale);
return std::make_shared<DataTypeArray>(nested);
}
@ -116,7 +120,8 @@ struct ArrayCumSumNonNegativeImpl
executeType<Float64,Float64>(mapped, array, res) ||
executeType<Decimal32, Decimal128>(mapped, array, res) ||
executeType<Decimal64, Decimal128>(mapped, array, res) ||
executeType<Decimal128, Decimal128>(mapped, array, res))
executeType<Decimal128, Decimal128>(mapped, array, res) ||
executeType<Decimal256, Decimal256>(mapped, array, res))
return res;
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Unexpected column for arrayCumSumNonNegativeImpl: {}", mapped->getName());
@ -133,4 +138,3 @@ REGISTER_FUNCTION(ArrayCumSumNonNegative)
}
}

View File

@ -145,7 +145,8 @@ struct ArrayDifferenceImpl
executeType<Float64,Float64>(mapped, array, res) ||
executeType<Decimal32, Decimal32>(mapped, array, res) ||
executeType<Decimal64, Decimal64>(mapped, array, res) ||
executeType<Decimal128, Decimal128>(mapped, array, res))
executeType<Decimal128, Decimal128>(mapped, array, res) ||
executeType<Decimal256, Decimal256>(mapped, array, res))
return res;
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Unexpected column for arrayDifference: {}", mapped->getName());
@ -161,4 +162,3 @@ REGISTER_FUNCTION(ArrayDifference)
}
}

View File

@ -22,7 +22,6 @@
#include <Common/assert_cast.h>
#include <base/TypeLists.h>
#include <Interpreters/castColumn.h>
#include <base/range.h>
namespace DB
@ -219,11 +218,12 @@ FunctionArrayIntersect::CastArgumentsResult FunctionArrayIntersect::castColumns(
const auto & type_nested = type_array->getNestedType();
auto type_not_nullable_nested = removeNullable(type_nested);
const bool is_numeric_or_string = isNativeNumber(type_not_nullable_nested)
|| isDate(type_not_nullable_nested)
|| isDateTime(type_not_nullable_nested)
|| isDateTime64(type_not_nullable_nested)
|| isStringOrFixedString(type_not_nullable_nested);
const bool is_numeric_or_string =
isNumber(type_not_nullable_nested)
|| isDate(type_not_nullable_nested)
|| isDateTime(type_not_nullable_nested)
|| isDateTime64(type_not_nullable_nested)
|| isStringOrFixedString(type_not_nullable_nested);
DataTypePtr nullable_return_type;
@ -303,7 +303,7 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(
bool all_const = true;
for (auto i : collections::range(0, columns_number))
for (size_t i = 0; i < columns_number; ++i)
{
auto & arg = arrays.args[i];
const auto * argument_column = columns[i].column.get();
@ -313,7 +313,7 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(
{
arg.is_const = true;
argument_column = argument_column_const->getDataColumnPtr().get();
initial_column = &typeid_cast<const ColumnConst &>(*initial_column).getDataColumn();
initial_column = typeid_cast<const ColumnConst &>(*initial_column).getDataColumnPtr().get();
}
if (const auto * argument_column_array = typeid_cast<const ColumnArray *>(argument_column))
@ -330,16 +330,21 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(
{
arg.null_map = &column_nullable->getNullMapData();
arg.nested_column = &column_nullable->getNestedColumn();
initial_column = &typeid_cast<const ColumnNullable &>(*initial_column).getNestedColumn();
if (initial_column->isNullable())
initial_column = &typeid_cast<const ColumnNullable &>(*initial_column).getNestedColumn();
}
/// In case column was casted need to create overflow mask for integer types.
/// In case the column was casted, we need to create an overflow mask for integer types.
if (arg.nested_column != initial_column)
{
const auto & nested_init_type = typeid_cast<const DataTypeArray *>(removeNullable(initial_columns[i].type).get())->getNestedType();
const auto & nested_cast_type = typeid_cast<const DataTypeArray *>(removeNullable(columns[i].type).get())->getNestedType();
const auto & nested_init_type = typeid_cast<const DataTypeArray &>(*removeNullable(initial_columns[i].type)).getNestedType();
const auto & nested_cast_type = typeid_cast<const DataTypeArray &>(*removeNullable(columns[i].type)).getNestedType();
if (isInteger(nested_init_type) || isDate(nested_init_type) || isDateTime(nested_init_type) || isDateTime64(nested_init_type))
if (isInteger(nested_init_type)
|| isDate(nested_init_type)
|| isDateTime(nested_init_type)
|| isDateTime64(nested_init_type))
{
/// Compare original and casted columns. It seem to be the easiest way.
auto overflow_mask = callFunctionNotEquals(
@ -347,7 +352,7 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(
{initial_column->getPtr(), nested_cast_type, ""},
context);
arg.overflow_mask = &typeid_cast<const ColumnUInt8 *>(overflow_mask.get())->getData();
arg.overflow_mask = &typeid_cast<const ColumnUInt8 &>(*overflow_mask).getData();
arrays.column_holders.emplace_back(std::move(overflow_mask));
}
}
@ -362,7 +367,7 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(
}
else
{
for (auto i : collections::range(0, columns_number))
for (size_t i = 0; i < columns_number; ++i)
{
if (arrays.args[i].is_const)
continue;
@ -371,7 +376,7 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(
if (arrays.base_rows == 0 && rows > 0)
arrays.base_rows = rows;
else if (arrays.base_rows != rows)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Non-const array columns in function {}should have same rows", getName());
throw Exception(ErrorCodes::LOGICAL_ERROR, "Non-const array columns in function {} should have the same number of rows", getName());
}
}
@ -397,7 +402,6 @@ ColumnPtr FunctionArrayIntersect::executeImpl(const ColumnsWithTypeAndName & arg
data_types.push_back(arguments[i].type);
auto return_type_with_nulls = getMostSubtype(data_types, true, true);
auto casted_columns = castColumns(arguments, result_type, return_type_with_nulls);
UnpackedArrays arrays = prepareArrays(casted_columns.casted, casted_columns.initial);
@ -501,13 +505,13 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable
Map map;
std::vector<size_t> prev_off(args, 0);
size_t result_offset = 0;
for (auto row : collections::range(0, rows))
for (size_t row = 0; row < rows; ++row)
{
map.clear();
bool all_has_nullable = all_nullable;
for (auto arg_num : collections::range(0, args))
for (size_t arg_num = 0; arg_num < args; ++arg_num)
{
const auto & arg = arrays.args[arg_num];
bool current_has_nullable = false;

View File

@ -10,7 +10,6 @@
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <base/arithmeticOverflow.h>
#include "Columns/ColumnMap.h"
#include "DataTypes/DataTypeMap.h"
@ -294,6 +293,10 @@ private:
return execute2<KeyType, UInt256>(row_count, args, res_type);
case TypeIndex::Float64:
return execute2<KeyType, Float64>(row_count, args, res_type);
case TypeIndex::Decimal128:
return execute2<KeyType, Decimal128>(row_count, args, res_type);
case TypeIndex::Decimal256:
return execute2<KeyType, Decimal256>(row_count, args, res_type);
default:
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column type {} for values in arguments of function {}",
res_value_type->getName(), getName());

View File

@ -149,6 +149,8 @@ Field convertDecimalType(const Field & from, const To & type)
return convertDecimalToDecimalType<Decimal64>(from, type);
if (from.getType() == Field::Types::Decimal128)
return convertDecimalToDecimalType<Decimal128>(from, type);
if (from.getType() == Field::Types::Decimal256)
return convertDecimalToDecimalType<Decimal256>(from, type);
if (from.getType() == Field::Types::Float64)
return convertFloatToDecimalType<Float64>(from, type);

View File

@ -5,7 +5,6 @@
#include <IO/WriteHelpers.h>
#include <Common/HashTable/Hash.h>
#include <Interpreters/InternalTextLogsQueue.h>
#include <Common/CurrentThread.h>
#include <base/terminalColors.h>

View File

@ -75,7 +75,7 @@ bool ExecutionThreadContext::executeTask()
if (trace_processors)
{
span = std::make_unique<OpenTelemetry::SpanHolder>("ExecutionThreadContext::executeTask() " + node->processor->getName());
span = std::make_unique<OpenTelemetry::SpanHolder>(node->processor->getName());
span->addAttribute("thread_number", thread_number);
}
std::optional<Stopwatch> execution_time_watch;

View File

@ -10,6 +10,8 @@
#include <Interpreters/ProcessList.h>
#include <Interpreters/Context.h>
#include <Common/scope_guard_safe.h>
#include <Common/Exception.h>
#include <Common/OpenTelemetryTraceContext.h>
#ifndef NDEBUG
#include <Common/Stopwatch.h>
@ -94,6 +96,9 @@ void PipelineExecutor::execute(size_t num_threads)
if (num_threads < 1)
num_threads = 1;
OpenTelemetry::SpanHolder span("PipelineExecutor::execute()");
span.addAttribute("clickhouse.thread_num", num_threads);
try
{
executeImpl(num_threads);
@ -108,6 +113,8 @@ void PipelineExecutor::execute(size_t num_threads)
}
catch (...)
{
span.addAttribute(ExecutionStatus::fromCurrentException());
#ifndef NDEBUG
LOG_TRACE(log, "Exception while executing query. Current state:\n{}", dumpPipeline());
#endif

View File

@ -1059,13 +1059,11 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
/** If exception is received from remote server, then stack trace is embedded in message.
* If exception is thrown on local server, then stack trace is in separate field.
*/
std::string exception_message = getCurrentExceptionMessage(with_stacktrace, true);
int exception_code = getCurrentExceptionCode();
trySendExceptionToClient(exception_message, exception_code, request, response, used_output);
ExecutionStatus status = ExecutionStatus::fromCurrentException("", with_stacktrace);
trySendExceptionToClient(status.message, status.code, request, response, used_output);
if (thread_trace_context)
thread_trace_context->root_span.addAttribute("clickhouse.exception_code", exception_code);
thread_trace_context->root_span.addAttribute(status);
}
used_output.finalize();

View File

@ -396,7 +396,7 @@ def arryToString(expected_clickhouse_values):
# if expected_clickhouse_values is "", compare MySQL and ClickHouse query results directly
@pytest.mark.parametrize(
"case_name, mysql_type, expected_ch_type, mysql_values, expected_clickhouse_values , setting_mysql_datatypes_support_level",
"case_name, mysql_type, expected_ch_type, mysql_values, expected_clickhouse_values, setting_mysql_datatypes_support_level",
[
pytest.param(
"common_types",
@ -725,11 +725,10 @@ def arryToString(expected_clickhouse_values):
"decimal,datetime64",
id="datetime_6_1",
),
# right now precision bigger than 39 is not supported by ClickHouse's Decimal, hence fall back to String
pytest.param(
"decimal_40_6",
"decimal(40, 6) NOT NULL",
"String",
"Decimal(40, 6)",
decimal_values,
"",
"decimal,datetime64",

View File

@ -67,9 +67,9 @@
[-50,-40,-30,-20,-10,0,10,20,30,40,50]
[-16.66666666,-13.33333333,-10,-6.66666666,-3.33333333,0,3.33333333,6.66666666,10,13.33333333,16.66666666]
[-10,-8,-6,-4,-2,0,2,4,6,8,10]
850 94.44444438684269 34 Float64 Float64 Float64
850 94.4444443868427 34.00000000000001 Float64 Float64 Float64
850 94.4444443868427 34.00000000000001
858.5 95.38888883071111 34.34 Float64 Float64 Float64
858.5 95.38888883071112 34.34 Float64 Float64 Float64
858.5 95.38888883071112 34.34
29.154759474226502 9.718253155111915 5.830951894845301 Float64 Float64 Float64
29.154759474226502 9.718253155111915 5.830951894845301

View File

@ -45,11 +45,11 @@
[0,0,0,0,0,0,0,0,0,0,0]
[0,0,0,0,0,0,0,0,0,0,0]
[0,0,0,0,0,0,0,0,0,0,0]
inf inf inf Float64 Float64 Float64
nan nan nan Float64 Float64 Float64
nan nan nan
nan nan nan Float64 Float64 Float64
nan nan nan
inf inf inf Float64 Float64 Float64
nan nan nan Float64 Float64 Float64
nan nan nan
nan nan nan Float64 Float64 Float64
nan nan nan

View File

@ -16,3 +16,9 @@
64 64
64 64
64 64
256 256
256 256
256 256
256 256
256 256
256 256

View File

@ -5,7 +5,7 @@ CREATE TABLE temp
y Nullable(Decimal(38, 2))
) ENGINE = Memory;
INSERT INTO temp VALUES (32, 32), (64, 64), (128, 128);
INSERT INTO temp VALUES (32, 32), (64, 64), (128, 128), (256, 256);
SELECT * FROM temp WHERE x IN (toDecimal128(128, 1));
SELECT * FROM temp WHERE x IN (toDecimal128(128, 2));
@ -28,7 +28,11 @@ SELECT * FROM temp WHERE y IN (toDecimal64(64, 1));
SELECT * FROM temp WHERE y IN (toDecimal64(64, 2));
SELECT * FROM temp WHERE y IN (toDecimal64(64, 3));
SELECT * FROM temp WHERE x IN (toDecimal256(256, 1)); -- { serverError 53 }
SELECT * FROM temp WHERE y IN (toDecimal256(256, 1)); -- { serverError 53 }
SELECT * FROM temp WHERE x IN (toDecimal256(256, 1));
SELECT * FROM temp WHERE x IN (toDecimal256(256, 2));
SELECT * FROM temp WHERE x IN (toDecimal256(256, 3));
SELECT * FROM temp WHERE y IN (toDecimal256(256, 1));
SELECT * FROM temp WHERE y IN (toDecimal256(256, 2));
SELECT * FROM temp WHERE y IN (toDecimal256(256, 3));
DROP TABLE IF EXISTS temp;

View File

@ -54,7 +54,7 @@ hello
2011-04-05 14:19:19
-123.45
-123.45
inf
inf
nan
nan
-123.45
-123.45

View File

@ -8,16 +8,16 @@ INSERT INTO dtest VALUES ('33', '44.4', '35');
SELECT count() == 0 FROM dtest WHERE a IN toDecimal32('33.3000', 4);
SELECT count() == 0 FROM dtest WHERE a IN toDecimal64('33.3000', 4);
SELECT count() == 0 FROM dtest WHERE a IN toDecimal128('33.3000', 4);
SELECT count() == 0 FROM dtest WHERE a IN toDecimal256('33.3000', 4); -- { serverError 53 }
SELECT count() == 0 FROM dtest WHERE a IN toDecimal256('33.3000', 4);
SELECT count() == 0 FROM dtest WHERE b IN toDecimal32('44.4000', 0);
SELECT count() == 0 FROM dtest WHERE b IN toDecimal64('44.4000', 0);
SELECT count() == 0 FROM dtest WHERE b IN toDecimal128('44.4000', 0);
SELECT count() == 0 FROM dtest WHERE b IN toDecimal256('44.4000', 0); -- { serverError 53 }
SELECT count() == 0 FROM dtest WHERE b IN toDecimal256('44.4000', 0);
SELECT count() == 1 FROM dtest WHERE b IN toDecimal32('44.4000', 4);
SELECT count() == 1 FROM dtest WHERE b IN toDecimal64('44.4000', 4);
SELECT count() == 1 FROM dtest WHERE b IN toDecimal128('44.4000', 4);
SELECT count() == 1 FROM dtest WHERE b IN toDecimal256('44.4000', 4); -- { serverError 53 }
SELECT count() == 1 FROM dtest WHERE b IN toDecimal256('44.4000', 4);
DROP TABLE IF EXISTS dtest;

View File

@ -0,0 +1,97 @@
-- { echoOn }
SELECT 1.1::Decimal(60, 30);
1.1
SELECT round(1.1::Decimal(60, 30));
1
SELECT round(1.1::Decimal(60, 30), 1);
1.1
SELECT round(1.234567890123456789012345678901::Decimal(60, 30), 1);
1.2
SELECT round(1.234567890123456789012345678901::Decimal(60, 30), 30);
1.234567890123456789012345678901
SELECT round(1.234567890123456789012345678901::Decimal(60, 30), 31);
1.234567890123456789012345678901
SELECT round(1.234567890123456789012345678901::Decimal(60, 30), 20);
1.23456789012345678901
SELECT hex(1.234567890123456789012345678901::Decimal(60, 30));
356C760E4FC986A2A39F1A950F00000000000000000000000000000000000000
SELECT bin(1.234567890123456789012345678901::Decimal(60, 30));
0011010101101100011101100000111001001111110010011000011010100010101000111001111100011010100101010000111100000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000
SELECT reinterpret(unhex(hex(1.234567890123456789012345678901::Decimal(60, 30))), 'Decimal(60, 30)');
1.234567890123456789012345678901
SELECT arraySum([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]);
4.65
SELECT arraySum([1.2::Decimal(60, 30), 3.45::Decimal(3, 2)]);
4.65
SELECT arrayMin([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]);
1.2
SELECT arrayMax([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]);
3.45
SELECT arrayAvg([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]);
2.325
SELECT round(arrayProduct([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]), 6);
4.14
SELECT toTypeName(arrayProduct([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]));
Float64
SELECT arrayCumSum([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]);
[1.2,4.65]
SELECT arrayCumSumNonNegative([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]);
[1.2,4.65]
SELECT arrayDifference([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]);
[0,2.25]
SELECT arrayCompact([1.2::Decimal(60, 30) AS x, x, x, x, 3.45::Decimal(3, 2) AS y, y, x, x]);
[1.2,3.45,1.2]
SELECT 1.2::Decimal(2, 1) IN (1.2::Decimal(60, 30), 3.4::Decimal(60, 30));
1
SELECT 1.23::Decimal(3, 2) IN (1.2::Decimal(60, 30), 3.4::Decimal(60, 30));
0
SELECT 1.2::Decimal(60, 30) IN (1.2::Decimal(2, 1));
1
SELECT toTypeName([1.2::Decimal(60, 30), 3.45::Decimal(3, 2)]);
Array(Decimal(76, 30))
SELECT toTypeName(arraySum([1.2::Decimal(60, 30), 3.45::Decimal(3, 2)]));
Decimal(76, 30)
SELECT arrayJoin(sumMap(x)) FROM (SELECT [('Hello', 1.2::Decimal256(30)), ('World', 3.4::Decimal256(30))]::Map(String, Decimal256(30)) AS x UNION ALL SELECT [('World', 5.6::Decimal256(30)), ('GoodBye', -111.222::Decimal256(30))]::Map(String, Decimal256(30))) ORDER BY 1;
('GoodBye',-111.222)
('Hello',1.2)
('World',9)
SELECT mapAdd(map('Hello', 1.2::Decimal128(30), 'World', 3.4::Decimal128(30)), map('World', 5.6::Decimal128(30), 'GoodBye', -111.222::Decimal128(30)));
{'GoodBye':-111.222,'Hello':1.2,'World':9}
SELECT mapSubtract(map('Hello', 1.2::Decimal128(30), 'World', 3.4::Decimal128(30)), map('World', 5.6::Decimal128(30), 'GoodBye', -111.222::Decimal128(30)));
{'GoodBye':111.222,'Hello':1.2,'World':-2.2}
SELECT arraySort(arrayIntersect([1, 2, 3]::Array(UInt256), [2, 3, 4]::Array(UInt256)));
[2,3]
SELECT toTypeName(arraySort(arrayIntersect([1, 2, 3]::Array(UInt256), [2, 3, 4]::Array(UInt128))));
Array(UInt128)
SELECT toTypeName(arraySort(arrayIntersect([1, 2, 3]::Array(UInt256), [2, 3, 4]::Array(Int128))));
Array(Int128)
SELECT arraySort(arrayIntersect([1, 2, 3]::Array(UInt256), [2, 3, 4]::Array(Int128)));
[2,3]
SELECT arraySort(arrayIntersect([1, 2, 3]::Array(UInt256), [2, 3, 4]::Array(Int8)));
[2,3]
SELECT toTypeName(arraySort(arrayIntersect([1, 2, 3]::Array(UInt256), [2, 3, 4]::Array(Int8))));
Array(Int8)
SELECT arraySort(arrayIntersect([1.1::Decimal256(70), 2.34::Decimal256(60), 3.456::Decimal256(50)], [2.34::Decimal256(65), 3.456::Decimal256(55), 4.5678::Decimal256(45)]));
[2.34,3.456]
SELECT arraySort(arrayIntersect([1.1::Decimal256(1)], [1.12::Decimal256(2)])); -- Note: this is correct but the semantics has to be clarified in the docs.
[1.1]
SELECT arraySort(arrayIntersect([1.1::Decimal256(2)], [1.12::Decimal256(2)]));
[]
SELECT arraySort(arrayIntersect([1.1::Decimal128(1)], [1.12::Decimal128(2)])); -- Note: this is correct but the semantics has to be clarified in the docs.
[1.1]
SELECT arraySort(arrayIntersect([1.1::Decimal128(2)], [1.12::Decimal128(2)]));
[]
select coalesce(cast('123', 'Nullable(Decimal(20, 10))'), 0);
123
select coalesce(cast('123', 'Nullable(Decimal(40, 10))'), 0);
123
select coalesce(cast('123', 'Decimal(40, 10)'), 0);
123
DROP TABLE IF EXISTS decimal_insert_cast_issue;
create table decimal_insert_cast_issue (a Decimal(76, 0)) engine = TinyLog;
SET param_param = 1;
INSERT INTO decimal_insert_cast_issue VALUES ({param:Nullable(Decimal(41, 0))});
SELECT * FROM decimal_insert_cast_issue;
1
DROP TABLE decimal_insert_cast_issue;

View File

@ -0,0 +1,65 @@
-- { echoOn }
SELECT 1.1::Decimal(60, 30);
SELECT round(1.1::Decimal(60, 30));
SELECT round(1.1::Decimal(60, 30), 1);
SELECT round(1.234567890123456789012345678901::Decimal(60, 30), 1);
SELECT round(1.234567890123456789012345678901::Decimal(60, 30), 30);
SELECT round(1.234567890123456789012345678901::Decimal(60, 30), 31);
SELECT round(1.234567890123456789012345678901::Decimal(60, 30), 20);
SELECT hex(1.234567890123456789012345678901::Decimal(60, 30));
SELECT bin(1.234567890123456789012345678901::Decimal(60, 30));
SELECT reinterpret(unhex(hex(1.234567890123456789012345678901::Decimal(60, 30))), 'Decimal(60, 30)');
SELECT arraySum([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]);
SELECT arraySum([1.2::Decimal(60, 30), 3.45::Decimal(3, 2)]);
SELECT arrayMin([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]);
SELECT arrayMax([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]);
SELECT arrayAvg([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]);
SELECT round(arrayProduct([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]), 6);
SELECT toTypeName(arrayProduct([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]));
SELECT arrayCumSum([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]);
SELECT arrayCumSumNonNegative([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]);
SELECT arrayDifference([1.2::Decimal(60, 30), 3.45::Decimal(61, 29)]);
SELECT arrayCompact([1.2::Decimal(60, 30) AS x, x, x, x, 3.45::Decimal(3, 2) AS y, y, x, x]);
SELECT 1.2::Decimal(2, 1) IN (1.2::Decimal(60, 30), 3.4::Decimal(60, 30));
SELECT 1.23::Decimal(3, 2) IN (1.2::Decimal(60, 30), 3.4::Decimal(60, 30));
SELECT 1.2::Decimal(60, 30) IN (1.2::Decimal(2, 1));
SELECT toTypeName([1.2::Decimal(60, 30), 3.45::Decimal(3, 2)]);
SELECT toTypeName(arraySum([1.2::Decimal(60, 30), 3.45::Decimal(3, 2)]));
SELECT arrayJoin(sumMap(x)) FROM (SELECT [('Hello', 1.2::Decimal256(30)), ('World', 3.4::Decimal256(30))]::Map(String, Decimal256(30)) AS x UNION ALL SELECT [('World', 5.6::Decimal256(30)), ('GoodBye', -111.222::Decimal256(30))]::Map(String, Decimal256(30))) ORDER BY 1;
SELECT mapAdd(map('Hello', 1.2::Decimal128(30), 'World', 3.4::Decimal128(30)), map('World', 5.6::Decimal128(30), 'GoodBye', -111.222::Decimal128(30)));
SELECT mapSubtract(map('Hello', 1.2::Decimal128(30), 'World', 3.4::Decimal128(30)), map('World', 5.6::Decimal128(30), 'GoodBye', -111.222::Decimal128(30)));
SELECT arraySort(arrayIntersect([1, 2, 3]::Array(UInt256), [2, 3, 4]::Array(UInt256)));
SELECT toTypeName(arraySort(arrayIntersect([1, 2, 3]::Array(UInt256), [2, 3, 4]::Array(UInt128))));
SELECT toTypeName(arraySort(arrayIntersect([1, 2, 3]::Array(UInt256), [2, 3, 4]::Array(Int128))));
SELECT arraySort(arrayIntersect([1, 2, 3]::Array(UInt256), [2, 3, 4]::Array(Int128)));
SELECT arraySort(arrayIntersect([1, 2, 3]::Array(UInt256), [2, 3, 4]::Array(Int8)));
SELECT toTypeName(arraySort(arrayIntersect([1, 2, 3]::Array(UInt256), [2, 3, 4]::Array(Int8))));
SELECT arraySort(arrayIntersect([1.1::Decimal256(70), 2.34::Decimal256(60), 3.456::Decimal256(50)], [2.34::Decimal256(65), 3.456::Decimal256(55), 4.5678::Decimal256(45)]));
SELECT arraySort(arrayIntersect([1.1::Decimal256(1)], [1.12::Decimal256(2)])); -- Note: this is correct but the semantics has to be clarified in the docs.
SELECT arraySort(arrayIntersect([1.1::Decimal256(2)], [1.12::Decimal256(2)]));
SELECT arraySort(arrayIntersect([1.1::Decimal128(1)], [1.12::Decimal128(2)])); -- Note: this is correct but the semantics has to be clarified in the docs.
SELECT arraySort(arrayIntersect([1.1::Decimal128(2)], [1.12::Decimal128(2)]));
select coalesce(cast('123', 'Nullable(Decimal(20, 10))'), 0);
select coalesce(cast('123', 'Nullable(Decimal(40, 10))'), 0);
select coalesce(cast('123', 'Decimal(40, 10)'), 0);
DROP TABLE IF EXISTS decimal_insert_cast_issue;
create table decimal_insert_cast_issue (a Decimal(76, 0)) engine = TinyLog;
SET param_param = 1;
INSERT INTO decimal_insert_cast_issue VALUES ({param:Nullable(Decimal(41, 0))});
SELECT * FROM decimal_insert_cast_issue;
DROP TABLE decimal_insert_cast_issue;

View File

@ -0,0 +1,5 @@
test
------
1.23
(1 row)

View File

@ -0,0 +1,14 @@
#!/usr/bin/env bash
# Tags: no-parallel, no-fasttest
# Tag no-fasttest: needs psql
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
echo "
DROP USER IF EXISTS postgresql_user;
CREATE USER postgresql_user HOST IP '127.0.0.1' IDENTIFIED WITH no_password;
" | $CLICKHOUSE_CLIENT -n
psql --host localhost --port ${CLICKHOUSE_PORT_POSTGRESQL} ${CLICKHOUSE_DATABASE} --user postgresql_user -c "SELECT 1.23::Decimal256(70) AS test;"