mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Untangle UUID
This commit is contained in:
parent
2375b0503f
commit
02ce9cc725
@ -4,7 +4,8 @@
|
||||
#include <type_traits>
|
||||
#include <utility>
|
||||
|
||||
template <class T, class Tag>
|
||||
|
||||
template <typename T, typename Tag>
|
||||
struct StrongTypedef
|
||||
{
|
||||
private:
|
||||
@ -38,14 +39,17 @@ public:
|
||||
|
||||
bool operator==(const Self & rhs) const { return t == rhs.t; }
|
||||
bool operator<(const Self & rhs) const { return t < rhs.t; }
|
||||
bool operator>(const Self & rhs) const { return t > rhs.t; }
|
||||
|
||||
T & toUnderType() { return t; }
|
||||
const T & toUnderType() const { return t; }
|
||||
};
|
||||
|
||||
|
||||
|
||||
namespace std
|
||||
{
|
||||
template <class T, class Tag>
|
||||
template <typename T, typename Tag>
|
||||
struct hash<StrongTypedef<T, Tag>>
|
||||
{
|
||||
size_t operator()(const StrongTypedef<T, Tag> & x) const
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
|
@ -10,7 +10,6 @@
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/PODArray_fwd.h>
|
||||
#include <common/types.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
@ -21,7 +20,7 @@
|
||||
|
||||
#include <Common/ArenaAllocator.h>
|
||||
|
||||
#include <iostream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -2,9 +2,10 @@
|
||||
#include <AggregateFunctions/AggregateFunctionTopK.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
#define TOP_K_MAX_SIZE 0xFFFFFF
|
||||
|
||||
|
@ -10,7 +10,6 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
|
||||
#include <Common/SpaceSaving.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
@ -69,7 +69,7 @@ struct AggregateFunctionUniqHLL12Data<String>
|
||||
};
|
||||
|
||||
template <>
|
||||
struct AggregateFunctionUniqHLL12Data<UInt128>
|
||||
struct AggregateFunctionUniqHLL12Data<UUID>
|
||||
{
|
||||
using Set = HyperLogLogWithSmallSetOptimization<UInt64, 16, 12>;
|
||||
Set set;
|
||||
@ -133,16 +133,14 @@ template <typename T> struct AggregateFunctionUniqTraits
|
||||
{
|
||||
static UInt64 hash(T x)
|
||||
{
|
||||
if constexpr (std::is_same_v<T, UInt128>)
|
||||
{
|
||||
return sipHash64(x);
|
||||
}
|
||||
else if constexpr (std::is_same_v<T, Float32> || std::is_same_v<T, Float64>)
|
||||
if constexpr (std::is_same_v<T, Float32> || std::is_same_v<T, Float64>)
|
||||
{
|
||||
return ext::bit_cast<UInt64>(x);
|
||||
}
|
||||
else if constexpr (sizeof(T) <= sizeof(UInt64))
|
||||
{
|
||||
return x;
|
||||
}
|
||||
else
|
||||
return DefaultHash64<T>(x);
|
||||
}
|
||||
|
@ -3,11 +3,13 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
#include <functional>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/CombinedCardinalityEstimator.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
@ -1,11 +1,12 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include <AggregateFunctions/AggregateFunctionUniqUpTo.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include "registerAggregateFunctions.h"
|
||||
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
|
@ -292,7 +292,7 @@ MutableColumnPtr ColumnVector<T>::cloneResized(size_t size) const
|
||||
memcpy(new_col.data.data(), data.data(), count * sizeof(data[0]));
|
||||
|
||||
if (size > count)
|
||||
memset(static_cast<void *>(&new_col.data[count]), static_cast<int>(ValueType()), (size - count) * sizeof(ValueType));
|
||||
memset(static_cast<void *>(&new_col.data[count]), 0, (size - count) * sizeof(ValueType));
|
||||
}
|
||||
|
||||
return res;
|
||||
@ -301,19 +301,28 @@ MutableColumnPtr ColumnVector<T>::cloneResized(size_t size) const
|
||||
template <typename T>
|
||||
UInt64 ColumnVector<T>::get64(size_t n) const
|
||||
{
|
||||
return ext::bit_cast<UInt64>(data[n]);
|
||||
if constexpr (IsNumber<T>)
|
||||
return ext::bit_cast<UInt64>(data[n]);
|
||||
else
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get the value of {} as UInt64", TypeName<T>::get());
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
inline Float64 ColumnVector<T>::getFloat64(size_t n) const
|
||||
{
|
||||
return static_cast<Float64>(data[n]);
|
||||
if constexpr (IsNumber<T>)
|
||||
return static_cast<Float64>(data[n]);
|
||||
else
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get the value of {} as Float64", TypeName<T>::get());
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
Float32 ColumnVector<T>::getFloat32(size_t n) const
|
||||
{
|
||||
return static_cast<Float32>(data[n]);
|
||||
if constexpr (IsNumber<T>)
|
||||
return static_cast<Float32>(data[n]);
|
||||
else
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get the value of {} as Float32", TypeName<T>::get());
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
@ -567,5 +576,6 @@ template class ColumnVector<Int128>;
|
||||
template class ColumnVector<Int256>;
|
||||
template class ColumnVector<Float32>;
|
||||
template class ColumnVector<Float64>;
|
||||
template class ColumnVector<UUID>;
|
||||
|
||||
}
|
||||
|
@ -12,6 +12,12 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
|
||||
/** Stuff for comparing numbers.
|
||||
* Integer values are compared as usual.
|
||||
* Floating-point numbers are compared this way that NaNs always end up at the end
|
||||
@ -90,6 +96,7 @@ struct FloatCompareHelper
|
||||
template <class U> struct CompareHelper<Float32, U> : public FloatCompareHelper<Float32> {};
|
||||
template <class U> struct CompareHelper<Float64, U> : public FloatCompareHelper<Float64> {};
|
||||
|
||||
|
||||
/** A template for columns that use a simple array to store.
|
||||
*/
|
||||
template <typename T>
|
||||
@ -245,18 +252,27 @@ public:
|
||||
/// Out of range conversion is permitted.
|
||||
UInt64 NO_SANITIZE_UNDEFINED getUInt(size_t n) const override
|
||||
{
|
||||
return UInt64(data[n]);
|
||||
if constexpr (IsNumber<T>)
|
||||
return UInt64(data[n]);
|
||||
else
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get the value of {} as UInt", TypeName<T>::get());
|
||||
}
|
||||
|
||||
/// Out of range conversion is permitted.
|
||||
Int64 NO_SANITIZE_UNDEFINED getInt(size_t n) const override
|
||||
{
|
||||
return Int64(data[n]);
|
||||
if constexpr (IsNumber<T>)
|
||||
return Int64(data[n]);
|
||||
else
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get the value of {} as Int", TypeName<T>::get());
|
||||
}
|
||||
|
||||
bool getBool(size_t n) const override
|
||||
{
|
||||
return bool(data[n]);
|
||||
if constexpr (IsNumber<T>)
|
||||
return bool(data[n]);
|
||||
else
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get the value of {} as bool", TypeName<T>::get());
|
||||
}
|
||||
|
||||
void insert(const Field & x) override
|
||||
@ -370,5 +386,6 @@ extern template class ColumnVector<Int128>;
|
||||
extern template class ColumnVector<Int256>;
|
||||
extern template class ColumnVector<Float32>;
|
||||
extern template class ColumnVector<Float64>;
|
||||
extern template class ColumnVector<UUID>;
|
||||
|
||||
}
|
||||
|
@ -26,7 +26,7 @@ template <typename T>
|
||||
static inline String formatQuotedWithPrefix(T x, const char * prefix)
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
wb.write(prefix, strlen(prefix));
|
||||
writeCString(prefix, wb);
|
||||
writeQuoted(x, wb);
|
||||
return wb.str();
|
||||
}
|
||||
@ -51,6 +51,7 @@ String FieldVisitorDump::operator() (const UInt128 & x) const { return formatQuo
|
||||
String FieldVisitorDump::operator() (const UInt256 & x) const { return formatQuotedWithPrefix(x, "UInt256_"); }
|
||||
String FieldVisitorDump::operator() (const Int128 & x) const { return formatQuotedWithPrefix(x, "Int128_"); }
|
||||
String FieldVisitorDump::operator() (const Int256 & x) const { return formatQuotedWithPrefix(x, "Int256_"); }
|
||||
String FieldVisitorDump::operator() (const UUID & x) const { return formatQuotedWithPrefix(x, "UUID_"); }
|
||||
|
||||
|
||||
String FieldVisitorDump::operator() (const String & x) const
|
||||
@ -154,6 +155,7 @@ String FieldVisitorToString::operator() (const Int128 & x) const { return format
|
||||
String FieldVisitorToString::operator() (const UInt128 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const UInt256 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const Int256 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const UUID & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const AggregateFunctionStateData & x) const { return formatQuoted(x.data); }
|
||||
|
||||
String FieldVisitorToString::operator() (const Array & x) const
|
||||
@ -224,6 +226,7 @@ void FieldVisitorWriteBinary::operator() (const UInt128 & x, WriteBuffer & buf)
|
||||
void FieldVisitorWriteBinary::operator() (const Int128 & x, WriteBuffer & buf) const { DB::writeVarInt(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const UInt256 & x, WriteBuffer & buf) const { DB::writeBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const Int256 & x, WriteBuffer & buf) const { DB::writeBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const UUID & x, WriteBuffer & buf) const { DB::writeBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const DecimalField<Decimal32> & x, WriteBuffer & buf) const { DB::writeBinary(x.getValue(), buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const DecimalField<Decimal64> & x, WriteBuffer & buf) const { DB::writeBinary(x.getValue(), buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const DecimalField<Decimal128> & x, WriteBuffer & buf) const { DB::writeBinary(x.getValue(), buf); }
|
||||
@ -311,6 +314,13 @@ void FieldVisitorHash::operator() (const Int128 & x) const
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const UUID & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::UUID;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const Float64 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Float64;
|
||||
|
@ -21,7 +21,7 @@ namespace ErrorCodes
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wredundant-decls"
|
||||
// Just dont mess with it. If the redundant redeclaration is removed then ReaderHelpers.h should be included.
|
||||
// Just dont mess with it. If the redundant redeclaration is removed then ReadHelpers.h should be included.
|
||||
// This leads to Arena.h inclusion which has a problem with ASAN stuff included properly and messing macro definition
|
||||
// which intefrers with... You dont want to know, really.
|
||||
UInt128 stringToUUID(const String & str);
|
||||
@ -75,6 +75,7 @@ public:
|
||||
String operator() (const Int64 & x) const;
|
||||
String operator() (const Int128 & x) const;
|
||||
String operator() (const Int256 & x) const;
|
||||
String operator() (const UUID & x) const;
|
||||
String operator() (const Float64 & x) const;
|
||||
String operator() (const String & x) const;
|
||||
String operator() (const Array & x) const;
|
||||
@ -98,6 +99,7 @@ public:
|
||||
void operator() (const Int64 & x, WriteBuffer & buf) const;
|
||||
void operator() (const Int128 & x, WriteBuffer & buf) const;
|
||||
void operator() (const Int256 & x, WriteBuffer & buf) const;
|
||||
void operator() (const UUID & x, WriteBuffer & buf) const;
|
||||
void operator() (const Float64 & x, WriteBuffer & buf) const;
|
||||
void operator() (const String & x, WriteBuffer & buf) const;
|
||||
void operator() (const Array & x, WriteBuffer & buf) const;
|
||||
@ -122,6 +124,7 @@ public:
|
||||
String operator() (const Int64 & x) const;
|
||||
String operator() (const Int128 & x) const;
|
||||
String operator() (const Int256 & x) const;
|
||||
String operator() (const UUID & x) const;
|
||||
String operator() (const Float64 & x) const;
|
||||
String operator() (const String & x) const;
|
||||
String operator() (const Array & x) const;
|
||||
@ -168,6 +171,7 @@ public:
|
||||
T operator() (const UInt64 & x) const { return T(x); }
|
||||
T operator() (const Int64 & x) const { return T(x); }
|
||||
T operator() (const Int128 & x) const { return T(x); }
|
||||
T operator() (const UUID & x) const { return T(x.toUnderType()); }
|
||||
|
||||
T operator() (const Float64 & x) const
|
||||
{
|
||||
@ -260,6 +264,7 @@ public:
|
||||
void operator() (const Int64 & x) const;
|
||||
void operator() (const Int128 & x) const;
|
||||
void operator() (const Int256 & x) const;
|
||||
void operator() (const UUID & x) const;
|
||||
void operator() (const Float64 & x) const;
|
||||
void operator() (const String & x) const;
|
||||
void operator() (const Array & x) const;
|
||||
@ -305,7 +310,7 @@ public:
|
||||
bool operator() (Array &) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (Tuple &) const { throw Exception("Cannot sum Tuples", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (Map &) const { throw Exception("Cannot sum Maps", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (UInt128 &) const { throw Exception("Cannot sum UUIDs", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (UUID &) const { throw Exception("Cannot sum UUIDs", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (AggregateFunctionStateData &) const { throw Exception("Cannot sum AggregateFunctionStates", ErrorCodes::LOGICAL_ERROR); }
|
||||
|
||||
bool operator() (Int128 & x) const
|
||||
|
@ -47,16 +47,10 @@ public:
|
||||
|
||||
if constexpr (std::is_same_v<T, String> && std::is_arithmetic_v<U>)
|
||||
{
|
||||
if constexpr (std::is_same_v<U, UInt128>)
|
||||
return stringToUUID(l) == r;
|
||||
|
||||
if constexpr (std::is_arithmetic_v<U>)
|
||||
{
|
||||
ReadBufferFromString in(l);
|
||||
U parsed;
|
||||
readText(parsed, in);
|
||||
return operator()(parsed, r);
|
||||
}
|
||||
ReadBufferFromString in(l);
|
||||
U parsed;
|
||||
readText(parsed, in);
|
||||
return operator()(parsed, r);
|
||||
}
|
||||
|
||||
if constexpr (std::is_same_v<U, String> && std::is_arithmetic_v<T>)
|
||||
@ -101,16 +95,10 @@ public:
|
||||
|
||||
if constexpr (std::is_same_v<T, String> && std::is_arithmetic_v<U>)
|
||||
{
|
||||
if constexpr (std::is_same_v<U, UInt128>)
|
||||
return stringToUUID(l) < r;
|
||||
|
||||
if constexpr (std::is_arithmetic_v<U>)
|
||||
{
|
||||
ReadBufferFromString in(l);
|
||||
U parsed;
|
||||
readText(parsed, in);
|
||||
return operator()(parsed, r);
|
||||
}
|
||||
ReadBufferFromString in(l);
|
||||
U parsed;
|
||||
readText(parsed, in);
|
||||
return operator()(parsed, r);
|
||||
}
|
||||
|
||||
if constexpr (std::is_same_v<U, String> && std::is_arithmetic_v<T>)
|
||||
|
@ -179,6 +179,12 @@ inline size_t DefaultHash64(std::enable_if_t<(sizeof(T) > sizeof(UInt64)), T> ke
|
||||
static_cast<UInt64>(key) ^
|
||||
static_cast<UInt64>(key >> 64));
|
||||
}
|
||||
else if constexpr (std::is_same_v<T, DB::UUID>)
|
||||
{
|
||||
return intHash64(
|
||||
static_cast<UInt64>(key.toUnderType()) ^
|
||||
static_cast<UInt64>(key.toUnderType() >> 64));
|
||||
}
|
||||
else if constexpr (is_big_int_v<T> && sizeof(T) == 32)
|
||||
{
|
||||
return intHash64(
|
||||
@ -187,6 +193,7 @@ inline size_t DefaultHash64(std::enable_if_t<(sizeof(T) > sizeof(UInt64)), T> ke
|
||||
static_cast<UInt64>(key >> 128) ^
|
||||
static_cast<UInt64>(key >> 256));
|
||||
}
|
||||
assert(false);
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
@ -256,6 +263,7 @@ DEFINE_HASH(DB::Int128)
|
||||
DEFINE_HASH(DB::Int256)
|
||||
DEFINE_HASH(DB::Float32)
|
||||
DEFINE_HASH(DB::Float64)
|
||||
DEFINE_HASH(DB::UUID)
|
||||
|
||||
#undef DEFINE_HASH
|
||||
|
||||
@ -268,6 +276,14 @@ struct UInt128Hash
|
||||
}
|
||||
};
|
||||
|
||||
struct UUIDHash
|
||||
{
|
||||
size_t operator()(DB::UUID x) const
|
||||
{
|
||||
return UInt128Hash()(x.toUnderType());
|
||||
}
|
||||
};
|
||||
|
||||
#ifdef __SSE4_2__
|
||||
|
||||
struct UInt128HashCRC32
|
||||
@ -293,6 +309,11 @@ struct UInt128TrivialHash
|
||||
size_t operator()(UInt128 x) const { return x.items[0]; }
|
||||
};
|
||||
|
||||
struct UUIDTrivialHash
|
||||
{
|
||||
size_t operator()(DB::UUID x) const { return x.toUnderType().items[0]; }
|
||||
};
|
||||
|
||||
struct UInt256Hash
|
||||
{
|
||||
size_t operator()(UInt256 x) const
|
||||
@ -332,6 +353,9 @@ struct DefaultHash<DB::UInt128> : public UInt128Hash {};
|
||||
template <>
|
||||
struct DefaultHash<DB::UInt256> : public UInt256Hash {};
|
||||
|
||||
template <>
|
||||
struct DefaultHash<DB::UUID> : public UUIDHash {};
|
||||
|
||||
|
||||
/// It is reasonable to use for UInt8, UInt16 with sufficient hash table size.
|
||||
struct TrivialHash
|
||||
@ -383,23 +407,18 @@ struct IntHash32
|
||||
{
|
||||
size_t operator() (const T & key) const
|
||||
{
|
||||
if constexpr (std::is_same_v<T, DB::Int128>)
|
||||
if constexpr (is_big_int_v<T> && sizeof(T) == 16)
|
||||
{
|
||||
return intHash32<salt>(static_cast<UInt64>(key) ^ static_cast<UInt64>(key >> 64));
|
||||
}
|
||||
else if constexpr (std::is_same_v<T, DB::UInt128>)
|
||||
{
|
||||
return intHash32<salt>(key.low ^ key.high);
|
||||
return intHash32<salt>(key.items[0] ^ key.items[1]);
|
||||
}
|
||||
else if constexpr (is_big_int_v<T> && sizeof(T) == 32)
|
||||
{
|
||||
return intHash32<salt>(static_cast<UInt64>(key) ^
|
||||
static_cast<UInt64>(key >> 64) ^
|
||||
static_cast<UInt64>(key >> 128) ^
|
||||
static_cast<UInt64>(key >> 256));
|
||||
return intHash32<salt>(key.items[0] ^ key.items[1] ^ key.items[2] ^ key.items[3]);
|
||||
}
|
||||
else if constexpr (sizeof(T) <= sizeof(UInt64))
|
||||
{
|
||||
return intHash32<salt>(key);
|
||||
}
|
||||
|
||||
assert(false);
|
||||
__builtin_unreachable();
|
||||
|
@ -70,10 +70,10 @@ namespace ZeroTraits
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
bool check(const T x) { return x == 0; }
|
||||
bool check(const T x) { return x == T{}; }
|
||||
|
||||
template <typename T>
|
||||
void set(T & x) { x = 0; }
|
||||
void set(T & x) { x = {}; }
|
||||
|
||||
}
|
||||
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <Core/SettingsFields.h>
|
||||
#include <Common/SettingsChanges.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <ext/range.h>
|
||||
#include <boost/blank.hpp>
|
||||
#include <unordered_map>
|
||||
|
@ -176,7 +176,7 @@ template <> struct NearestFieldTypeImpl<UInt16> { using Type = UInt64; };
|
||||
template <> struct NearestFieldTypeImpl<UInt32> { using Type = UInt64; };
|
||||
|
||||
template <> struct NearestFieldTypeImpl<DayNum> { using Type = UInt64; };
|
||||
template <> struct NearestFieldTypeImpl<UUID> { using Type = UInt128; };
|
||||
template <> struct NearestFieldTypeImpl<UUID> { using Type = UUID; };
|
||||
template <> struct NearestFieldTypeImpl<Int16> { using Type = Int64; };
|
||||
template <> struct NearestFieldTypeImpl<Int32> { using Type = Int64; };
|
||||
|
||||
@ -234,7 +234,7 @@ struct NearestFieldTypeImpl<T, std::enable_if_t<std::is_enum_v<T>>>
|
||||
* but somewhat more efficient, and simpler.
|
||||
*
|
||||
* Used to represent a single value of one of several types in memory.
|
||||
* Warning! Prefer to use chunks of columns instead of single values. See Column.h
|
||||
* Warning! Prefer to use chunks of columns instead of single values. See IColumn.h
|
||||
*/
|
||||
class Field
|
||||
{
|
||||
@ -251,8 +251,6 @@ public:
|
||||
UInt128 = 4,
|
||||
Int128 = 5,
|
||||
|
||||
/// Non-POD types.
|
||||
|
||||
String = 16,
|
||||
Array = 17,
|
||||
Tuple = 18,
|
||||
@ -264,10 +262,9 @@ public:
|
||||
UInt256 = 24,
|
||||
Int256 = 25,
|
||||
Map = 26,
|
||||
UUID = 27,
|
||||
};
|
||||
|
||||
static const int MIN_NON_POD = 16;
|
||||
|
||||
static const char * toString(Which which)
|
||||
{
|
||||
switch (which)
|
||||
@ -275,8 +272,11 @@ public:
|
||||
case Null: return "Null";
|
||||
case UInt64: return "UInt64";
|
||||
case UInt128: return "UInt128";
|
||||
case UInt256: return "UInt256";
|
||||
case Int64: return "Int64";
|
||||
case Int128: return "Int128";
|
||||
case Int256: return "Int256";
|
||||
case UUID: return "UUID";
|
||||
case Float64: return "Float64";
|
||||
case String: return "String";
|
||||
case Array: return "Array";
|
||||
@ -287,8 +287,6 @@ public:
|
||||
case Decimal128: return "Decimal128";
|
||||
case Decimal256: return "Decimal256";
|
||||
case AggregateFunctionState: return "AggregateFunctionState";
|
||||
case UInt256: return "UInt256";
|
||||
case Int256: return "Int256";
|
||||
}
|
||||
|
||||
throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD);
|
||||
@ -302,12 +300,17 @@ public:
|
||||
|
||||
static bool IsDecimal(Types::Which which)
|
||||
{
|
||||
return (which >= Types::Decimal32 && which <= Types::Decimal128) || which == Types::Decimal256;
|
||||
return which == Types::Decimal32
|
||||
|| which == Types::Decimal64
|
||||
|| which == Types::Decimal128
|
||||
|| which == Types::Decimal256;
|
||||
}
|
||||
|
||||
/// Templates to avoid ambiguity.
|
||||
template <typename T, typename Z = void *>
|
||||
using enable_if_not_field_or_stringlike_t = std::enable_if_t<!std::is_same_v<std::decay_t<T>, Field> && !std::is_same_v<NearestFieldType<std::decay_t<T>>, String>, Z>;
|
||||
using enable_if_not_field_or_stringlike_t = std::enable_if_t<
|
||||
!std::is_same_v<std::decay_t<T>, Field>
|
||||
&& !std::is_same_v<NearestFieldType<std::decay_t<T>>, String>, Z>;
|
||||
|
||||
Field() //-V730
|
||||
: which(Types::Null)
|
||||
@ -381,10 +384,10 @@ public:
|
||||
enable_if_not_field_or_stringlike_t<T, Field> &
|
||||
operator=(T && rhs);
|
||||
|
||||
Field & operator =(const std::string_view & str);
|
||||
Field & operator =(const String & str) { return *this = std::string_view{str}; }
|
||||
Field & operator =(String && str);
|
||||
Field & operator =(const char * str) { return *this = std::string_view{str}; }
|
||||
Field & operator= (const std::string_view & str);
|
||||
Field & operator= (const String & str) { return *this = std::string_view{str}; }
|
||||
Field & operator= (String && str);
|
||||
Field & operator= (const char * str) { return *this = std::string_view{str}; }
|
||||
|
||||
~Field()
|
||||
{
|
||||
@ -456,7 +459,8 @@ public:
|
||||
case Types::UInt256: return get<UInt256>() < rhs.get<UInt256>();
|
||||
case Types::Int64: return get<Int64>() < rhs.get<Int64>();
|
||||
case Types::Int128: return get<Int128>() < rhs.get<Int128>();
|
||||
case Types::Int256: return get<Int256>() < rhs.get<Int256>();
|
||||
case Types::Int256: return get<Int256>() < rhs.get<Int256>();
|
||||
case Types::UUID: return get<UUID>() < rhs.get<UUID>();
|
||||
case Types::Float64: return get<Float64>() < rhs.get<Float64>();
|
||||
case Types::String: return get<String>() < rhs.get<String>();
|
||||
case Types::Array: return get<Array>() < rhs.get<Array>();
|
||||
@ -492,7 +496,8 @@ public:
|
||||
case Types::UInt256: return get<UInt256>() <= rhs.get<UInt256>();
|
||||
case Types::Int64: return get<Int64>() <= rhs.get<Int64>();
|
||||
case Types::Int128: return get<Int128>() <= rhs.get<Int128>();
|
||||
case Types::Int256: return get<Int256>() <= rhs.get<Int256>();
|
||||
case Types::Int256: return get<Int256>() <= rhs.get<Int256>();
|
||||
case Types::UUID: return get<UUID>().toUnderType() <= rhs.get<UUID>().toUnderType();
|
||||
case Types::Float64: return get<Float64>() <= rhs.get<Float64>();
|
||||
case Types::String: return get<String>() <= rhs.get<String>();
|
||||
case Types::Array: return get<Array>() <= rhs.get<Array>();
|
||||
@ -530,6 +535,7 @@ public:
|
||||
// Compare as UInt64 so that NaNs compare as equal.
|
||||
return reinterpret<UInt64>() == rhs.reinterpret<UInt64>();
|
||||
}
|
||||
case Types::UUID: return get<UUID>() == rhs.get<UUID>();
|
||||
case Types::String: return get<String>() == rhs.get<String>();
|
||||
case Types::Array: return get<Array>() == rhs.get<Array>();
|
||||
case Types::Tuple: return get<Tuple>() == rhs.get<Tuple>();
|
||||
@ -570,8 +576,9 @@ public:
|
||||
case Types::UInt128: return f(field.template get<UInt128>());
|
||||
case Types::UInt256: return f(field.template get<UInt256>());
|
||||
case Types::Int64: return f(field.template get<Int64>());
|
||||
case Types::Int128: return f(field.template get<Int128>());
|
||||
case Types::Int256: return f(field.template get<Int256>());
|
||||
case Types::Int128: return f(field.template get<Int128>());
|
||||
case Types::Int256: return f(field.template get<Int256>());
|
||||
case Types::UUID: return f(field.template get<UUID>());
|
||||
case Types::Float64: return f(field.template get<Float64>());
|
||||
case Types::String: return f(field.template get<String>());
|
||||
case Types::Array: return f(field.template get<Array>());
|
||||
@ -587,11 +594,7 @@ public:
|
||||
#endif
|
||||
}
|
||||
|
||||
// GCC 9 complains that control reaches the end, despite that we handle
|
||||
// all the cases above (maybe because of throw?). Return something to
|
||||
// silence it.
|
||||
Null null{};
|
||||
return f(null);
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
String dump() const;
|
||||
@ -599,7 +602,7 @@ public:
|
||||
|
||||
private:
|
||||
std::aligned_union_t<DBMS_MIN_FIELD_SIZE - sizeof(Types::Which),
|
||||
Null, UInt64, UInt128, UInt256, Int64, Int128, Int256, Float64, String, Array, Tuple, Map,
|
||||
Null, UInt64, UInt128, UInt256, Int64, Int128, Int256, UUID, Float64, String, Array, Tuple, Map,
|
||||
DecimalField<Decimal32>, DecimalField<Decimal64>, DecimalField<Decimal128>, DecimalField<Decimal256>,
|
||||
AggregateFunctionStateData
|
||||
> storage;
|
||||
@ -683,9 +686,6 @@ private:
|
||||
|
||||
ALWAYS_INLINE void destroy()
|
||||
{
|
||||
if (which < Types::MIN_NON_POD)
|
||||
return;
|
||||
|
||||
switch (which)
|
||||
{
|
||||
case Types::String:
|
||||
@ -727,7 +727,8 @@ template <> struct Field::TypeToEnum<UInt128> { static const Types::Which value
|
||||
template <> struct Field::TypeToEnum<UInt256> { static const Types::Which value = Types::UInt256; };
|
||||
template <> struct Field::TypeToEnum<Int64> { static const Types::Which value = Types::Int64; };
|
||||
template <> struct Field::TypeToEnum<Int128> { static const Types::Which value = Types::Int128; };
|
||||
template <> struct Field::TypeToEnum<Int256> { static const Types::Which value = Types::Int256; };
|
||||
template <> struct Field::TypeToEnum<Int256> { static const Types::Which value = Types::Int256; };
|
||||
template <> struct Field::TypeToEnum<UUID> { static const Types::Which value = Types::UUID; };
|
||||
template <> struct Field::TypeToEnum<Float64> { static const Types::Which value = Types::Float64; };
|
||||
template <> struct Field::TypeToEnum<String> { static const Types::Which value = Types::String; };
|
||||
template <> struct Field::TypeToEnum<Array> { static const Types::Which value = Types::Array; };
|
||||
@ -747,6 +748,7 @@ template <> struct Field::EnumToType<Field::Types::UInt256> { using Type = UInt2
|
||||
template <> struct Field::EnumToType<Field::Types::Int64> { using Type = Int64; };
|
||||
template <> struct Field::EnumToType<Field::Types::Int128> { using Type = Int128; };
|
||||
template <> struct Field::EnumToType<Field::Types::Int256> { using Type = Int256; };
|
||||
template <> struct Field::EnumToType<Field::Types::UUID> { using Type = UUID; };
|
||||
template <> struct Field::EnumToType<Field::Types::Float64> { using Type = Float64; };
|
||||
template <> struct Field::EnumToType<Field::Types::String> { using Type = String; };
|
||||
template <> struct Field::EnumToType<Field::Types::Array> { using Type = Array; };
|
||||
@ -758,7 +760,7 @@ template <> struct Field::EnumToType<Field::Types::Decimal128> { using Type = De
|
||||
template <> struct Field::EnumToType<Field::Types::Decimal256> { using Type = DecimalField<Decimal256>; };
|
||||
template <> struct Field::EnumToType<Field::Types::AggregateFunctionState> { using Type = DecimalField<AggregateFunctionStateData>; };
|
||||
|
||||
inline constexpr bool isInt64FieldType(Field::Types::Which t)
|
||||
inline constexpr bool isInt64OrUInt64FieldType(Field::Types::Which t)
|
||||
{
|
||||
return t == Field::Types::Int64
|
||||
|| t == Field::Types::UInt64;
|
||||
@ -780,7 +782,7 @@ NearestFieldType<std::decay_t<T>> & Field::get()
|
||||
// Disregard signedness when converting between int64 types.
|
||||
constexpr Field::Types::Which target = TypeToEnum<StoredType>::value;
|
||||
if (target != which
|
||||
&& (!isInt64FieldType(target) || !isInt64FieldType(which)))
|
||||
&& (!isInt64OrUInt64FieldType(target) || !isInt64OrUInt64FieldType(which)))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid Field get from type {} to type {}", Types::toString(which), Types::toString(target));
|
||||
#endif
|
||||
|
||||
@ -844,8 +846,6 @@ decltype(auto) castToNearestFieldType(T && x)
|
||||
using U = NearestFieldType<std::decay_t<T>>;
|
||||
if constexpr (std::is_same_v<std::decay_t<T>, U>)
|
||||
return std::forward<T>(x);
|
||||
else if constexpr (std::is_same_v<std::decay_t<T>, UUID>)
|
||||
return U(x.toUnderType());
|
||||
else
|
||||
return U(x);
|
||||
}
|
||||
@ -965,7 +965,7 @@ struct fmt::formatter<DB::Field>
|
||||
|
||||
/// Only support {}.
|
||||
if (it != end && *it != '}')
|
||||
throw format_error("invalid format");
|
||||
throw format_error("Invalid format");
|
||||
|
||||
return it;
|
||||
}
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <cstdint>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
#include <common/strong_typedef.h>
|
||||
#include <common/extended_types.h>
|
||||
#include <common/defines.h>
|
||||
|
||||
@ -69,6 +70,8 @@ using UInt256 = ::UInt256;
|
||||
using Int128 = ::Int128;
|
||||
using Int256 = ::Int256;
|
||||
|
||||
STRONG_TYPEDEF(UInt128, UUID)
|
||||
|
||||
|
||||
/** Note that for types not used in DB, IsNumber is false.
|
||||
*/
|
||||
@ -102,26 +105,28 @@ template <> struct TypeName<Int16> { static constexpr const char * get() { ret
|
||||
template <> struct TypeName<Int32> { static constexpr const char * get() { return "Int32"; } };
|
||||
template <> struct TypeName<Int64> { static constexpr const char * get() { return "Int64"; } };
|
||||
template <> struct TypeName<Int128> { static constexpr const char * get() { return "Int128"; } };
|
||||
template <> struct TypeName<Int256> { static constexpr const char * get() { return "Int256"; } };
|
||||
template <> struct TypeName<Int256> { static constexpr const char * get() { return "Int256"; } };
|
||||
template <> struct TypeName<Float32> { static constexpr const char * get() { return "Float32"; } };
|
||||
template <> struct TypeName<Float64> { static constexpr const char * get() { return "Float64"; } };
|
||||
template <> struct TypeName<String> { static constexpr const char * get() { return "String"; } };
|
||||
template <> struct TypeName<UUID> { static constexpr const char * get() { return "UUID"; } };
|
||||
|
||||
template <typename T> struct TypeId;
|
||||
template <> struct TypeId<UInt8> { static constexpr const TypeIndex value = TypeIndex::UInt8; };
|
||||
template <> struct TypeId<UInt16> { static constexpr const TypeIndex value = TypeIndex::UInt16; };
|
||||
template <> struct TypeId<UInt32> { static constexpr const TypeIndex value = TypeIndex::UInt32; };
|
||||
template <> struct TypeId<UInt64> { static constexpr const TypeIndex value = TypeIndex::UInt64; };
|
||||
template <> struct TypeId<UInt128> { static constexpr const TypeIndex value = TypeIndex::UInt64; };
|
||||
template <> struct TypeId<UInt8> { static constexpr const TypeIndex value = TypeIndex::UInt8; };
|
||||
template <> struct TypeId<UInt16> { static constexpr const TypeIndex value = TypeIndex::UInt16; };
|
||||
template <> struct TypeId<UInt32> { static constexpr const TypeIndex value = TypeIndex::UInt32; };
|
||||
template <> struct TypeId<UInt64> { static constexpr const TypeIndex value = TypeIndex::UInt64; };
|
||||
template <> struct TypeId<UInt128> { static constexpr const TypeIndex value = TypeIndex::UInt64; };
|
||||
template <> struct TypeId<UInt256> { static constexpr const TypeIndex value = TypeIndex::UInt256; };
|
||||
template <> struct TypeId<Int8> { static constexpr const TypeIndex value = TypeIndex::Int8; };
|
||||
template <> struct TypeId<Int8> { static constexpr const TypeIndex value = TypeIndex::Int8; };
|
||||
template <> struct TypeId<Int16> { static constexpr const TypeIndex value = TypeIndex::Int16; };
|
||||
template <> struct TypeId<Int32> { static constexpr const TypeIndex value = TypeIndex::Int32; };
|
||||
template <> struct TypeId<Int64> { static constexpr const TypeIndex value = TypeIndex::Int64; };
|
||||
template <> struct TypeId<Int128> { static constexpr const TypeIndex value = TypeIndex::Int128; };
|
||||
template <> struct TypeId<Int256> { static constexpr const TypeIndex value = TypeIndex::Int256; };
|
||||
template <> struct TypeId<Float32> { static constexpr const TypeIndex value = TypeIndex::Float32; };
|
||||
template <> struct TypeId<Float64> { static constexpr const TypeIndex value = TypeIndex::Float64; };
|
||||
template <> struct TypeId<Float32> { static constexpr const TypeIndex value = TypeIndex::Float32; };
|
||||
template <> struct TypeId<Float64> { static constexpr const TypeIndex value = TypeIndex::Float64; };
|
||||
template <> struct TypeId<UUID> { static constexpr const TypeIndex value = TypeIndex::UUID; };
|
||||
|
||||
/// Not a data type in database, defined just for convenience.
|
||||
using Strings = std::vector<String>;
|
||||
|
@ -1,14 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/strong_typedef.h>
|
||||
#include <common/extended_types.h>
|
||||
#include <Core/Types.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
STRONG_TYPEDEF(UInt128, UUID)
|
||||
|
||||
namespace UUIDHelpers
|
||||
{
|
||||
/// Generate random UUID.
|
||||
|
@ -25,7 +25,6 @@
|
||||
// Poco/MongoDB/BSONWriter.h:54: void writeCString(const std::string & value);
|
||||
// src/IO/WriteHelpers.h:146 #define writeCString(s, buf)
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <ext/enumerate.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <DataTypes/DataTypeCustom.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
|
@ -16,6 +16,16 @@ SerializationPtr DataTypeUUID::doGetDefaultSerialization() const
|
||||
return std::make_shared<SerializationUUID>();
|
||||
}
|
||||
|
||||
Field DataTypeUUID::getDefault() const
|
||||
{
|
||||
return UUID{};
|
||||
}
|
||||
|
||||
MutableColumnPtr DataTypeUUID::createColumn() const
|
||||
{
|
||||
return ColumnVector<UUID>::create();
|
||||
}
|
||||
|
||||
void registerDataTypeUUID(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataType("UUID", [] { return DataTypePtr(std::make_shared<DataTypeUUID>()); });
|
||||
|
@ -1,25 +1,44 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/DataTypeNumberBase.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Core/UUID.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class DataTypeUUID final : public DataTypeNumberBase<UInt128>
|
||||
class DataTypeUUID : public IDataType
|
||||
{
|
||||
|
||||
public:
|
||||
static constexpr bool is_parametric = false;
|
||||
|
||||
using FieldType = UUID;
|
||||
using ColumnType = ColumnVector<UUID>;
|
||||
|
||||
const char * getFamilyName() const override { return "UUID"; }
|
||||
TypeIndex getTypeId() const override { return TypeIndex::UUID; }
|
||||
|
||||
Field getDefault() const override;
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
|
||||
bool isParametric() const override { return false; }
|
||||
bool haveSubtypes() const override { return false; }
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
bool canBeUsedInBitOperations() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
bool canBeInsideLowCardinality() const override { return false; }
|
||||
|
||||
bool canBePromoted() const override { return false; }
|
||||
bool shouldAlignRightInPrettyFormats() const override { return false; }
|
||||
bool textCanContainOnlyValidUTF8() const override { return true; }
|
||||
bool isComparable() const override { return true; }
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||
bool haveMaximumSizeOfValue() const override { return true; }
|
||||
size_t getSizeOfValueInMemory() const override { return sizeof(UUID); }
|
||||
bool isCategorial() const override { return true; }
|
||||
bool canBeInsideLowCardinality() const override { return true; }
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
};
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <DataTypes/FieldToDataType.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
@ -8,6 +7,7 @@
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Common/Exception.h>
|
||||
@ -69,6 +69,11 @@ DataTypePtr FieldToDataType::operator() (const Int256 &) const
|
||||
return std::make_shared<DataTypeInt256>();
|
||||
}
|
||||
|
||||
DataTypePtr FieldToDataType::operator() (const UUID &) const
|
||||
{
|
||||
return std::make_shared<DataTypeUUID>();
|
||||
}
|
||||
|
||||
DataTypePtr FieldToDataType::operator() (const String &) const
|
||||
{
|
||||
return std::make_shared<DataTypeString>();
|
||||
|
@ -1,5 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <memory>
|
||||
#include <Core/Types.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
|
||||
@ -24,6 +27,7 @@ public:
|
||||
DataTypePtr operator() (const Int64 & x) const;
|
||||
DataTypePtr operator() (const Int128 & x) const;
|
||||
DataTypePtr operator() (const Int256 & x) const;
|
||||
DataTypePtr operator() (const UUID & x) const;
|
||||
DataTypePtr operator() (const Float64 & x) const;
|
||||
DataTypePtr operator() (const String & x) const;
|
||||
DataTypePtr operator() (const Array & x) const;
|
||||
|
@ -50,7 +50,6 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
|
||||
// Poco/MongoDB/BSONWriter.h:54: void writeCString(const std::string & value);
|
||||
// src/IO/WriteHelpers.h:146 #define writeCString(s, buf)
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <ext/enumerate.h>
|
||||
#include <DataStreams/MongoDBBlockInputStream.h>
|
||||
|
||||
|
@ -30,7 +30,6 @@ void registerDictionarySourceRedis(DictionarySourceFactory & factory)
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
#include "RedisBlockInputStream.h"
|
||||
|
||||
|
@ -36,7 +36,6 @@
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Core/AccurateComparison.h>
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Common/UnicodeBar.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionUnaryArithmetic.h>
|
||||
#include <DataTypes/NumberTraits.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -668,7 +668,6 @@ void formatUUID(std::reverse_iterator<const UInt8 *> src16, UInt8 * dst36);
|
||||
inline void writeUUIDText(const UUID & uuid, WriteBuffer & buf)
|
||||
{
|
||||
char s[36];
|
||||
|
||||
formatUUID(std::reverse_iterator<const UInt8 *>(reinterpret_cast<const UInt8 *>(&uuid) + 16), reinterpret_cast<UInt8 *>(s));
|
||||
buf.write(s, sizeof(s));
|
||||
}
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/Native.h>
|
||||
|
@ -185,7 +185,7 @@ static inline std::tuple<NamesAndTypesList, NamesAndTypesList, NamesAndTypesList
|
||||
{
|
||||
const auto & prefix_limit = function->arguments->children[0]->as<ASTLiteral>();
|
||||
|
||||
if (prefix_limit && isInt64FieldType(prefix_limit->value.getType()))
|
||||
if (prefix_limit && isInt64OrUInt64FieldType(prefix_limit->value.getType()))
|
||||
res->children.back() = std::make_shared<ASTIdentifier>(function->name);
|
||||
}
|
||||
}
|
||||
|
@ -29,7 +29,7 @@ void RewriteSumIfFunctionMatcher::visit(const ASTFunction & func, ASTPtr & ast,
|
||||
{
|
||||
/// sumIf(1, cond) -> countIf(cond)
|
||||
const auto * literal = func_arguments[0]->as<ASTLiteral>();
|
||||
if (!literal || !DB::isInt64FieldType(literal->value.getType()))
|
||||
if (!literal || !DB::isInt64OrUInt64FieldType(literal->value.getType()))
|
||||
return;
|
||||
|
||||
if (func_arguments.size() == 2 && literal->value.get<UInt64>() == 1)
|
||||
@ -54,7 +54,7 @@ void RewriteSumIfFunctionMatcher::visit(const ASTFunction & func, ASTPtr & ast,
|
||||
|
||||
if (first_literal && second_literal)
|
||||
{
|
||||
if (!DB::isInt64FieldType(first_literal->value.getType()) || !DB::isInt64FieldType(second_literal->value.getType()))
|
||||
if (!DB::isInt64OrUInt64FieldType(first_literal->value.getType()) || !DB::isInt64OrUInt64FieldType(second_literal->value.getType()))
|
||||
return;
|
||||
|
||||
auto first_value = first_literal->value.get<UInt64>();
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include <optional>
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Core/Row.h>
|
||||
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
@ -1,9 +1,11 @@
|
||||
#include <Interpreters/WindowDescription.h>
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
@ -17,7 +17,6 @@
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
||||
#include <Core/AccurateComparison.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
|
@ -415,7 +415,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
|
||||
// them, and we want to have consistent formatting.
|
||||
if (tuple_arguments_valid && lit_right)
|
||||
{
|
||||
if (isInt64FieldType(lit_right->value.getType())
|
||||
if (isInt64OrUInt64FieldType(lit_right->value.getType())
|
||||
&& lit_right->value.get<Int64>() >= 0)
|
||||
{
|
||||
if (frame.need_parens)
|
||||
|
@ -1,8 +1,10 @@
|
||||
#include <Parsers/ASTWindowDefinition.h>
|
||||
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Processors/Formats/Impl/ValuesBlockInputFormat.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Core/Block.h>
|
||||
#include <common/find_symbols.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
@ -1460,7 +1460,7 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction
|
||||
return;
|
||||
}
|
||||
|
||||
if (!isInt64FieldType(argument_types[1]->getDefault().getType()))
|
||||
if (!isInt64OrUInt64FieldType(argument_types[1]->getDefault().getType()))
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Offset must be an integer, '{}' given",
|
||||
|
@ -5,7 +5,6 @@
|
||||
|
||||
#include <Poco/File.h>
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
||||
#include <Storages/MergeTree/MergeTreeSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeReverseSelectProcessor.h>
|
||||
@ -40,6 +39,7 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event SelectedParts;
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Interpreters/InterpreterAlterQuery.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <Core/Defines.h>
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/Macros.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
|
Loading…
Reference in New Issue
Block a user