Add support for function if with Array(UUID) arguments #11066

This commit is contained in:
Alexey Milovidov 2020-07-22 00:15:19 +03:00
parent e3594b89ea
commit 7b81640e9b
10 changed files with 25 additions and 17 deletions

View File

@ -124,9 +124,9 @@ public:
throw Exception("Cannot convert Tuple to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
}
T operator() (const UInt64 & x) const { return x; }
T operator() (const Int64 & x) const { return x; }
T operator() (const Float64 & x) const { return x; }
T operator() (const UInt64 & x) const { return T(x); }
T operator() (const Int64 & x) const { return T(x); }
T operator() (const Float64 & x) const { return T(x); }
T operator() (const UInt128 &) const
{
@ -139,7 +139,7 @@ public:
if constexpr (std::is_floating_point_v<T>)
return static_cast<T>(x.getValue()) / x.getScaleMultiplier();
else
return x.getValue() / x.getScaleMultiplier();
return static_cast<T>(x.getValue() / x.getScaleMultiplier());
}
T operator() (const AggregateFunctionStateData &) const

View File

@ -245,6 +245,13 @@ DEFINE_HASH(DB::Float64)
#undef DEFINE_HASH
template <>
struct DefaultHash<DB::UInt128> : public DB::UInt128Hash {};
template <>
struct DefaultHash<DB::UInt256> : public DB::UInt256Hash {};
/// It is reasonable to use for UInt8, UInt16 with sufficient hash table size.
struct TrivialHash
{

View File

@ -1,5 +1,7 @@
#pragma once
#include <Core/Types.h>
#include <Common/UInt128.h>
#include <Common/TypeList.h>
namespace DB
@ -7,7 +9,9 @@ namespace DB
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>;
using TypeListNumbers = typename TypeListConcat<TypeListNativeNumbers, TypeListDecimalNumbers>::Type;
/// Currently separate because UInt128 cannot be used in every context where other numbers can be used.
using TypeListNumbersAndUInt128 = typename AppendToTypeList<UInt128, TypeListNumbers>::Type;
}

View File

@ -13,7 +13,7 @@ struct GenericArraySink;
template <typename ArraySink>
struct NullableArraySink;
using NumericArraySinks = typename TypeListMap<NumericArraySink, TypeListNumbers>::Type;
using NumericArraySinks = typename TypeListMap<NumericArraySink, TypeListNumbersAndUInt128>::Type;
using BasicArraySinks = typename AppendToTypeList<GenericArraySink, NumericArraySinks>::Type;
using NullableArraySinks = typename TypeListMap<NullableArraySink, BasicArraySinks>::Type;
using TypeListArraySinks = typename TypeListConcat<BasicArraySinks, NullableArraySinks>::Type;

View File

@ -16,7 +16,7 @@ struct NullableArraySource;
template <typename Base>
struct ConstSource;
using NumericArraySources = typename TypeListMap<NumericArraySource, TypeListNumbers>::Type;
using NumericArraySources = typename TypeListMap<NumericArraySource, TypeListNumbersAndUInt128>::Type;
using BasicArraySources = typename AppendToTypeList<GenericArraySource, NumericArraySources>::Type;
using NullableArraySources = typename TypeListMap<NullableArraySource, BasicArraySources>::Type;
using BasicAndNullableArraySources = typename TypeListConcat<BasicArraySources, NullableArraySources>::Type;

View File

@ -16,7 +16,7 @@ struct NullableValueSource;
template <typename Base>
struct ConstSource;
using NumericValueSources = typename TypeListMap<NumericValueSource, TypeListNumbers>::Type;
using NumericValueSources = typename TypeListMap<NumericValueSource, TypeListNumbersAndUInt128>::Type;
using BasicValueSources = typename AppendToTypeList<GenericValueSource, NumericValueSources>::Type;
using NullableValueSources = typename TypeListMap<NullableValueSource, BasicValueSources>::Type;
using BasicAndNullableValueSources = typename TypeListConcat<BasicValueSources, NullableValueSources>::Type;

View File

@ -41,7 +41,7 @@ struct ArraySinkCreator<>
std::unique_ptr<IArraySink> createArraySink(ColumnArray & col, size_t column_size)
{
using Creator = ApplyTypeListForClass<ArraySinkCreator, TypeListNumbers>::Type;
using Creator = ApplyTypeListForClass<ArraySinkCreator, TypeListNumbersAndUInt128>::Type;
if (auto * column_nullable = typeid_cast<ColumnNullable *>(&col.getData()))
{
auto column = ColumnArray::create(column_nullable->getNestedColumnPtr()->assumeMutable(), col.getOffsetsPtr()->assumeMutable());

View File

@ -53,7 +53,7 @@ struct ArraySourceCreator<>
std::unique_ptr<IArraySource> createArraySource(const ColumnArray & col, bool is_const, size_t total_rows)
{
using Creator = typename ApplyTypeListForClass<ArraySourceCreator, TypeListNumbers>::Type;
using Creator = typename ApplyTypeListForClass<ArraySourceCreator, TypeListNumbersAndUInt128>::Type;
if (const auto * column_nullable = typeid_cast<const ColumnNullable *>(&col.getData()))
{
auto column = ColumnArray::create(column_nullable->getNestedColumnPtr(), col.getOffsetsPtr());

View File

@ -53,7 +53,7 @@ struct ValueSourceCreator<>
std::unique_ptr<IValueSource> createValueSource(const IColumn & col, bool is_const, size_t total_rows)
{
using Creator = typename ApplyTypeListForClass<ValueSourceCreator, TypeListNumbers>::Type;
using Creator = typename ApplyTypeListForClass<ValueSourceCreator, TypeListNumbersAndUInt128>::Type;
if (const auto * column_nullable = typeid_cast<const ColumnNullable *>(&col))
{
return Creator::create(column_nullable->getNestedColumn(), &column_nullable->getNullMapData(), is_const, total_rows);

View File

@ -175,9 +175,6 @@ public:
static FunctionPtr create(const Context &) { return std::make_shared<FunctionIf>(); }
private:
template <typename T0, typename T1>
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]])
{
@ -262,7 +259,7 @@ private:
{
if constexpr (std::is_same_v<NumberTraits::Error, typename NumberTraits::ResultOfIf<T0, T1>::Type>)
return false;
else if constexpr (allow_arrays<T0, T1>)
else
{
using ResultType = typename NumberTraits::ResultOfIf<T0, T1>::Type;
@ -318,7 +315,7 @@ private:
{
if constexpr (std::is_same_v<NumberTraits::Error, typename NumberTraits::ResultOfIf<T0, T1>::Type>)
return false;
else if constexpr (allow_arrays<T0, T1>)
else
{
using ResultType = typename NumberTraits::ResultOfIf<T0, T1>::Type;