mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
commit
7d8f0a6140
@ -180,6 +180,21 @@ if (OS_LINUX AND CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
|
|||||||
endif ()
|
endif ()
|
||||||
endif ()
|
endif ()
|
||||||
|
|
||||||
|
if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
|
||||||
|
# If we leave this optimization enabled, gcc-7 replaces a pair of SSE intrinsics (16 byte load, store) with a call to memcpy. It leads to slow code. This is compiler bug.
|
||||||
|
# It looks like this:
|
||||||
|
#
|
||||||
|
# (gdb) bt
|
||||||
|
#0 memcpy (destination=0x7faa6e9f1638, source=0x7faa81d9e9a8, size=16) at ../libs/libmemcpy/memcpy.h:11
|
||||||
|
#1 0x0000000005341c5f in _mm_storeu_si128 (__B=..., __P=<optimized out>) at /usr/lib/gcc/x86_64-linux-gnu/7/include/emmintrin.h:720
|
||||||
|
#2 memcpySmallAllowReadWriteOverflow15Impl (n=<optimized out>, src=<optimized out>, dst=<optimized out>) at ../dbms/src/Common/memcpySmall.h:37
|
||||||
|
#3 memcpySmallAllowReadWriteOverflow15 (n=<optimized out>, src=<optimized out>, dst=<optimized out>) at ../dbms/src/Common/memcpySmall.h:52
|
||||||
|
#4 extractKeysAndPlaceInPoolContiguous (pool=..., keys=..., key_columns=..., keys_size=<optimized out>, i=<optimized out>) at ../dbms/src/Interpreters/AggregationCommon.h:262
|
||||||
|
|
||||||
|
set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -fno-tree-loop-distribute-patterns")
|
||||||
|
set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -fno-tree-loop-distribute-patterns")
|
||||||
|
endif ()
|
||||||
|
|
||||||
if (USE_STATIC_LIBRARIES AND HAVE_NO_PIE)
|
if (USE_STATIC_LIBRARIES AND HAVE_NO_PIE)
|
||||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${FLAG_NO_PIE}")
|
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${FLAG_NO_PIE}")
|
||||||
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAG_NO_PIE}")
|
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAG_NO_PIE}")
|
||||||
|
@ -1,6 +1,7 @@
|
|||||||
#include <cstring>
|
#include <cstring>
|
||||||
#include <cmath>
|
#include <cmath>
|
||||||
|
|
||||||
|
#include <common/unaligned.h>
|
||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
#include <Common/Arena.h>
|
#include <Common/Arena.h>
|
||||||
#include <Common/SipHash.h>
|
#include <Common/SipHash.h>
|
||||||
@ -36,14 +37,14 @@ template <typename T>
|
|||||||
StringRef ColumnVector<T>::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
|
StringRef ColumnVector<T>::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
|
||||||
{
|
{
|
||||||
auto pos = arena.allocContinue(sizeof(T), begin);
|
auto pos = arena.allocContinue(sizeof(T), begin);
|
||||||
memcpy(pos, &data[n], sizeof(T));
|
unalignedStore(pos, data[n]);
|
||||||
return StringRef(pos, sizeof(T));
|
return StringRef(pos, sizeof(T));
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
const char * ColumnVector<T>::deserializeAndInsertFromArena(const char * pos)
|
const char * ColumnVector<T>::deserializeAndInsertFromArena(const char * pos)
|
||||||
{
|
{
|
||||||
data.push_back(*reinterpret_cast<const T *>(pos));
|
data.push_back(unalignedLoad<T>(pos));
|
||||||
return pos + sizeof(T);
|
return pos + sizeof(T);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
#include <boost/noncopyable.hpp>
|
#include <boost/noncopyable.hpp>
|
||||||
#include <common/likely.h>
|
#include <common/likely.h>
|
||||||
#include <Core/Defines.h>
|
#include <Core/Defines.h>
|
||||||
|
#include <Common/memcpySmall.h>
|
||||||
#include <Common/ProfileEvents.h>
|
#include <Common/ProfileEvents.h>
|
||||||
#include <Common/Allocator.h>
|
#include <Common/Allocator.h>
|
||||||
|
|
||||||
@ -31,12 +32,15 @@ namespace DB
|
|||||||
class Arena : private boost::noncopyable
|
class Arena : private boost::noncopyable
|
||||||
{
|
{
|
||||||
private:
|
private:
|
||||||
|
/// Padding allows to use 'memcpySmallAllowReadWriteOverflow15' instead of 'memcpy'.
|
||||||
|
static constexpr size_t pad_right = 15;
|
||||||
|
|
||||||
/// Contiguous chunk of memory and pointer to free space inside it. Member of single-linked list.
|
/// Contiguous chunk of memory and pointer to free space inside it. Member of single-linked list.
|
||||||
struct Chunk : private Allocator<false> /// empty base optimization
|
struct Chunk : private Allocator<false> /// empty base optimization
|
||||||
{
|
{
|
||||||
char * begin;
|
char * begin;
|
||||||
char * pos;
|
char * pos;
|
||||||
char * end;
|
char * end; /// does not include padding.
|
||||||
|
|
||||||
Chunk * prev;
|
Chunk * prev;
|
||||||
|
|
||||||
@ -47,7 +51,7 @@ private:
|
|||||||
|
|
||||||
begin = reinterpret_cast<char *>(Allocator::alloc(size_));
|
begin = reinterpret_cast<char *>(Allocator::alloc(size_));
|
||||||
pos = begin;
|
pos = begin;
|
||||||
end = begin + size_;
|
end = begin + size_ - pad_right;
|
||||||
prev = prev_;
|
prev = prev_;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -59,7 +63,7 @@ private:
|
|||||||
delete prev;
|
delete prev;
|
||||||
}
|
}
|
||||||
|
|
||||||
size_t size() const { return end - begin; }
|
size_t size() const { return end + pad_right - begin; }
|
||||||
size_t remaining() const { return end - pos; }
|
size_t remaining() const { return end - pos; }
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -95,7 +99,7 @@ private:
|
|||||||
/// Add next contiguous chunk of memory with size not less than specified.
|
/// Add next contiguous chunk of memory with size not less than specified.
|
||||||
void NO_INLINE addChunk(size_t min_size)
|
void NO_INLINE addChunk(size_t min_size)
|
||||||
{
|
{
|
||||||
head = new Chunk(nextSize(min_size), head);
|
head = new Chunk(nextSize(min_size + pad_right), head);
|
||||||
size_in_bytes += head->size();
|
size_in_bytes += head->size();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -216,7 +220,7 @@ public:
|
|||||||
{
|
{
|
||||||
char * res = alloc(new_size);
|
char * res = alloc(new_size);
|
||||||
if (old_data)
|
if (old_data)
|
||||||
memcpy(res, old_data, old_size);
|
memcpySmallAllowReadWriteOverflow15(res, old_data, old_size);
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -224,7 +228,7 @@ public:
|
|||||||
{
|
{
|
||||||
char * res = alignedAlloc(new_size, alignment);
|
char * res = alignedAlloc(new_size, alignment);
|
||||||
if (old_data)
|
if (old_data)
|
||||||
memcpy(res, old_data, old_size);
|
memcpySmallAllowReadWriteOverflow15(res, old_data, old_size);
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -232,14 +236,14 @@ public:
|
|||||||
const char * insert(const char * data, size_t size)
|
const char * insert(const char * data, size_t size)
|
||||||
{
|
{
|
||||||
char * res = alloc(size);
|
char * res = alloc(size);
|
||||||
memcpy(res, data, size);
|
memcpySmallAllowReadWriteOverflow15(res, data, size);
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
const char * alignedInsert(const char * data, size_t size, size_t alignment)
|
const char * alignedInsert(const char * data, size_t size, size_t alignment)
|
||||||
{
|
{
|
||||||
char * res = alignedAlloc(size, alignment);
|
char * res = alignedAlloc(size, alignment);
|
||||||
memcpy(res, data, size);
|
memcpySmallAllowReadWriteOverflow15(res, data, size);
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
#include <Common/Arena.h>
|
#include <Common/Arena.h>
|
||||||
#include <Common/UInt128.h>
|
#include <Common/UInt128.h>
|
||||||
#include <Common/HashTable/Hash.h>
|
#include <Common/HashTable/Hash.h>
|
||||||
|
#include <Common/memcpySmall.h>
|
||||||
#include <Core/Defines.h>
|
#include <Core/Defines.h>
|
||||||
#include <common/StringRef.h>
|
#include <common/StringRef.h>
|
||||||
#include <Columns/IColumn.h>
|
#include <Columns/IColumn.h>
|
||||||
@ -173,26 +174,6 @@ static inline T ALWAYS_INLINE packFixed(
|
|||||||
|
|
||||||
|
|
||||||
/// Hash a set of keys into a UInt128 value.
|
/// Hash a set of keys into a UInt128 value.
|
||||||
static inline UInt128 ALWAYS_INLINE hash128(
|
|
||||||
size_t i, size_t keys_size, const ColumnRawPtrs & key_columns, StringRefs & keys)
|
|
||||||
{
|
|
||||||
UInt128 key;
|
|
||||||
SipHash hash;
|
|
||||||
|
|
||||||
for (size_t j = 0; j < keys_size; ++j)
|
|
||||||
{
|
|
||||||
/// Hashes the key.
|
|
||||||
keys[j] = key_columns[j]->getDataAtWithTerminatingZero(i);
|
|
||||||
hash.update(keys[j].data, keys[j].size);
|
|
||||||
}
|
|
||||||
|
|
||||||
hash.get128(key.low, key.high);
|
|
||||||
|
|
||||||
return key;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/// Almost the same as above but it doesn't return any reference to key data.
|
|
||||||
static inline UInt128 ALWAYS_INLINE hash128(
|
static inline UInt128 ALWAYS_INLINE hash128(
|
||||||
size_t i, size_t keys_size, const ColumnRawPtrs & key_columns)
|
size_t i, size_t keys_size, const ColumnRawPtrs & key_columns)
|
||||||
{
|
{
|
||||||
@ -215,78 +196,18 @@ static inline StringRef * ALWAYS_INLINE placeKeysInPool(
|
|||||||
for (size_t j = 0; j < keys_size; ++j)
|
for (size_t j = 0; j < keys_size; ++j)
|
||||||
{
|
{
|
||||||
char * place = pool.alloc(keys[j].size);
|
char * place = pool.alloc(keys[j].size);
|
||||||
memcpy(place, keys[j].data, keys[j].size); /// TODO padding in Arena and memcpySmall
|
memcpySmallAllowReadWriteOverflow15(place, keys[j].data, keys[j].size);
|
||||||
keys[j].data = place;
|
keys[j].data = place;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Place the StringRefs on the newly copied keys in the pool.
|
/// Place the StringRefs on the newly copied keys in the pool.
|
||||||
char * res = pool.alloc(keys_size * sizeof(StringRef));
|
char * res = pool.alignedAlloc(keys_size * sizeof(StringRef), alignof(StringRef));
|
||||||
memcpy(res, keys.data(), keys_size * sizeof(StringRef));
|
memcpySmallAllowReadWriteOverflow15(res, keys.data(), keys_size * sizeof(StringRef));
|
||||||
|
|
||||||
return reinterpret_cast<StringRef *>(res);
|
return reinterpret_cast<StringRef *>(res);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/// Copy keys to the pool. Then put into pool StringRefs to them and return the pointer to the first.
|
|
||||||
static inline StringRef * ALWAYS_INLINE extractKeysAndPlaceInPool(
|
|
||||||
size_t i, size_t keys_size, const ColumnRawPtrs & key_columns, StringRefs & keys, Arena & pool)
|
|
||||||
{
|
|
||||||
for (size_t j = 0; j < keys_size; ++j)
|
|
||||||
{
|
|
||||||
keys[j] = key_columns[j]->getDataAtWithTerminatingZero(i);
|
|
||||||
char * place = pool.alloc(keys[j].size);
|
|
||||||
memcpy(place, keys[j].data, keys[j].size);
|
|
||||||
keys[j].data = place;
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Place the StringRefs on the newly copied keys in the pool.
|
|
||||||
char * res = pool.alloc(keys_size * sizeof(StringRef));
|
|
||||||
memcpy(res, keys.data(), keys_size * sizeof(StringRef));
|
|
||||||
|
|
||||||
return reinterpret_cast<StringRef *>(res);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/// Copy the specified keys to a continuous memory chunk of a pool.
|
|
||||||
/// Subsequently append StringRef objects referring to each key.
|
|
||||||
///
|
|
||||||
/// [key1][key2]...[keyN][ref1][ref2]...[refN]
|
|
||||||
/// ^ ^ : | |
|
|
||||||
/// +-----|--------:-----+ |
|
|
||||||
/// : +--------:-----------+
|
|
||||||
/// : :
|
|
||||||
/// <-------------->
|
|
||||||
/// (1)
|
|
||||||
///
|
|
||||||
/// Return a StringRef object, referring to the area (1) of the memory
|
|
||||||
/// chunk that contains the keys. In other words, we ignore their StringRefs.
|
|
||||||
inline StringRef ALWAYS_INLINE extractKeysAndPlaceInPoolContiguous(
|
|
||||||
size_t i, size_t keys_size, const ColumnRawPtrs & key_columns, StringRefs & keys, Arena & pool)
|
|
||||||
{
|
|
||||||
size_t sum_keys_size = 0;
|
|
||||||
for (size_t j = 0; j < keys_size; ++j)
|
|
||||||
{
|
|
||||||
keys[j] = key_columns[j]->getDataAtWithTerminatingZero(i);
|
|
||||||
sum_keys_size += keys[j].size;
|
|
||||||
}
|
|
||||||
|
|
||||||
char * res = pool.alloc(sum_keys_size + keys_size * sizeof(StringRef));
|
|
||||||
char * place = res;
|
|
||||||
|
|
||||||
for (size_t j = 0; j < keys_size; ++j)
|
|
||||||
{
|
|
||||||
memcpy(place, keys[j].data, keys[j].size);
|
|
||||||
keys[j].data = place;
|
|
||||||
place += keys[j].size;
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Place the StringRefs on the newly copied keys in the pool.
|
|
||||||
memcpy(place, keys.data(), keys_size * sizeof(StringRef));
|
|
||||||
|
|
||||||
return {res, sum_keys_size};
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/** Serialize keys into a continuous chunk of memory.
|
/** Serialize keys into a continuous chunk of memory.
|
||||||
*/
|
*/
|
||||||
static inline StringRef ALWAYS_INLINE serializeKeysToPoolContiguous(
|
static inline StringRef ALWAYS_INLINE serializeKeysToPoolContiguous(
|
||||||
|
@ -412,29 +412,19 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod()
|
|||||||
*/
|
*/
|
||||||
|
|
||||||
size_t keys_bytes = 0;
|
size_t keys_bytes = 0;
|
||||||
|
|
||||||
size_t num_contiguous_keys = 0;
|
|
||||||
size_t num_fixed_contiguous_keys = 0;
|
size_t num_fixed_contiguous_keys = 0;
|
||||||
size_t num_string_keys = 0;
|
|
||||||
|
|
||||||
key_sizes.resize(params.keys_size);
|
key_sizes.resize(params.keys_size);
|
||||||
for (size_t j = 0; j < params.keys_size; ++j)
|
for (size_t j = 0; j < params.keys_size; ++j)
|
||||||
{
|
{
|
||||||
if (types_removed_nullable[j]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
|
if (types_removed_nullable[j]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
|
||||||
{
|
{
|
||||||
++num_contiguous_keys;
|
|
||||||
|
|
||||||
if (types_removed_nullable[j]->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion())
|
if (types_removed_nullable[j]->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion())
|
||||||
{
|
{
|
||||||
++num_fixed_contiguous_keys;
|
++num_fixed_contiguous_keys;
|
||||||
key_sizes[j] = types_removed_nullable[j]->getSizeOfValueInMemory();
|
key_sizes[j] = types_removed_nullable[j]->getSizeOfValueInMemory();
|
||||||
keys_bytes += key_sizes[j];
|
keys_bytes += key_sizes[j];
|
||||||
}
|
}
|
||||||
|
|
||||||
if (types_removed_nullable[j]->isString())
|
|
||||||
{
|
|
||||||
++num_string_keys;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -489,23 +479,7 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod()
|
|||||||
if (params.keys_size == 1 && types_removed_nullable[0]->isFixedString())
|
if (params.keys_size == 1 && types_removed_nullable[0]->isFixedString())
|
||||||
return AggregatedDataVariants::Type::key_fixed_string;
|
return AggregatedDataVariants::Type::key_fixed_string;
|
||||||
|
|
||||||
/** If it is possible to use 'concat' method due to one-to-one correspondense. Otherwise the method will be 'serialized'.
|
|
||||||
*/
|
|
||||||
if (params.keys_size == num_contiguous_keys && num_fixed_contiguous_keys + 1 >= num_contiguous_keys)
|
|
||||||
return AggregatedDataVariants::Type::concat;
|
|
||||||
|
|
||||||
/** For case with multiple strings, we use 'concat' method despite the fact, that correspondense is not one-to-one.
|
|
||||||
* Concat will concatenate strings including its zero terminators.
|
|
||||||
* But if strings contains zero bytes in between, different keys may clash.
|
|
||||||
* For example, keys ('a\0b', 'c') and ('a', 'b\0c') will be aggregated as one key.
|
|
||||||
* This is documented behaviour. It may be avoided by just switching to 'serialized' method, which is less efficient.
|
|
||||||
*/
|
|
||||||
if (params.keys_size == num_fixed_contiguous_keys + num_string_keys)
|
|
||||||
return AggregatedDataVariants::Type::concat;
|
|
||||||
|
|
||||||
return AggregatedDataVariants::Type::serialized;
|
return AggregatedDataVariants::Type::serialized;
|
||||||
|
|
||||||
/// NOTE AggregatedDataVariants::Type::hashed is not used. It's proven to be less efficient than 'serialized' in most cases.
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -2162,7 +2136,6 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
|
|||||||
M(key_fixed_string) \
|
M(key_fixed_string) \
|
||||||
M(keys128) \
|
M(keys128) \
|
||||||
M(keys256) \
|
M(keys256) \
|
||||||
M(concat) \
|
|
||||||
M(serialized) \
|
M(serialized) \
|
||||||
|
|
||||||
#define M(NAME) \
|
#define M(NAME) \
|
||||||
|
@ -67,13 +67,11 @@ using AggregatedDataWithUInt64Key = HashMap<UInt64, AggregateDataPtr, HashCRC32<
|
|||||||
using AggregatedDataWithStringKey = HashMapWithSavedHash<StringRef, AggregateDataPtr>;
|
using AggregatedDataWithStringKey = HashMapWithSavedHash<StringRef, AggregateDataPtr>;
|
||||||
using AggregatedDataWithKeys128 = HashMap<UInt128, AggregateDataPtr, UInt128HashCRC32>;
|
using AggregatedDataWithKeys128 = HashMap<UInt128, AggregateDataPtr, UInt128HashCRC32>;
|
||||||
using AggregatedDataWithKeys256 = HashMap<UInt256, AggregateDataPtr, UInt256HashCRC32>;
|
using AggregatedDataWithKeys256 = HashMap<UInt256, AggregateDataPtr, UInt256HashCRC32>;
|
||||||
using AggregatedDataHashed = HashMap<UInt128, std::pair<StringRef*, AggregateDataPtr>, UInt128TrivialHash>;
|
|
||||||
|
|
||||||
using AggregatedDataWithUInt64KeyTwoLevel = TwoLevelHashMap<UInt64, AggregateDataPtr, HashCRC32<UInt64>>;
|
using AggregatedDataWithUInt64KeyTwoLevel = TwoLevelHashMap<UInt64, AggregateDataPtr, HashCRC32<UInt64>>;
|
||||||
using AggregatedDataWithStringKeyTwoLevel = TwoLevelHashMapWithSavedHash<StringRef, AggregateDataPtr>;
|
using AggregatedDataWithStringKeyTwoLevel = TwoLevelHashMapWithSavedHash<StringRef, AggregateDataPtr>;
|
||||||
using AggregatedDataWithKeys128TwoLevel = TwoLevelHashMap<UInt128, AggregateDataPtr, UInt128HashCRC32>;
|
using AggregatedDataWithKeys128TwoLevel = TwoLevelHashMap<UInt128, AggregateDataPtr, UInt128HashCRC32>;
|
||||||
using AggregatedDataWithKeys256TwoLevel = TwoLevelHashMap<UInt256, AggregateDataPtr, UInt256HashCRC32>;
|
using AggregatedDataWithKeys256TwoLevel = TwoLevelHashMap<UInt256, AggregateDataPtr, UInt256HashCRC32>;
|
||||||
using AggregatedDataHashedTwoLevel = TwoLevelHashMap<UInt128, std::pair<StringRef*, AggregateDataPtr>, UInt128TrivialHash>;
|
|
||||||
|
|
||||||
/** Variants with better hash function, using more than 32 bits for hash.
|
/** Variants with better hash function, using more than 32 bits for hash.
|
||||||
* Using for merging phase of external aggregation, where number of keys may be far greater than 4 billion,
|
* Using for merging phase of external aggregation, where number of keys may be far greater than 4 billion,
|
||||||
@ -119,7 +117,7 @@ struct AggregationMethodOneNumber
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Get the key from the key columns for insertion into the hash table.
|
/// Get the key from the key columns for insertion into the hash table.
|
||||||
Key getKey(
|
ALWAYS_INLINE Key getKey(
|
||||||
const ColumnRawPtrs & /*key_columns*/,
|
const ColumnRawPtrs & /*key_columns*/,
|
||||||
size_t /*keys_size*/, /// Number of key columns.
|
size_t /*keys_size*/, /// Number of key columns.
|
||||||
size_t i, /// From which row of the block, get the key.
|
size_t i, /// From which row of the block, get the key.
|
||||||
@ -137,13 +135,13 @@ struct AggregationMethodOneNumber
|
|||||||
|
|
||||||
/** Place additional data, if necessary, in case a new key was inserted into the hash table.
|
/** Place additional data, if necessary, in case a new key was inserted into the hash table.
|
||||||
*/
|
*/
|
||||||
static void onNewKey(typename Data::value_type & /*value*/, size_t /*keys_size*/, StringRefs & /*keys*/, Arena & /*pool*/)
|
static ALWAYS_INLINE void onNewKey(typename Data::value_type & /*value*/, size_t /*keys_size*/, StringRefs & /*keys*/, Arena & /*pool*/)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
/** The action to be taken if the key is not new. For example, roll back the memory allocation in the pool.
|
/** The action to be taken if the key is not new. For example, roll back the memory allocation in the pool.
|
||||||
*/
|
*/
|
||||||
static void onExistingKey(const Key & /*key*/, StringRefs & /*keys*/, Arena & /*pool*/) {}
|
static ALWAYS_INLINE void onExistingKey(const Key & /*key*/, StringRefs & /*keys*/, Arena & /*pool*/) {}
|
||||||
|
|
||||||
/** Do not use optimization for consecutive keys.
|
/** Do not use optimization for consecutive keys.
|
||||||
*/
|
*/
|
||||||
@ -188,7 +186,7 @@ struct AggregationMethodString
|
|||||||
chars = &column_string.getChars();
|
chars = &column_string.getChars();
|
||||||
}
|
}
|
||||||
|
|
||||||
Key getKey(
|
ALWAYS_INLINE Key getKey(
|
||||||
const ColumnRawPtrs & /*key_columns*/,
|
const ColumnRawPtrs & /*key_columns*/,
|
||||||
size_t /*keys_size*/,
|
size_t /*keys_size*/,
|
||||||
size_t i,
|
size_t i,
|
||||||
@ -205,12 +203,12 @@ struct AggregationMethodString
|
|||||||
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
||||||
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
||||||
|
|
||||||
static void onNewKey(typename Data::value_type & value, size_t /*keys_size*/, StringRefs & /*keys*/, Arena & pool)
|
static ALWAYS_INLINE void onNewKey(typename Data::value_type & value, size_t /*keys_size*/, StringRefs & /*keys*/, Arena & pool)
|
||||||
{
|
{
|
||||||
value.first.data = pool.insert(value.first.data, value.first.size);
|
value.first.data = pool.insert(value.first.data, value.first.size);
|
||||||
}
|
}
|
||||||
|
|
||||||
static void onExistingKey(const Key & /*key*/, StringRefs & /*keys*/, Arena & /*pool*/) {}
|
static ALWAYS_INLINE void onExistingKey(const Key & /*key*/, StringRefs & /*keys*/, Arena & /*pool*/) {}
|
||||||
|
|
||||||
static const bool no_consecutive_keys_optimization = false;
|
static const bool no_consecutive_keys_optimization = false;
|
||||||
|
|
||||||
@ -251,7 +249,7 @@ struct AggregationMethodFixedString
|
|||||||
chars = &column_string.getChars();
|
chars = &column_string.getChars();
|
||||||
}
|
}
|
||||||
|
|
||||||
Key getKey(
|
ALWAYS_INLINE Key getKey(
|
||||||
const ColumnRawPtrs &,
|
const ColumnRawPtrs &,
|
||||||
size_t,
|
size_t,
|
||||||
size_t i,
|
size_t i,
|
||||||
@ -266,12 +264,12 @@ struct AggregationMethodFixedString
|
|||||||
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
||||||
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
||||||
|
|
||||||
static void onNewKey(typename Data::value_type & value, size_t, StringRefs &, Arena & pool)
|
static ALWAYS_INLINE void onNewKey(typename Data::value_type & value, size_t, StringRefs &, Arena & pool)
|
||||||
{
|
{
|
||||||
value.first.data = pool.insert(value.first.data, value.first.size);
|
value.first.data = pool.insert(value.first.data, value.first.size);
|
||||||
}
|
}
|
||||||
|
|
||||||
static void onExistingKey(const Key &, StringRefs &, Arena &) {}
|
static ALWAYS_INLINE void onExistingKey(const Key &, StringRefs &, Arena &) {}
|
||||||
|
|
||||||
static const bool no_consecutive_keys_optimization = false;
|
static const bool no_consecutive_keys_optimization = false;
|
||||||
|
|
||||||
@ -407,7 +405,7 @@ struct AggregationMethodKeysFixed
|
|||||||
Base::init(key_columns);
|
Base::init(key_columns);
|
||||||
}
|
}
|
||||||
|
|
||||||
Key getKey(
|
ALWAYS_INLINE Key getKey(
|
||||||
const ColumnRawPtrs & key_columns,
|
const ColumnRawPtrs & key_columns,
|
||||||
size_t keys_size,
|
size_t keys_size,
|
||||||
size_t i,
|
size_t i,
|
||||||
@ -428,11 +426,11 @@ struct AggregationMethodKeysFixed
|
|||||||
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
||||||
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
||||||
|
|
||||||
static void onNewKey(typename Data::value_type &, size_t, StringRefs &, Arena &)
|
static ALWAYS_INLINE void onNewKey(typename Data::value_type &, size_t, StringRefs &, Arena &)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
static void onExistingKey(const Key &, StringRefs &, Arena &) {}
|
static ALWAYS_INLINE void onExistingKey(const Key &, StringRefs &, Arena &) {}
|
||||||
|
|
||||||
static const bool no_consecutive_keys_optimization = false;
|
static const bool no_consecutive_keys_optimization = false;
|
||||||
|
|
||||||
@ -487,88 +485,7 @@ struct AggregationMethodKeysFixed
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
/// Aggregates by key concatenation. (In this case, strings containing zeros in the middle can stick together.)
|
|
||||||
template <typename TData>
|
|
||||||
struct AggregationMethodConcat
|
|
||||||
{
|
|
||||||
using Data = TData;
|
|
||||||
using Key = typename Data::key_type;
|
|
||||||
using Mapped = typename Data::mapped_type;
|
|
||||||
using iterator = typename Data::iterator;
|
|
||||||
using const_iterator = typename Data::const_iterator;
|
|
||||||
|
|
||||||
Data data;
|
|
||||||
|
|
||||||
AggregationMethodConcat() {}
|
|
||||||
|
|
||||||
template <typename Other>
|
|
||||||
AggregationMethodConcat(const Other & other) : data(other.data) {}
|
|
||||||
|
|
||||||
struct State
|
|
||||||
{
|
|
||||||
void init(ColumnRawPtrs &)
|
|
||||||
{
|
|
||||||
}
|
|
||||||
|
|
||||||
Key getKey(
|
|
||||||
const ColumnRawPtrs & key_columns,
|
|
||||||
size_t keys_size,
|
|
||||||
size_t i,
|
|
||||||
const Sizes &,
|
|
||||||
StringRefs & keys,
|
|
||||||
Arena & pool) const
|
|
||||||
{
|
|
||||||
return extractKeysAndPlaceInPoolContiguous(i, keys_size, key_columns, keys, pool);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
|
||||||
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
|
||||||
|
|
||||||
static void onNewKey(typename Data::value_type &, size_t, StringRefs &, Arena &)
|
|
||||||
{
|
|
||||||
}
|
|
||||||
|
|
||||||
static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool)
|
|
||||||
{
|
|
||||||
pool.rollback(key.size + keys.size() * sizeof(keys[0]));
|
|
||||||
}
|
|
||||||
|
|
||||||
/// If the key already was, then it is removed from the pool (overwritten), and the next key can not be compared with it.
|
|
||||||
static const bool no_consecutive_keys_optimization = true;
|
|
||||||
|
|
||||||
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, size_t keys_size, const Sizes & key_sizes)
|
|
||||||
{
|
|
||||||
insertKeyIntoColumnsImpl(value, key_columns, keys_size, key_sizes);
|
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
|
||||||
/// Insert the values of the specified keys into the corresponding columns.
|
|
||||||
static void insertKeyIntoColumnsImpl(const typename Data::value_type & value, MutableColumns & key_columns, size_t keys_size, const Sizes &)
|
|
||||||
{
|
|
||||||
/// See function extractKeysAndPlaceInPoolContiguous.
|
|
||||||
const StringRef * key_refs = reinterpret_cast<const StringRef *>(value.first.data + value.first.size);
|
|
||||||
|
|
||||||
if (unlikely(0 == value.first.size))
|
|
||||||
{
|
|
||||||
/** Fix if all keys are empty arrays. For them, a zero-length StringRef is written to the hash table, but with a non-zero pointer.
|
|
||||||
* But when inserted into a hash table, this StringRef occurs equal to another key of zero length,
|
|
||||||
* whose data pointer can be any garbage and can not be used.
|
|
||||||
*/
|
|
||||||
for (size_t i = 0; i < keys_size; ++i)
|
|
||||||
key_columns[i]->insertDefault();
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
for (size_t i = 0; i < keys_size; ++i)
|
|
||||||
key_columns[i]->insertDataWithTerminatingZero(key_refs[i].data, key_refs[i].size);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
|
|
||||||
/** Aggregates by concatenating serialized key values.
|
/** Aggregates by concatenating serialized key values.
|
||||||
* Similar to AggregationMethodConcat, but it is suitable, for example, for arrays of strings or multiple arrays.
|
|
||||||
* The serialized value differs in that it uniquely allows to deserialize it, having only the position with which it starts.
|
* The serialized value differs in that it uniquely allows to deserialize it, having only the position with which it starts.
|
||||||
* That is, for example, for strings, it contains first the serialized length of the string, and then the bytes.
|
* That is, for example, for strings, it contains first the serialized length of the string, and then the bytes.
|
||||||
* Therefore, when aggregating by several strings, there is no ambiguity.
|
* Therefore, when aggregating by several strings, there is no ambiguity.
|
||||||
@ -595,7 +512,7 @@ struct AggregationMethodSerialized
|
|||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
Key getKey(
|
ALWAYS_INLINE Key getKey(
|
||||||
const ColumnRawPtrs & key_columns,
|
const ColumnRawPtrs & key_columns,
|
||||||
size_t keys_size,
|
size_t keys_size,
|
||||||
size_t i,
|
size_t i,
|
||||||
@ -610,11 +527,11 @@ struct AggregationMethodSerialized
|
|||||||
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
||||||
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
||||||
|
|
||||||
static void onNewKey(typename Data::value_type &, size_t, StringRefs &, Arena &)
|
static ALWAYS_INLINE void onNewKey(typename Data::value_type &, size_t, StringRefs &, Arena &)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
static void onExistingKey(const Key & key, StringRefs &, Arena & pool)
|
static ALWAYS_INLINE void onExistingKey(const Key & key, StringRefs &, Arena & pool)
|
||||||
{
|
{
|
||||||
pool.rollback(key.size);
|
pool.rollback(key.size);
|
||||||
}
|
}
|
||||||
@ -631,61 +548,6 @@ struct AggregationMethodSerialized
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
/// For other cases. Aggregates by 128-bit hash from the key.
|
|
||||||
template <typename TData>
|
|
||||||
struct AggregationMethodHashed
|
|
||||||
{
|
|
||||||
using Data = TData;
|
|
||||||
using Key = typename Data::key_type;
|
|
||||||
using Mapped = typename Data::mapped_type;
|
|
||||||
using iterator = typename Data::iterator;
|
|
||||||
using const_iterator = typename Data::const_iterator;
|
|
||||||
|
|
||||||
Data data;
|
|
||||||
|
|
||||||
AggregationMethodHashed() {}
|
|
||||||
|
|
||||||
template <typename Other>
|
|
||||||
AggregationMethodHashed(const Other & other) : data(other.data) {}
|
|
||||||
|
|
||||||
struct State
|
|
||||||
{
|
|
||||||
void init(ColumnRawPtrs &)
|
|
||||||
{
|
|
||||||
}
|
|
||||||
|
|
||||||
Key getKey(
|
|
||||||
const ColumnRawPtrs & key_columns,
|
|
||||||
size_t keys_size,
|
|
||||||
size_t i,
|
|
||||||
const Sizes &,
|
|
||||||
StringRefs & keys,
|
|
||||||
Arena &) const
|
|
||||||
{
|
|
||||||
return hash128(i, keys_size, key_columns, keys);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
static AggregateDataPtr & getAggregateData(Mapped & value) { return value.second; }
|
|
||||||
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value.second; }
|
|
||||||
|
|
||||||
static void onNewKey(typename Data::value_type & value, size_t keys_size, StringRefs & keys, Arena & pool)
|
|
||||||
{
|
|
||||||
value.second.first = placeKeysInPool(keys_size, keys, pool);
|
|
||||||
}
|
|
||||||
|
|
||||||
static void onExistingKey(const Key &, StringRefs &, Arena &) {}
|
|
||||||
|
|
||||||
static const bool no_consecutive_keys_optimization = false;
|
|
||||||
|
|
||||||
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, size_t keys_size, const Sizes &)
|
|
||||||
{
|
|
||||||
for (size_t i = 0; i < keys_size; ++i)
|
|
||||||
key_columns[i]->insertDataWithTerminatingZero(value.second.first[i].data, value.second.first[i].size);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
|
|
||||||
class Aggregator;
|
class Aggregator;
|
||||||
|
|
||||||
struct AggregatedDataVariants : private boost::noncopyable
|
struct AggregatedDataVariants : private boost::noncopyable
|
||||||
@ -729,8 +591,6 @@ struct AggregatedDataVariants : private boost::noncopyable
|
|||||||
std::unique_ptr<AggregationMethodFixedString<AggregatedDataWithStringKey>> key_fixed_string;
|
std::unique_ptr<AggregationMethodFixedString<AggregatedDataWithStringKey>> key_fixed_string;
|
||||||
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128>> keys128;
|
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128>> keys128;
|
||||||
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256>> keys256;
|
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256>> keys256;
|
||||||
std::unique_ptr<AggregationMethodHashed<AggregatedDataHashed>> hashed;
|
|
||||||
std::unique_ptr<AggregationMethodConcat<AggregatedDataWithStringKey>> concat;
|
|
||||||
std::unique_ptr<AggregationMethodSerialized<AggregatedDataWithStringKey>> serialized;
|
std::unique_ptr<AggregationMethodSerialized<AggregatedDataWithStringKey>> serialized;
|
||||||
|
|
||||||
std::unique_ptr<AggregationMethodOneNumber<UInt32, AggregatedDataWithUInt64KeyTwoLevel>> key32_two_level;
|
std::unique_ptr<AggregationMethodOneNumber<UInt32, AggregatedDataWithUInt64KeyTwoLevel>> key32_two_level;
|
||||||
@ -739,8 +599,6 @@ struct AggregatedDataVariants : private boost::noncopyable
|
|||||||
std::unique_ptr<AggregationMethodFixedString<AggregatedDataWithStringKeyTwoLevel>> key_fixed_string_two_level;
|
std::unique_ptr<AggregationMethodFixedString<AggregatedDataWithStringKeyTwoLevel>> key_fixed_string_two_level;
|
||||||
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128TwoLevel>> keys128_two_level;
|
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128TwoLevel>> keys128_two_level;
|
||||||
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256TwoLevel>> keys256_two_level;
|
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256TwoLevel>> keys256_two_level;
|
||||||
std::unique_ptr<AggregationMethodHashed<AggregatedDataHashedTwoLevel>> hashed_two_level;
|
|
||||||
std::unique_ptr<AggregationMethodConcat<AggregatedDataWithStringKeyTwoLevel>> concat_two_level;
|
|
||||||
std::unique_ptr<AggregationMethodSerialized<AggregatedDataWithStringKeyTwoLevel>> serialized_two_level;
|
std::unique_ptr<AggregationMethodSerialized<AggregatedDataWithStringKeyTwoLevel>> serialized_two_level;
|
||||||
|
|
||||||
std::unique_ptr<AggregationMethodOneNumber<UInt64, AggregatedDataWithUInt64KeyHash64>> key64_hash64;
|
std::unique_ptr<AggregationMethodOneNumber<UInt64, AggregatedDataWithUInt64KeyHash64>> key64_hash64;
|
||||||
@ -748,7 +606,6 @@ struct AggregatedDataVariants : private boost::noncopyable
|
|||||||
std::unique_ptr<AggregationMethodFixedString<AggregatedDataWithStringKeyHash64>> key_fixed_string_hash64;
|
std::unique_ptr<AggregationMethodFixedString<AggregatedDataWithStringKeyHash64>> key_fixed_string_hash64;
|
||||||
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128Hash64>> keys128_hash64;
|
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128Hash64>> keys128_hash64;
|
||||||
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256Hash64>> keys256_hash64;
|
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256Hash64>> keys256_hash64;
|
||||||
std::unique_ptr<AggregationMethodConcat<AggregatedDataWithStringKeyHash64>> concat_hash64;
|
|
||||||
std::unique_ptr<AggregationMethodSerialized<AggregatedDataWithStringKeyHash64>> serialized_hash64;
|
std::unique_ptr<AggregationMethodSerialized<AggregatedDataWithStringKeyHash64>> serialized_hash64;
|
||||||
|
|
||||||
/// Support for nullable keys.
|
/// Support for nullable keys.
|
||||||
@ -767,8 +624,6 @@ struct AggregatedDataVariants : private boost::noncopyable
|
|||||||
M(key_fixed_string, false) \
|
M(key_fixed_string, false) \
|
||||||
M(keys128, false) \
|
M(keys128, false) \
|
||||||
M(keys256, false) \
|
M(keys256, false) \
|
||||||
M(hashed, false) \
|
|
||||||
M(concat, false) \
|
|
||||||
M(serialized, false) \
|
M(serialized, false) \
|
||||||
M(key32_two_level, true) \
|
M(key32_two_level, true) \
|
||||||
M(key64_two_level, true) \
|
M(key64_two_level, true) \
|
||||||
@ -776,15 +631,12 @@ struct AggregatedDataVariants : private boost::noncopyable
|
|||||||
M(key_fixed_string_two_level, true) \
|
M(key_fixed_string_two_level, true) \
|
||||||
M(keys128_two_level, true) \
|
M(keys128_two_level, true) \
|
||||||
M(keys256_two_level, true) \
|
M(keys256_two_level, true) \
|
||||||
M(hashed_two_level, true) \
|
|
||||||
M(concat_two_level, true) \
|
|
||||||
M(serialized_two_level, true) \
|
M(serialized_two_level, true) \
|
||||||
M(key64_hash64, false) \
|
M(key64_hash64, false) \
|
||||||
M(key_string_hash64, false) \
|
M(key_string_hash64, false) \
|
||||||
M(key_fixed_string_hash64, false) \
|
M(key_fixed_string_hash64, false) \
|
||||||
M(keys128_hash64, false) \
|
M(keys128_hash64, false) \
|
||||||
M(keys256_hash64, false) \
|
M(keys256_hash64, false) \
|
||||||
M(concat_hash64, false) \
|
|
||||||
M(serialized_hash64, false) \
|
M(serialized_hash64, false) \
|
||||||
M(nullable_keys128, false) \
|
M(nullable_keys128, false) \
|
||||||
M(nullable_keys256, false) \
|
M(nullable_keys256, false) \
|
||||||
@ -904,8 +756,6 @@ struct AggregatedDataVariants : private boost::noncopyable
|
|||||||
M(key_fixed_string) \
|
M(key_fixed_string) \
|
||||||
M(keys128) \
|
M(keys128) \
|
||||||
M(keys256) \
|
M(keys256) \
|
||||||
M(hashed) \
|
|
||||||
M(concat) \
|
|
||||||
M(serialized) \
|
M(serialized) \
|
||||||
M(nullable_keys128) \
|
M(nullable_keys128) \
|
||||||
M(nullable_keys256) \
|
M(nullable_keys256) \
|
||||||
@ -918,7 +768,6 @@ struct AggregatedDataVariants : private boost::noncopyable
|
|||||||
M(key_fixed_string_hash64) \
|
M(key_fixed_string_hash64) \
|
||||||
M(keys128_hash64) \
|
M(keys128_hash64) \
|
||||||
M(keys256_hash64) \
|
M(keys256_hash64) \
|
||||||
M(concat_hash64) \
|
|
||||||
M(serialized_hash64) \
|
M(serialized_hash64) \
|
||||||
|
|
||||||
#define APPLY_FOR_VARIANTS_SINGLE_LEVEL(M) \
|
#define APPLY_FOR_VARIANTS_SINGLE_LEVEL(M) \
|
||||||
@ -949,8 +798,6 @@ struct AggregatedDataVariants : private boost::noncopyable
|
|||||||
M(key_fixed_string_two_level) \
|
M(key_fixed_string_two_level) \
|
||||||
M(keys128_two_level) \
|
M(keys128_two_level) \
|
||||||
M(keys256_two_level) \
|
M(keys256_two_level) \
|
||||||
M(hashed_two_level) \
|
|
||||||
M(concat_two_level) \
|
|
||||||
M(serialized_two_level) \
|
M(serialized_two_level) \
|
||||||
M(nullable_keys128_two_level) \
|
M(nullable_keys128_two_level) \
|
||||||
M(nullable_keys256_two_level)
|
M(nullable_keys256_two_level)
|
||||||
|
Loading…
Reference in New Issue
Block a user