Assert that allocator & container are consistent wrt. inline memory

There are lots of places where HashTable is used with
AllocatorWithStackMemory, but the size of allocator inline memory is set
incorrectly, and it's less than the initial HashTable buffer size.
Because of this, HashTable is always allocated on the heap, and the
inline memory becomes a useless dead weight.

For PODArray, we have previously added a helper template that makes sure
these values are in sync, so there was only one such discrepancy left,
in the unit test.
This commit is contained in:
Alexander Kuzmenkov 2020-05-15 19:23:31 +03:00
parent c3d9bb829b
commit 50df893dc6
16 changed files with 145 additions and 113 deletions

View File

@ -23,18 +23,12 @@ struct EntropyData
{ {
using Weight = UInt64; using Weight = UInt64;
using HashingMap = HashMap< using HashingMap = HASH_TABLE_WITH_STACK_MEMORY(HashMap, Value, Weight,
Value, Weight, HashCRC32<Value>, HashTableGrower<4>);
HashCRC32<Value>,
HashTableGrower<4>,
HashTableAllocatorWithStackMemory<sizeof(std::pair<Value, Weight>) * (1 << 3)>>;
/// For the case of pre-hashed values. /// For the case of pre-hashed values.
using TrivialMap = HashMap< using TrivialMap = HASH_TABLE_WITH_STACK_MEMORY(HashMap, Value, Weight,
Value, Weight, UInt128TrivialHash, HashTableGrower<4>);
UInt128TrivialHash,
HashTableGrower<4>,
HashTableAllocatorWithStackMemory<sizeof(std::pair<Value, Weight>) * (1 << 3)>>;
using Map = std::conditional_t<std::is_same_v<UInt128, Value>, TrivialMap, HashingMap>; using Map = std::conditional_t<std::is_same_v<UInt128, Value>, TrivialMap, HashingMap>;

View File

@ -28,12 +28,8 @@ template <typename T>
struct AggregateFunctionGroupUniqArrayData struct AggregateFunctionGroupUniqArrayData
{ {
/// When creating, the hash table must be small. /// When creating, the hash table must be small.
using Set = HashSet< using Set = HASH_TABLE_WITH_STACK_MEMORY(HashSet, T, DefaultHash<T>,
T, HashTableGrower<4>);
DefaultHash<T>,
HashTableGrower<4>,
HashTableAllocatorWithStackMemory<sizeof(T) * (1 << 4)>
>;
Set value; Set value;
}; };
@ -126,9 +122,10 @@ public:
/// Generic implementation, it uses serialized representation as object descriptor. /// Generic implementation, it uses serialized representation as object descriptor.
struct AggregateFunctionGroupUniqArrayGenericData struct AggregateFunctionGroupUniqArrayGenericData
{ {
static constexpr size_t INIT_ELEMS = 2; /// adjustable static constexpr size_t INITIAL_SIZE_DEGREE = 1; /// adjustable
static constexpr size_t ELEM_SIZE = sizeof(HashSetCellWithSavedHash<StringRef, StringRefHash>);
using Set = HashSetWithSavedHash<StringRef, StringRefHash, HashTableGrower<INIT_ELEMS>, HashTableAllocatorWithStackMemory<INIT_ELEMS * ELEM_SIZE>>; using Set = HASH_TABLE_WITH_STACK_MEMORY(HashSetWithSavedHash,
StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>);
Set value; Set value;
}; };

View File

@ -23,13 +23,8 @@ namespace DB
template <typename T> template <typename T>
struct AggregateFunctionTopKData struct AggregateFunctionTopKData
{ {
using Set = SpaceSaving using Set = SpaceSaving<T, HashCRC32<T>>;
<
T,
HashCRC32<T>,
HashTableGrower<4>,
HashTableAllocatorWithStackMemory<sizeof(T) * (1 << 4)>
>;
Set value; Set value;
}; };
@ -109,13 +104,7 @@ public:
/// Generic implementation, it uses serialized representation as object descriptor. /// Generic implementation, it uses serialized representation as object descriptor.
struct AggregateFunctionTopKGenericData struct AggregateFunctionTopKGenericData
{ {
using Set = SpaceSaving using Set = SpaceSaving<StringRef, StringRefHash>;
<
StringRef,
StringRefHash,
HashTableGrower<4>,
HashTableAllocatorWithStackMemory<sizeof(StringRef) * (1 << 4)>
>;
Set value; Set value;
}; };

View File

@ -33,12 +33,8 @@ struct QuantileExactWeighted
using Hasher = std::conditional_t<std::is_same_v<Value, Decimal128>, Int128Hash, HashCRC32<UnderlyingType>>; using Hasher = std::conditional_t<std::is_same_v<Value, Decimal128>, Int128Hash, HashCRC32<UnderlyingType>>;
/// When creating, the hash table must be small. /// When creating, the hash table must be small.
using Map = HashMap< using Map = HASH_TABLE_WITH_STACK_MEMORY(HashMap, UnderlyingType, Weight,
UnderlyingType, Weight, Hasher, HashTableGrower<4>);
Hasher,
HashTableGrower<4>,
HashTableAllocatorWithStackMemory<sizeof(std::pair<Value, Weight>) * (1 << 3)>
>;
Map map; Map map;

View File

@ -278,13 +278,15 @@ private:
/** Allocator with optimization to place small memory ranges in automatic memory. /** Allocator with optimization to place small memory ranges in automatic memory.
*/ */
template <typename Base, size_t N, size_t Alignment> template <typename Base, size_t _initial_bytes, size_t Alignment>
class AllocatorWithStackMemory : private Base class AllocatorWithStackMemory : private Base
{ {
private: private:
alignas(Alignment) char stack_memory[N]; alignas(Alignment) char stack_memory[_initial_bytes];
public: public:
static constexpr size_t initial_bytes = _initial_bytes;
/// Do not use boost::noncopyable to avoid the warning about direct base /// Do not use boost::noncopyable to avoid the warning about direct base
/// being inaccessible due to ambiguity, when derived classes are also /// being inaccessible due to ambiguity, when derived classes are also
/// noncopiable (-Winaccessible-base). /// noncopiable (-Winaccessible-base).
@ -295,10 +297,10 @@ public:
void * alloc(size_t size) void * alloc(size_t size)
{ {
if (size <= N) if (size <= initial_bytes)
{ {
if constexpr (Base::clear_memory) if constexpr (Base::clear_memory)
memset(stack_memory, 0, N); memset(stack_memory, 0, initial_bytes);
return stack_memory; return stack_memory;
} }
@ -307,18 +309,18 @@ public:
void free(void * buf, size_t size) void free(void * buf, size_t size)
{ {
if (size > N) if (size > initial_bytes)
Base::free(buf, size); Base::free(buf, size);
} }
void * realloc(void * buf, size_t old_size, size_t new_size) void * realloc(void * buf, size_t old_size, size_t new_size)
{ {
/// Was in stack_memory, will remain there. /// Was in stack_memory, will remain there.
if (new_size <= N) if (new_size <= initial_bytes)
return buf; return buf;
/// Already was big enough to not fit in stack_memory. /// Already was big enough to not fit in stack_memory.
if (old_size > N) if (old_size > initial_bytes)
return Base::realloc(buf, old_size, new_size, Alignment); return Base::realloc(buf, old_size, new_size, Alignment);
/// Was in stack memory, but now will not fit there. /// Was in stack memory, but now will not fit there.
@ -330,10 +332,20 @@ public:
protected: protected:
static constexpr size_t getStackThreshold() static constexpr size_t getStackThreshold()
{ {
return N; return initial_bytes;
} }
}; };
// A constant that gives the number of initially available bytes in
// the allocator. Used to check that this number is in sync with the
// initial size of array or hash table that uses the allocator.
template<typename TAllocator>
constexpr size_t allocatorInitialBytes = 0;
template<typename Base, size_t initial_bytes, size_t Alignment>
constexpr size_t allocatorInitialBytes<AllocatorWithStackMemory<
Base, initial_bytes, Alignment>> = initial_bytes;
#if !__clang__ #if !__clang__
#pragma GCC diagnostic pop #pragma GCC diagnostic pop

View File

@ -308,7 +308,16 @@ struct ZeroValueStorage<false, Cell>
const Cell * zeroValue() const { return nullptr; } const Cell * zeroValue() const { return nullptr; }
}; };
// These templates give the initial hash table size, so that we can check
// that it is in sync with initial allocator size.
template <typename GenericGrower>
constexpr size_t growerInitialCount = 0;
template <size_t initial_size_degree>
constexpr size_t growerInitialCount<HashTableGrower<initial_size_degree>>
= 1ULL << initial_size_degree;
// The HashTable
template template
< <
typename Key, typename Key,
@ -324,6 +333,17 @@ class HashTable :
protected Cell::State, protected Cell::State,
protected ZeroValueStorage<Cell::need_zero_value_storage, Cell> /// empty base optimization protected ZeroValueStorage<Cell::need_zero_value_storage, Cell> /// empty base optimization
{ {
public:
// Export the initial buffer sizes for the ease of using allocators with
// inline memory.
static constexpr size_t initial_buffer_bytes
= growerInitialCount<Grower> * sizeof(Cell);
// If we use an allocator with inline memory, check that the initial
// size of the hash table is in sync with the amount of this memory.
static_assert(allocatorInitialBytes<Allocator> == 0
|| allocatorInitialBytes<Allocator> == initial_buffer_bytes);
protected: protected:
friend class const_iterator; friend class const_iterator;
friend class iterator; friend class iterator;
@ -1075,3 +1095,13 @@ public:
} }
#endif #endif
}; };
// A helper macro that declares hash table with allocator with stack memory,
// and the initial size of the allocator is in sync with initial size of the
// hash table.
#define HASH_TABLE_WITH_STACK_MEMORY(HASH_TABLE_VARIANT, ...) \
HASH_TABLE_VARIANT<__VA_ARGS__, \
HashTableAllocatorWithStackMemory< \
HASH_TABLE_VARIANT<__VA_ARGS__>::initial_buffer_bytes \
> \
>

View File

@ -10,5 +10,5 @@
*/ */
using HashTableAllocator = Allocator<true /* clear_memory */, true /* mmap_populate */>; using HashTableAllocator = Allocator<true /* clear_memory */, true /* mmap_populate */>;
template <size_t N = 64> template <size_t initial_bytes = 64>
using HashTableAllocatorWithStackMemory = AllocatorWithStackMemory<HashTableAllocator, N>; using HashTableAllocatorWithStackMemory = AllocatorWithStackMemory<HashTableAllocator, initial_bytes>;

View File

@ -85,6 +85,11 @@ protected:
static_assert(pad_left <= EmptyPODArraySize && "Left Padding exceeds EmptyPODArraySize. Is the element size too large?"); static_assert(pad_left <= EmptyPODArraySize && "Left Padding exceeds EmptyPODArraySize. Is the element size too large?");
// If we are using allocator with inline memory, the minimal size of
// array must be in sync with the size of this memory.
static_assert(allocatorInitialBytes<TAllocator> == 0
|| allocatorInitialBytes<TAllocator> == initial_bytes);
char * c_start = null; /// Does not include pad_left. char * c_start = null; /// Does not include pad_left.
char * c_end = null; char * c_end = null;
char * c_end_of_storage = null; /// Does not include pad_right. char * c_end_of_storage = null; /// Does not include pad_right.

View File

@ -67,9 +67,7 @@ private:
template template
< <
typename TKey, typename TKey,
typename Hash = DefaultHash<TKey>, typename Hash = DefaultHash<TKey>
typename Grower = HashTableGrower<>,
typename Allocator = HashTableAllocator
> >
class SpaceSaving class SpaceSaving
{ {
@ -380,7 +378,8 @@ private:
counter_map[counter->key] = counter; counter_map[counter->key] = counter;
} }
using CounterMap = HashMap<TKey, Counter *, Hash, Grower, Allocator>; using CounterMap = HASH_TABLE_WITH_STACK_MEMORY(HashMap,
TKey, Counter *, Hash, HashTableGrower<4>);
CounterMap counter_map; CounterMap counter_map;
std::vector<Counter *> counter_list; std::vector<Counter *> counter_list;

View File

@ -18,9 +18,9 @@ static void test1()
{ {
using namespace DB; using namespace DB;
static constexpr size_t initial_size = 8; static constexpr size_t initial_bytes = 32;
static constexpr size_t stack_threshold = 32; using Array = PODArray<UInt64, initial_bytes,
using Array = PODArray<UInt64, initial_size, AllocatorWithStackMemory<Allocator<false>, stack_threshold>>; AllocatorWithStackMemory<Allocator<false>, initial_bytes>>;
bool res = true; bool res = true;
@ -139,9 +139,9 @@ static void test2()
{ {
using namespace DB; using namespace DB;
static constexpr size_t initial_size = 8; static constexpr size_t initial_bytes = 32;
static constexpr size_t stack_threshold = 32; using Array = PODArray<UInt64, initial_bytes,
using Array = PODArray<UInt64, initial_size, AllocatorWithStackMemory<Allocator<false>, stack_threshold>>; AllocatorWithStackMemory<Allocator<false>, initial_bytes>>;
bool res = true; bool res = true;
@ -389,9 +389,9 @@ static void test3()
{ {
using namespace DB; using namespace DB;
static constexpr size_t initial_size = 8; static constexpr size_t initial_bytes = 32;
static constexpr size_t stack_threshold = 32; using Array = PODArray<UInt64, initial_bytes,
using Array = PODArray<UInt64, initial_size, AllocatorWithStackMemory<Allocator<false>, stack_threshold>>; AllocatorWithStackMemory<Allocator<false>, initial_bytes>>;
bool res = true; bool res = true;

View File

@ -153,10 +153,8 @@ bool FunctionArrayDistinct::executeNumber(
if (nullable_col) if (nullable_col)
src_null_map = &nullable_col->getNullMapData(); src_null_map = &nullable_col->getNullMapData();
using Set = ClearableHashSet<T, using Set = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashSet,
DefaultHash<T>, T, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>);
HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
Set set; Set set;
@ -201,10 +199,8 @@ bool FunctionArrayDistinct::executeString(
ColumnString & res_data_column_string = typeid_cast<ColumnString &>(res_data_col); ColumnString & res_data_column_string = typeid_cast<ColumnString &>(res_data_col);
using Set = ClearableHashSet<StringRef, using Set = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashSet,
StringRefHash, StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>);
HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
const PaddedPODArray<UInt8> * src_null_map = nullptr; const PaddedPODArray<UInt8> * src_null_map = nullptr;
@ -249,8 +245,8 @@ void FunctionArrayDistinct::executeHashed(
ColumnArray::Offsets & res_offsets, ColumnArray::Offsets & res_offsets,
const ColumnNullable * nullable_col) const ColumnNullable * nullable_col)
{ {
using Set = ClearableHashSet<UInt128, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>, using Set = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashSet,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; UInt128, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>);
const PaddedPODArray<UInt8> * src_null_map = nullptr; const PaddedPODArray<UInt8> * src_null_map = nullptr;

View File

@ -64,36 +64,46 @@ private:
template <typename T> template <typename T>
struct MethodOneNumber struct MethodOneNumber
{ {
using Set = ClearableHashMap<T, UInt32, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>, using Set = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashMap,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; T, UInt32, DefaultHash<T>,
HashTableGrower<INITIAL_SIZE_DEGREE>);
using Method = ColumnsHashing::HashMethodOneNumber<typename Set::value_type, UInt32, T, false>; using Method = ColumnsHashing::HashMethodOneNumber<typename Set::value_type, UInt32, T, false>;
}; };
struct MethodString struct MethodString
{ {
using Set = ClearableHashMap<StringRef, UInt32, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>, using Set = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashMap,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; StringRef, UInt32, StringRefHash,
HashTableGrower<INITIAL_SIZE_DEGREE>);
using Method = ColumnsHashing::HashMethodString<typename Set::value_type, UInt32, false, false>; using Method = ColumnsHashing::HashMethodString<typename Set::value_type, UInt32, false, false>;
}; };
struct MethodFixedString struct MethodFixedString
{ {
using Set = ClearableHashMap<StringRef, UInt32, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>, using Set = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashMap,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; StringRef, UInt32, StringRefHash,
HashTableGrower<INITIAL_SIZE_DEGREE>);
using Method = ColumnsHashing::HashMethodFixedString<typename Set::value_type, UInt32, false, false>; using Method = ColumnsHashing::HashMethodFixedString<typename Set::value_type, UInt32, false, false>;
}; };
struct MethodFixed struct MethodFixed
{ {
using Set = ClearableHashMap<UInt128, UInt32, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>, using Set = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashMap,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; UInt128, UInt32, UInt128HashCRC32,
HashTableGrower<INITIAL_SIZE_DEGREE>);
using Method = ColumnsHashing::HashMethodKeysFixed<typename Set::value_type, UInt128, UInt32, false, false, false>; using Method = ColumnsHashing::HashMethodKeysFixed<typename Set::value_type, UInt128, UInt32, false, false, false>;
}; };
struct MethodHashed struct MethodHashed
{ {
using Set = ClearableHashMap<UInt128, UInt32, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>, using Set = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashMap,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; UInt128, UInt32, UInt128TrivialHash,
HashTableGrower<INITIAL_SIZE_DEGREE>);
using Method = ColumnsHashing::HashMethodHashed<typename Set::value_type, UInt32, false>; using Method = ColumnsHashing::HashMethodHashed<typename Set::value_type, UInt32, false>;
}; };

View File

@ -308,12 +308,10 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeMethodImpl(
const size_t depth_to_look = arrays_depths.max_array_depth; const size_t depth_to_look = arrays_depths.max_array_depth;
const auto & offsets = *offsets_by_depth[depth_to_look - 1]; const auto & offsets = *offsets_by_depth[depth_to_look - 1];
using Map = ClearableHashMap< using Map = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashMap,
UInt128, UInt128, UInt32, UInt128TrivialHash,
UInt32, HashTableGrower<INITIAL_SIZE_DEGREE>);
UInt128TrivialHash,
HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
Map indices; Map indices;
std::vector<size_t> indices_by_depth(depth_to_look); std::vector<size_t> indices_by_depth(depth_to_look);

View File

@ -418,16 +418,18 @@ void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & ar
TypeListNativeNumbers::forEach(NumberExecutor(arrays, not_nullable_nested_return_type, result_column)); TypeListNativeNumbers::forEach(NumberExecutor(arrays, not_nullable_nested_return_type, result_column));
TypeListDecimalNumbers::forEach(DecimalExecutor(arrays, not_nullable_nested_return_type, result_column)); TypeListDecimalNumbers::forEach(DecimalExecutor(arrays, not_nullable_nested_return_type, result_column));
using DateMap = ClearableHashMap<DataTypeDate::FieldType, size_t, DefaultHash<DataTypeDate::FieldType>, using DateMap = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashMap,
HashTableGrower<INITIAL_SIZE_DEGREE>, DataTypeDate::FieldType, size_t, DefaultHash<DataTypeDate::FieldType>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(DataTypeDate::FieldType)>>; HashTableGrower<INITIAL_SIZE_DEGREE>);
using DateTimeMap = ClearableHashMap<DataTypeDateTime::FieldType, size_t, DefaultHash<DataTypeDateTime::FieldType>, using DateTimeMap = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashMap,
HashTableGrower<INITIAL_SIZE_DEGREE>, DataTypeDateTime::FieldType, size_t,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(DataTypeDateTime::FieldType)>>; DefaultHash<DataTypeDateTime::FieldType>,
HashTableGrower<INITIAL_SIZE_DEGREE>);
using StringMap = ClearableHashMap<StringRef, size_t, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>, using StringMap = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashMap,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; StringRef, size_t, StringRefHash,
HashTableGrower<INITIAL_SIZE_DEGREE>);
if (!result_column) if (!result_column)
{ {
@ -455,8 +457,8 @@ void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & ar
template <typename T, size_t> template <typename T, size_t>
void FunctionArrayIntersect::NumberExecutor::operator()() void FunctionArrayIntersect::NumberExecutor::operator()()
{ {
using Map = ClearableHashMap<T, size_t, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>, using Map = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashMap,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; T, size_t, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>);
if (!result && typeid_cast<const DataTypeNumber<T> *>(data_type.get())) if (!result && typeid_cast<const DataTypeNumber<T> *>(data_type.get()))
result = execute<Map, ColumnVector<T>, true>(arrays, ColumnVector<T>::create()); result = execute<Map, ColumnVector<T>, true>(arrays, ColumnVector<T>::create());
@ -465,8 +467,8 @@ void FunctionArrayIntersect::NumberExecutor::operator()()
template <typename T, size_t> template <typename T, size_t>
void FunctionArrayIntersect::DecimalExecutor::operator()() void FunctionArrayIntersect::DecimalExecutor::operator()()
{ {
using Map = ClearableHashMap<T, size_t, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>, using Map = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashMap,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; T, size_t, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>);
if (!result) if (!result)
if (auto * decimal = typeid_cast<const DataTypeDecimal<T> *>(data_type.get())) if (auto * decimal = typeid_cast<const DataTypeDecimal<T> *>(data_type.get()))

View File

@ -66,36 +66,41 @@ private:
template <typename T> template <typename T>
struct MethodOneNumber struct MethodOneNumber
{ {
using Set = ClearableHashSet<T, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>, using Set = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashSet,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; T, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>);
using Method = ColumnsHashing::HashMethodOneNumber<typename Set::value_type, void, T, false>; using Method = ColumnsHashing::HashMethodOneNumber<typename Set::value_type, void, T, false>;
}; };
struct MethodString struct MethodString
{ {
using Set = ClearableHashSet<StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>, using Set = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashSet,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>);
using Method = ColumnsHashing::HashMethodString<typename Set::value_type, void, false, false>; using Method = ColumnsHashing::HashMethodString<typename Set::value_type, void, false, false>;
}; };
struct MethodFixedString struct MethodFixedString
{ {
using Set = ClearableHashSet<StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>, using Set = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashSet,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>);
using Method = ColumnsHashing::HashMethodFixedString<typename Set::value_type, void, false, false>; using Method = ColumnsHashing::HashMethodFixedString<typename Set::value_type, void, false, false>;
}; };
struct MethodFixed struct MethodFixed
{ {
using Set = ClearableHashSet<UInt128, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>, using Set = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashSet,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; UInt128, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>);
using Method = ColumnsHashing::HashMethodKeysFixed<typename Set::value_type, UInt128, void, false, false, false>; using Method = ColumnsHashing::HashMethodKeysFixed<typename Set::value_type, UInt128, void, false, false, false>;
}; };
struct MethodHashed struct MethodHashed
{ {
using Set = ClearableHashSet<UInt128, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>, using Set = HASH_TABLE_WITH_STACK_MEMORY(ClearableHashSet,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; UInt128, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>);
using Method = ColumnsHashing::HashMethodHashed<typename Set::value_type, void, false>; using Method = ColumnsHashing::HashMethodHashed<typename Set::value_type, void, false>;
}; };

View File

@ -61,12 +61,11 @@ struct Grower : public HashTableGrower<2>
int main(int, char **) int main(int, char **)
{ {
using Map = HashMapWithDump< using Map = HASH_TABLE_WITH_STACK_MEMORY(HashMapWithDump,
StringRef, StringRef,
UInt64, UInt64,
SimpleHash, SimpleHash,
Grower, Grower);
HashTableAllocatorWithStackMemory<4 * 24>>;
Map map; Map map;