gather utils for Decimals (no scale checks)

This commit is contained in:
chertus 2019-11-11 19:18:37 +03:00
parent 5548c3328d
commit 03933a1039
15 changed files with 120 additions and 20 deletions

View File

@ -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]; }

View File

@ -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;

View File

@ -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>;
}

View File

@ -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);

View File

@ -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>

View File

@ -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);
}

View File

@ -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;

View File

@ -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);

View File

@ -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)
{

View File

@ -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)
{

View File

@ -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);
}

View File

@ -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)
{

View File

@ -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]])

View File

@ -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]

View 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)')]);