mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
gather utils for Decimals (no scale checks)
This commit is contained in:
parent
5548c3328d
commit
03933a1039
@ -144,7 +144,7 @@ public:
|
||||
}
|
||||
|
||||
|
||||
void insert(const T value) { data.push_back(value); }
|
||||
void insertValue(const T value) { data.push_back(value); }
|
||||
Container & getData() { return data; }
|
||||
const Container & getData() const { return data; }
|
||||
const T & getElement(size_t n) const { return data[n]; }
|
||||
|
@ -84,6 +84,23 @@ struct DefaultHash<T, std::enable_if_t<is_arithmetic_v<T>>>
|
||||
}
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct DefaultHash<T, std::enable_if_t<DB::IsDecimalNumber<T> && sizeof(T) <= 8>>
|
||||
{
|
||||
size_t operator() (T key) const
|
||||
{
|
||||
return DefaultHash64<typename T::NativeType>(key);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct DefaultHash<T, std::enable_if_t<DB::IsDecimalNumber<T> && sizeof(T) == 16>>
|
||||
{
|
||||
size_t operator() (T key) const
|
||||
{
|
||||
return DefaultHash64<Int64>(key >> 64) ^ DefaultHash64<Int64>(key);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename T> struct HashCRC32;
|
||||
|
||||
|
@ -5,6 +5,9 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using TypeListNumbers = TypeList<UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64>;
|
||||
using TypeListNativeNumbers = TypeList<UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64>;
|
||||
using TypeListDecimalNumbers = TypeList<Decimal32, Decimal64, Decimal128>;
|
||||
using TypeListNumbers = TypeList<UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64,
|
||||
Decimal32, Decimal64, Decimal128>;
|
||||
|
||||
}
|
||||
|
@ -894,7 +894,7 @@ MutableColumnUniquePtr DataTypeLowCardinality::createColumnUniqueImpl(const IDat
|
||||
if (isColumnedAsNumber(type))
|
||||
{
|
||||
MutableColumnUniquePtr column;
|
||||
TypeListNumbers::forEach(CreateColumnVector(column, *type, creator));
|
||||
TypeListNativeNumbers::forEach(CreateColumnVector(column, *type, creator));
|
||||
|
||||
if (!column)
|
||||
throw Exception("Unexpected numeric type: " + type->getName(), ErrorCodes::LOGICAL_ERROR);
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include "Sources.h"
|
||||
#include "Sinks.h"
|
||||
@ -79,8 +80,16 @@ inline ALWAYS_INLINE void writeSlice(const NumericArraySlice<T> & slice, Generic
|
||||
{
|
||||
for (size_t i = 0; i < slice.size; ++i)
|
||||
{
|
||||
Field field = T(slice.data[i]);
|
||||
sink.elements.insert(field);
|
||||
if constexpr (IsDecimalNumber<T>)
|
||||
{
|
||||
DecimalField field(T(slice.data[i]), 0); /// TODO: Decimal scale
|
||||
sink.elements.insert(field);
|
||||
}
|
||||
else
|
||||
{
|
||||
Field field = T(slice.data[i]);
|
||||
sink.elements.insert(field);
|
||||
}
|
||||
}
|
||||
sink.current_offset += slice.size;
|
||||
}
|
||||
@ -424,7 +433,13 @@ bool sliceHasImpl(const FirstSliceType & first, const SecondSliceType & second,
|
||||
template <typename T, typename U>
|
||||
bool sliceEqualElements(const NumericArraySlice<T> & first, const NumericArraySlice<U> & second, size_t first_ind, size_t second_ind)
|
||||
{
|
||||
return accurate::equalsOp(first.data[first_ind], second.data[second_ind]);
|
||||
/// TODO: Decimal scale
|
||||
if constexpr (IsDecimalNumber<T> && IsDecimalNumber<U>)
|
||||
return accurate::equalsOp(typename T::NativeType(first.data[first_ind]), typename U::NativeType(second.data[second_ind]));
|
||||
else if constexpr (IsDecimalNumber<T> || IsDecimalNumber<U>)
|
||||
return false;
|
||||
else
|
||||
return accurate::equalsOp(first.data[first_ind], second.data[second_ind]);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include "IArraySink.h"
|
||||
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
@ -33,17 +34,18 @@ struct NullableValueSource;
|
||||
template <typename T>
|
||||
struct NumericArraySink : public ArraySinkImpl<NumericArraySink<T>>
|
||||
{
|
||||
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
|
||||
using CompatibleArraySource = NumericArraySource<T>;
|
||||
using CompatibleValueSource = NumericValueSource<T>;
|
||||
|
||||
typename ColumnVector<T>::Container & elements;
|
||||
typename ColVecType::Container & elements;
|
||||
typename ColumnArray::Offsets & offsets;
|
||||
|
||||
size_t row_num = 0;
|
||||
ColumnArray::Offset current_offset = 0;
|
||||
|
||||
NumericArraySink(ColumnArray & arr, size_t column_size)
|
||||
: elements(typeid_cast<ColumnVector<T> &>(arr.getData()).getData()), offsets(arr.getOffsets())
|
||||
: elements(typeid_cast<ColVecType &>(arr.getData()).getData()), offsets(arr.getOffsets())
|
||||
{
|
||||
offsets.resize(column_size);
|
||||
}
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
@ -30,17 +31,18 @@ namespace GatherUtils
|
||||
template <typename T>
|
||||
struct NumericArraySource : public ArraySourceImpl<NumericArraySource<T>>
|
||||
{
|
||||
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
|
||||
using Slice = NumericArraySlice<T>;
|
||||
using Column = ColumnArray;
|
||||
|
||||
const typename ColumnVector<T>::Container & elements;
|
||||
const typename ColVecType::Container & elements;
|
||||
const typename ColumnArray::Offsets & offsets;
|
||||
|
||||
size_t row_num = 0;
|
||||
ColumnArray::Offset prev_offset = 0;
|
||||
|
||||
explicit NumericArraySource(const ColumnArray & arr)
|
||||
: elements(typeid_cast<const ColumnVector<T> &>(arr.getData()).getData()), offsets(arr.getOffsets())
|
||||
: elements(typeid_cast<const ColVecType &>(arr.getData()).getData()), offsets(arr.getOffsets())
|
||||
{
|
||||
}
|
||||
|
||||
@ -650,7 +652,7 @@ template <typename T>
|
||||
struct NumericValueSource : ValueSourceImpl<NumericValueSource<T>>
|
||||
{
|
||||
using Slice = NumericValueSlice<T>;
|
||||
using Column = ColumnVector<T>;
|
||||
using Column = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
|
||||
|
||||
const T * begin;
|
||||
size_t total_rows;
|
||||
|
@ -14,7 +14,9 @@ struct ArraySinkCreator<Type, Types...>
|
||||
{
|
||||
static std::unique_ptr<IArraySink> create(ColumnArray & col, NullMap * null_map, size_t column_size)
|
||||
{
|
||||
if (typeid_cast<ColumnVector<Type> *>(&col.getData()))
|
||||
using ColVecType = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
|
||||
|
||||
if (typeid_cast<ColVecType *>(&col.getData()))
|
||||
{
|
||||
if (null_map)
|
||||
return std::make_unique<NullableArraySink<NumericArraySink<Type>>>(col, *null_map, column_size);
|
||||
|
@ -14,7 +14,9 @@ struct ArraySourceCreator<Type, Types...>
|
||||
{
|
||||
static std::unique_ptr<IArraySource> create(const ColumnArray & col, const NullMap * null_map, bool is_const, size_t total_rows)
|
||||
{
|
||||
if (typeid_cast<const ColumnVector<Type> *>(&col.getData()))
|
||||
using ColVecType = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
|
||||
|
||||
if (typeid_cast<const ColVecType *>(&col.getData()))
|
||||
{
|
||||
if (null_map)
|
||||
{
|
||||
|
@ -14,7 +14,9 @@ struct ValueSourceCreator<Type, Types...>
|
||||
{
|
||||
static std::unique_ptr<IValueSource> create(const IColumn & col, const NullMap * null_map, bool is_const, size_t total_rows)
|
||||
{
|
||||
if (auto column_vector = typeid_cast<const ColumnVector<Type> *>(&col))
|
||||
using ColVecType = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
|
||||
|
||||
if (auto column_vector = typeid_cast<const ColVecType *>(&col))
|
||||
{
|
||||
if (null_map)
|
||||
{
|
||||
|
@ -590,7 +590,7 @@ struct CallPointInPolygon<Type, Types ...>
|
||||
template <typename PointInPolygonImpl>
|
||||
static ColumnPtr call(const IColumn & x, const IColumn & y, PointInPolygonImpl && impl)
|
||||
{
|
||||
using Impl = typename ApplyTypeListForClass<::DB::GeoUtils::CallPointInPolygon, TypeListNumbers>::Type;
|
||||
using Impl = typename ApplyTypeListForClass<::DB::GeoUtils::CallPointInPolygon, TypeListNativeNumbers>::Type;
|
||||
if (auto column = typeid_cast<const ColumnVector<Type> *>(&x))
|
||||
return Impl::template call<Type>(*column, y, impl);
|
||||
return CallPointInPolygon<Types ...>::call(x, y, impl);
|
||||
@ -616,7 +616,7 @@ struct CallPointInPolygon<>
|
||||
template <typename PointInPolygonImpl>
|
||||
ColumnPtr pointInPolygon(const IColumn & x, const IColumn & y, PointInPolygonImpl && impl)
|
||||
{
|
||||
using Impl = typename ApplyTypeListForClass<::DB::GeoUtils::CallPointInPolygon, TypeListNumbers>::Type;
|
||||
using Impl = typename ApplyTypeListForClass<::DB::GeoUtils::CallPointInPolygon, TypeListNativeNumbers>::Type;
|
||||
return Impl::call(x, y, impl);
|
||||
}
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
@ -12,6 +13,7 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/HashTable/ClearableHashMap.h>
|
||||
@ -88,6 +90,19 @@ private:
|
||||
template <typename T, size_t>
|
||||
void operator()();
|
||||
};
|
||||
|
||||
struct DecimalExecutor
|
||||
{
|
||||
const UnpackedArrays & arrays;
|
||||
const DataTypePtr & data_type;
|
||||
ColumnPtr & result;
|
||||
|
||||
DecimalExecutor(const UnpackedArrays & arrays_, const DataTypePtr & data_type_, ColumnPtr & result_)
|
||||
: arrays(arrays_), data_type(data_type_), result(result_) {}
|
||||
|
||||
template <typename T, size_t>
|
||||
void operator()();
|
||||
};
|
||||
};
|
||||
|
||||
|
||||
@ -328,7 +343,8 @@ void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & ar
|
||||
|
||||
ColumnPtr result_column;
|
||||
auto not_nullable_nested_return_type = removeNullable(nested_return_type);
|
||||
TypeListNumbers::forEach(NumberExecutor(arrays, not_nullable_nested_return_type, result_column));
|
||||
TypeListNativeNumbers::forEach(NumberExecutor(arrays, not_nullable_nested_return_type, result_column));
|
||||
TypeListDecimalNumbers::forEach(DecimalExecutor(arrays, not_nullable_nested_return_type, result_column));
|
||||
|
||||
using DateMap = ClearableHashMap<DataTypeDate::FieldType, size_t, DefaultHash<DataTypeDate::FieldType>,
|
||||
HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
@ -374,6 +390,17 @@ void FunctionArrayIntersect::NumberExecutor::operator()()
|
||||
result = execute<Map, ColumnVector<T>, true>(arrays, ColumnVector<T>::create());
|
||||
}
|
||||
|
||||
template <typename T, size_t>
|
||||
void FunctionArrayIntersect::DecimalExecutor::operator()()
|
||||
{
|
||||
using Map = ClearableHashMap<T, size_t, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
|
||||
|
||||
if (!result)
|
||||
if (auto * decimal = typeid_cast<const DataTypeDecimal<T> *>(data_type.get()))
|
||||
result = execute<Map, ColumnDecimal<T>, true>(arrays, ColumnDecimal<T>::create(0, decimal->getScale()));
|
||||
}
|
||||
|
||||
template <typename Map, typename ColumnType, bool is_numeric_column>
|
||||
ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, MutableColumnPtr result_data_ptr)
|
||||
{
|
||||
|
@ -175,9 +175,7 @@ public:
|
||||
|
||||
private:
|
||||
template <typename T0, typename T1>
|
||||
static constexpr bool allow_arrays =
|
||||
!IsDecimalNumber<T0> && !IsDecimalNumber<T1> &&
|
||||
!std::is_same_v<T0, UInt128> && !std::is_same_v<T1, UInt128>;
|
||||
static constexpr bool allow_arrays = !std::is_same_v<T0, UInt128> && !std::is_same_v<T1, UInt128>;
|
||||
|
||||
template <typename T0, typename T1>
|
||||
static UInt32 decimalScale(Block & block [[maybe_unused]], const ColumnNumbers & arguments [[maybe_unused]])
|
||||
|
@ -0,0 +1,13 @@
|
||||
[2.000]
|
||||
[2.0000000000]
|
||||
[2.000000000000000000]
|
||||
[1.000]
|
||||
[1.0000000000]
|
||||
[1.000000000000000000]
|
||||
-
|
||||
[2.000]
|
||||
[1]
|
||||
[2.000000000000000000]
|
||||
[1.000]
|
||||
[2]
|
||||
[1.000000000000000000]
|
17
dbms/tests/queries/0_stateless/00700_decimal_gathers.sql
Normal file
17
dbms/tests/queries/0_stateless/00700_decimal_gathers.sql
Normal file
@ -0,0 +1,17 @@
|
||||
select if(1, [cast(materialize(2.0),'Decimal(9,3)')], [cast(materialize(1.0),'Decimal(9,3)')]);
|
||||
select if(1, [cast(materialize(2.0),'Decimal(18,10)')], [cast(materialize(1.0),'Decimal(18,10)')]);
|
||||
select if(1, [cast(materialize(2.0),'Decimal(38,18)')], [cast(materialize(1.0),'Decimal(38,18)')]);
|
||||
|
||||
select if(0, [cast(materialize(2.0),'Decimal(9,3)')], [cast(materialize(1.0),'Decimal(9,3)')]);
|
||||
select if(0, [cast(materialize(2.0),'Decimal(18,10)')], [cast(materialize(1.0),'Decimal(18,10)')]);
|
||||
select if(0, [cast(materialize(2.0),'Decimal(38,18)')], [cast(materialize(1.0),'Decimal(38,18)')]);
|
||||
|
||||
select '-';
|
||||
|
||||
select if(1, [cast(materialize(2.0),'Decimal(9,3)')], [cast(materialize(1.0),'Decimal(9,0)')]);
|
||||
select if(0, [cast(materialize(2.0),'Decimal(18,10)')], [cast(materialize(1.0),'Decimal(18,0)')]);
|
||||
select if(1, [cast(materialize(2.0),'Decimal(38,18)')], [cast(materialize(1.0),'Decimal(38,8)')]);
|
||||
|
||||
select if(0, [cast(materialize(2.0),'Decimal(9,0)')], [cast(materialize(1.0),'Decimal(9,3)')]);
|
||||
select if(1, [cast(materialize(2.0),'Decimal(18,0)')], [cast(materialize(1.0),'Decimal(18,10)')]);
|
||||
select if(0, [cast(materialize(2.0),'Decimal(38,0)')], [cast(materialize(1.0),'Decimal(38,18)')]);
|
Loading…
Reference in New Issue
Block a user