Added LRUHashMap

This commit is contained in:
Maksim Kita 2021-02-07 00:06:27 +03:00
parent 342baa4223
commit eadd23aa38
6 changed files with 747 additions and 13 deletions

View File

@ -109,6 +109,11 @@ struct HashMapCell
DB::assertChar(',', rb);
DB::readDoubleQuoted(value.second, rb);
}
static bool constexpr need_to_notify_cell_during_move = false;
static void move(HashMapCell * /* old_location */, HashMapCell * /* new_location */) {}
};
template <typename Key, typename TMapped, typename Hash, typename TState = HashTableNoState>

View File

@ -204,6 +204,13 @@ struct HashTableCell
/// Deserialization, in binary and text form.
void read(DB::ReadBuffer & rb) { DB::readBinary(key, rb); }
void readText(DB::ReadBuffer & rb) { DB::readDoubleQuoted(key, rb); }
/// When cell pointer is moved during erase, reinsert or resize operations
static constexpr bool need_to_notify_cell_during_move = false;
static void move(HashTableCell * /* old_location */, HashTableCell * /* new_location */) {}
};
/**
@ -427,6 +434,34 @@ protected:
}
}
template<bool enable>
struct AllocatorBufferDeleter;
template<>
struct AllocatorBufferDeleter<false>
{
AllocatorBufferDeleter(Allocator &, size_t) {}
void operator()(Cell *) const {}
};
template<>
struct AllocatorBufferDeleter<true>
{
AllocatorBufferDeleter(Allocator & allocator_, size_t size_)
: allocator(allocator_)
, size(size_)
{}
void operator()(Cell * buffer) const
{
allocator.free(buffer, size);
}
Allocator & allocator;
size_t size;
};
/// Increase the size of the buffer.
void resize(size_t for_num_elems = 0, size_t for_buf_size = 0)
@ -460,7 +495,23 @@ protected:
new_grower.increaseSize();
/// Expand the space.
buf = reinterpret_cast<Cell *>(Allocator::realloc(buf, getBufferSizeInBytes(), new_grower.bufSize() * sizeof(Cell)));
size_t old_buffer_size = getBufferSizeInBytes();
/** If cell required to be notified during move we need to temporary keep old buffer
* because realloc does not quarantee for reallocated buffer to have same base address
*/
AllocatorBufferDeleter<Cell::need_to_notify_cell_during_move> buffer_deleter(*this, old_buffer_size);
std::unique_ptr<Cell, decltype(buffer_deleter)> old_buffer(buf, buffer_deleter);
if constexpr (Cell::need_to_notify_cell_during_move)
{
buf = reinterpret_cast<Cell *>(Allocator::alloc(new_grower.bufSize() * sizeof(Cell)));
memcpy(buf, old_buffer.get(), old_buffer_size);
}
else
buf = reinterpret_cast<Cell *>(Allocator::realloc(buf, old_buffer_size, new_grower.bufSize() * sizeof(Cell)));
grower = new_grower;
/** Now some items may need to be moved to a new location.
@ -470,7 +521,12 @@ protected:
size_t i = 0;
for (; i < old_size; ++i)
if (!buf[i].isZero(*this))
reinsert(buf[i], buf[i].getHash(*this));
{
size_t updated_place_value = reinsert(buf[i], buf[i].getHash(*this));
if constexpr (Cell::need_to_notify_cell_during_move)
Cell::move(&(old_buffer.get())[i], &buf[updated_place_value]);
}
/** There is also a special case:
* if the element was to be at the end of the old buffer, [ x]
@ -481,7 +537,13 @@ protected:
* process tail from the collision resolution chain immediately after it [ o x ]
*/
for (; !buf[i].isZero(*this); ++i)
reinsert(buf[i], buf[i].getHash(*this));
{
size_t updated_place_value = reinsert(buf[i], buf[i].getHash(*this));
if constexpr (Cell::need_to_notify_cell_during_move)
if (&buf[i] != &buf[updated_place_value])
Cell::move(&buf[i], &buf[updated_place_value]);
}
#ifdef DBMS_HASH_MAP_DEBUG_RESIZES
watch.stop();
@ -495,20 +557,20 @@ protected:
/** Paste into the new buffer the value that was in the old buffer.
* Used when increasing the buffer size.
*/
void reinsert(Cell & x, size_t hash_value)
size_t reinsert(Cell & x, size_t hash_value)
{
size_t place_value = grower.place(hash_value);
/// If the element is in its place.
if (&x == &buf[place_value])
return;
return place_value;
/// Compute a new location, taking into account the collision resolution chain.
place_value = findCell(Cell::getKey(x.getValue()), hash_value, place_value);
/// If the item remains in its place in the old collision resolution chain.
if (!buf[place_value].isZero(*this))
return;
return place_value;
/// Copy to a new location and zero the old one.
x.setHash(hash_value);
@ -516,6 +578,7 @@ protected:
x.setZero();
/// Then the elements that previously were in collision with this can move to the old place.
return place_value;
}
@ -881,7 +944,11 @@ public:
/// Reinsert node pointed to by iterator
void ALWAYS_INLINE reinsert(iterator & it, size_t hash_value)
{
reinsert(*it.getPtr(), hash_value);
size_t place_value = reinsert(*it.getPtr(), hash_value);
if constexpr (Cell::need_to_notify_cell_during_move)
if (it.getPtr() != &buf[place_value])
Cell::move(it.getPtr(), &buf[place_value]);
}
@ -958,8 +1025,14 @@ public:
return const_cast<std::decay_t<decltype(*this)> *>(this)->find(x, hash_value);
}
std::enable_if_t<Grower::performs_linear_probing_with_single_step, void>
std::enable_if_t<Grower::performs_linear_probing_with_single_step, bool>
ALWAYS_INLINE erase(const Key & x)
{
return erase(x, hash(x));
}
std::enable_if_t<Grower::performs_linear_probing_with_single_step, bool>
ALWAYS_INLINE erase(const Key & x, size_t hash_value)
{
/** Deletion from open addressing hash table without tombstones
*
@ -977,21 +1050,19 @@ public:
{
--m_size;
this->clearHasZero();
return true;
}
else
{
return;
return false;
}
}
size_t hash_value = hash(x);
size_t erased_key_position = findCell(x, hash_value, grower.place(hash_value));
/// Key is not found
if (buf[erased_key_position].isZero(*this))
{
return;
}
return false;
/// We need to guarantee loop termination because there will be empty position
assert(m_size < grower.bufSize());
@ -1056,12 +1127,18 @@ public:
/// Move the element to the freed place
memcpy(static_cast<void *>(&buf[erased_key_position]), static_cast<void *>(&buf[next_position]), sizeof(Cell));
if constexpr (Cell::need_to_notify_cell_during_move)
Cell::move(&buf[next_position], &buf[erased_key_position]);
/// Now we have another freed place
erased_key_position = next_position;
}
buf[erased_key_position].setZero();
--m_size;
return true;
}
bool ALWAYS_INLINE has(const Key & x) const

View File

@ -0,0 +1,244 @@
#pragma once
#include <common/types.h>
#include <boost/intrusive/trivial_value_traits.hpp>
#include <boost/intrusive/list.hpp>
#include <boost/noncopyable.hpp>
#include <Core/Defines.h>
#include <Common/Exception.h>
#include <Common/HashTable/HashMap.h>
#include <Common/PODArray.h>
template <typename TKey, typename TMapped, typename Hash, bool save_hash_in_cell>
struct LRUHashMapCell :
public std::conditional_t<save_hash_in_cell,
HashMapCellWithSavedHash<TKey, TMapped, Hash, HashTableNoState>,
HashMapCell<TKey, TMapped, Hash, HashTableNoState>>
{
public:
using Key = TKey;
using Base = std::conditional_t<save_hash_in_cell,
HashMapCellWithSavedHash<TKey, TMapped, Hash, HashTableNoState>,
HashMapCell<TKey, TMapped, Hash, HashTableNoState>>;
using Mapped = typename Base::Mapped;
using State = typename Base::State;
using mapped_type = Mapped;
using key_type = Key;
using Base::Base;
static bool constexpr need_to_notify_cell_during_move = true;
static void move(LRUHashMapCell * __restrict old_location, LRUHashMapCell * __restrict new_location)
{
/** We update new location prev and next pointers because during hash table resize
* they can be updated during move of another cell.
*/
new_location->prev = old_location->prev;
new_location->next = old_location->next;
LRUHashMapCell * prev = new_location->prev;
LRUHashMapCell * next = new_location->next;
/// Updated previous next and next previous nodes of list to point to new location
if (prev)
prev->next = new_location;
if (next)
next->prev = new_location;
}
private:
template<typename, typename, typename, bool>
friend class LRUHashMapCellNodeTraits;
LRUHashMapCell * next = nullptr;
LRUHashMapCell * prev = nullptr;
};
template<typename Key, typename Value, typename Hash, bool save_hash_in_cell>
struct LRUHashMapCellNodeTraits
{
using node = LRUHashMapCell<Key, Value, Hash, save_hash_in_cell>;
using node_ptr = LRUHashMapCell<Key, Value, Hash, save_hash_in_cell> *;
using const_node_ptr = const LRUHashMapCell<Key, Value, Hash, save_hash_in_cell> *;
static node * get_next(const node * ptr) { return ptr->next; }
static void set_next(node * __restrict ptr, node * __restrict next) { ptr->next = next; }
static node * get_previous(const node * ptr) { return ptr->prev; }
static void set_previous(node * __restrict ptr, node * __restrict prev) { ptr->prev = prev; }
};
template <typename TKey, typename TValue, typename Hash, bool save_hash_in_cells>
class LRUHashMapImpl :
private HashMapTable<
TKey,
LRUHashMapCell<TKey, TValue, Hash, save_hash_in_cells>,
Hash,
HashTableGrower<>,
HashTableAllocator>
{
using Base = HashMapTable<
TKey,
LRUHashMapCell<TKey, TValue, Hash, save_hash_in_cells>,
Hash,
HashTableGrower<>,
HashTableAllocator>;
public:
using Key = TKey;
using Value = TValue;
using Cell = LRUHashMapCell<Key, Value, Hash, save_hash_in_cells>;
using LRUHashMapCellIntrusiveValueTraits =
boost::intrusive::trivial_value_traits<
LRUHashMapCellNodeTraits<Key, Value, Hash, save_hash_in_cells>,
boost::intrusive::link_mode_type::normal_link>;
using LRUList = boost::intrusive::list<
Cell,
boost::intrusive::value_traits<LRUHashMapCellIntrusiveValueTraits>,
boost::intrusive::constant_time_size<false>>;
using iterator = typename LRUList::iterator;
using const_iterator = typename LRUList::const_iterator;
using reverse_iterator = typename LRUList::reverse_iterator;
using const_reverse_iterator = typename LRUList::const_reverse_iterator;
LRUHashMapImpl(size_t max_size_, bool preallocate_max_size_in_hash_map = false)
: Base(preallocate_max_size_in_hash_map ? max_size_ : 32)
, max_size(max_size_)
{
assert(max_size > 0);
}
std::pair<Cell *, bool> insert(const Key & key, const Value & value)
{
return emplace(key, value);
}
std::pair<Cell *, bool> insert(const Key & key, Value && value)
{
return emplace(key, std::move(value));
}
template<typename ...Args>
std::pair<Cell *, bool> emplace(const Key & key, Args&&... args)
{
size_t hash_value = Base::hash(key);
Cell * it = Base::find(key, hash_value);
if (it)
{
/// Cell contains element return it and put to the end of lru list
lru_list.splice(lru_list.end(), lru_list, lru_list.iterator_to(*it));
return std::make_pair(it, false);
}
if (size() == max_size)
{
/// Erase least recently used element from front of the list
Cell & node = lru_list.front();
const Key & element_to_remove_key = node.getKey();
size_t key_hash = node.getHash(*this);
lru_list.pop_front();
[[maybe_unused]] bool erased = Base::erase(element_to_remove_key, key_hash);
assert(erased);
}
[[maybe_unused]] bool inserted;
/// Insert value first try to insert in zero storage if not then insert in buffer
if (!Base::emplaceIfZero(key, it, inserted, hash_value))
Base::emplaceNonZero(key, it, inserted, hash_value);
assert(inserted);
new (&it->getMapped()) Value(std::forward<Args>(args)...);
/// Put cell to the end of lru list
lru_list.insert(lru_list.end(), *it);
return std::make_pair(it, true);
}
using Base::find;
Value & get(const Key & key)
{
auto it = Base::find(key);
assert(it);
Value & value = it->getMapped();
/// Put cell to the end of lru list
lru_list.splice(lru_list.end(), lru_list, lru_list.iterator_to(*it));
return value;
}
const Value & get(const Key & key) const
{
return const_cast<std::decay_t<decltype(*this)> *>(this)->get(key);
}
bool contains(const Key & key) const
{
return Base::has(key);
}
bool erase(const Key & key)
{
auto hash = Base::hash(key);
auto it = Base::find(key, hash);
if (!it)
return false;
lru_list.erase(lru_list.iterator_to(*it));
return Base::erase(key, hash);
}
void clear()
{
lru_list.clear();
Base::clear();
}
using Base::size;
size_t getMaxSize() const { return max_size; }
iterator begin() { return lru_list.begin(); }
const_iterator begin() const { return lru_list.cbegin(); }
iterator end() { return lru_list.end(); }
const_iterator end() const { return lru_list.cend(); }
reverse_iterator rbegin() { return lru_list.rbegin(); }
const_reverse_iterator rbegin() const { return lru_list.crbegin(); }
reverse_iterator rend() { return lru_list.rend(); }
const_reverse_iterator rend() const { return lru_list.crend(); }
private:
size_t max_size;
LRUList lru_list;
};
template <typename Key, typename Value, typename Hash = DefaultHash<Key>>
using LRUHashMap = LRUHashMapImpl<Key, Value, Hash, false>;
template <typename Key, typename Value, typename Hash = DefaultHash<Key>>
using LRUHashMapWithSavedHash = LRUHashMapImpl<Key, Value, Hash, true>;

View File

@ -38,6 +38,9 @@ target_link_libraries (arena_with_free_lists PRIVATE dbms)
add_executable (pod_array pod_array.cpp)
target_link_libraries (pod_array PRIVATE clickhouse_common_io)
add_executable (lru_hash_map_perf lru_hash_map_perf.cpp)
target_link_libraries (lru_hash_map_perf PRIVATE clickhouse_common_io)
add_executable (thread_creation_latency thread_creation_latency.cpp)
target_link_libraries (thread_creation_latency PRIVATE clickhouse_common_io)

View File

@ -0,0 +1,161 @@
#include <iomanip>
#include <iostream>
#include <Common/HashTable/LRUHashMap.h>
#include <gtest/gtest.h>
template<typename LRUHashMap>
std::vector<typename LRUHashMap::Key> convertToVector(const LRUHashMap & map)
{
std::vector<typename LRUHashMap::Key> result;
result.reserve(map.size());
for (auto & node: map)
result.emplace_back(node.getKey());
return result;
}
void testInsert(size_t elements_to_insert_size, size_t map_size)
{
using LRUHashMap = LRUHashMap<int, std::string>;
LRUHashMap map(map_size);
std::vector<int> expected;
for (size_t i = 0; i < elements_to_insert_size; ++i)
map.insert(i, std::to_string(i));
for (size_t i = elements_to_insert_size - map_size; i < elements_to_insert_size; ++i)
expected.emplace_back(i);
std::vector<int> actual = convertToVector(map);
ASSERT_EQ(map.size(), actual.size());
ASSERT_EQ(actual, expected);
}
TEST(LRUHashMap, Insert)
{
{
using LRUHashMap = LRUHashMap<int, std::string>;
LRUHashMap map(3);
map.emplace(1, "1");
map.insert(2, "2");
std::string v = "3";
map.insert(3, v);
map.emplace(4, "4");
std::vector<int> expected = { 2, 3, 4 };
std::vector<int> actual = convertToVector(map);
ASSERT_EQ(actual, expected);
}
testInsert(1200000, 1200000);
testInsert(10, 5);
testInsert(1200000, 2);
testInsert(1200000, 1);
}
TEST(LRUHashMap, GetModify)
{
using LRUHashMap = LRUHashMap<int, std::string>;
LRUHashMap map(3);
map.emplace(1, "1");
map.emplace(2, "2");
map.emplace(3, "3");
map.get(3) = "4";
std::vector<std::string> expected = { "1", "2", "4" };
std::vector<std::string> actual;
actual.reserve(map.size());
for (auto & node : map)
actual.emplace_back(node.getMapped());
ASSERT_EQ(actual, expected);
}
TEST(LRUHashMap, SetRecentKeyToTop)
{
using LRUHashMap = LRUHashMap<int, std::string>;
LRUHashMap map(3);
map.emplace(1, "1");
map.emplace(2, "2");
map.emplace(3, "3");
map.emplace(1, "4");
std::vector<int> expected = { 2, 3, 1 };
std::vector<int> actual = convertToVector(map);
ASSERT_EQ(actual, expected);
}
TEST(LRUHashMap, GetRecentKeyToTop)
{
using LRUHashMap = LRUHashMap<int, std::string>;
LRUHashMap map(3);
map.emplace(1, "1");
map.emplace(2, "2");
map.emplace(3, "3");
map.get(1);
std::vector<int> expected = { 2, 3, 1 };
std::vector<int> actual = convertToVector(map);
ASSERT_EQ(actual, expected);
}
TEST(LRUHashMap, Contains)
{
using LRUHashMap = LRUHashMap<int, std::string>;
LRUHashMap map(3);
map.emplace(1, "1");
map.emplace(2, "2");
map.emplace(3, "3");
ASSERT_TRUE(map.contains(1));
ASSERT_TRUE(map.contains(2));
ASSERT_TRUE(map.contains(3));
ASSERT_EQ(map.size(), 3);
map.erase(1);
map.erase(2);
map.erase(3);
ASSERT_EQ(map.size(), 0);
ASSERT_FALSE(map.contains(1));
ASSERT_FALSE(map.contains(2));
ASSERT_FALSE(map.contains(3));
}
TEST(LRUHashMap, Clear)
{
using LRUHashMap = LRUHashMap<int, std::string>;
LRUHashMap map(3);
map.emplace(1, "1");
map.emplace(2, "2");
map.emplace(3, "3");
map.clear();
std::vector<int> expected = {};
std::vector<int> actual = convertToVector(map);
ASSERT_EQ(actual, expected);
ASSERT_EQ(map.size(), 0);
}

View File

@ -0,0 +1,244 @@
#include <vector>
#include <list>
#include <map>
#include <random>
#include <pcg_random.hpp>
#include <Common/Stopwatch.h>
#include <Common/HashTable/LRUHashMap.h>
template<class Key, class Value>
class LRUHashMapBasic
{
public:
using key_type = Key;
using value_type = Value;
using list_type = std::list<key_type>;
using node = std::pair<value_type, typename list_type::iterator>;
using map_type = std::unordered_map<key_type, node, DefaultHash<Key>>;
LRUHashMapBasic(size_t max_size_, bool preallocated)
: hash_map(preallocated ? max_size_ : 32)
, max_size(max_size_)
{
}
void insert(const Key &key, const Value &value)
{
auto it = hash_map.find(key);
if (it == hash_map.end())
{
if (size() >= max_size)
{
auto iterator_to_remove = list.begin();
hash_map.erase(*iterator_to_remove);
list.erase(iterator_to_remove);
}
list.push_back(key);
hash_map[key] = std::make_pair(value, --list.end());
}
else
{
auto & [value_to_update, iterator_in_list_to_update] = it->second;
list.splice(list.end(), list, iterator_in_list_to_update);
iterator_in_list_to_update = list.end();
value_to_update = value;
}
}
value_type & get(const key_type &key)
{
auto iterator_in_map = hash_map.find(key);
assert(iterator_in_map != hash_map.end());
auto & [value_to_return, iterator_in_list_to_update] = iterator_in_map->second;
list.splice(list.end(), list, iterator_in_list_to_update);
iterator_in_list_to_update = list.end();
return value_to_return;
}
const value_type & get(const key_type & key) const
{
return const_cast<std::decay_t<decltype(*this)> *>(this)->get(key);
}
size_t getMaxSize() const
{
return max_size;
}
size_t size() const
{
return hash_map.size();
}
bool empty() const
{
return hash_map.empty();
}
bool contains(const Key & key)
{
return hash_map.find(key) != hash_map.end();
}
void clear()
{
hash_map.clear();
list.clear();
}
private:
map_type hash_map;
list_type list;
size_t max_size;
};
std::vector<UInt64> generateNumbersToInsert(size_t numbers_to_insert_size)
{
std::vector<UInt64> numbers;
numbers.reserve(numbers_to_insert_size);
std::random_device rd;
pcg64 gen(rd());
UInt64 min = std::numeric_limits<UInt64>::min();
UInt64 max = std::numeric_limits<UInt64>::max();
auto distribution = std::uniform_int_distribution<>(min, max);
for (size_t i = 0; i < numbers_to_insert_size; ++i)
{
UInt64 number = distribution(gen);
numbers.emplace_back(number);
}
return numbers;
}
void testInsertElementsIntoHashMap(size_t map_size, const std::vector<UInt64> & numbers_to_insert, bool preallocated)
{
size_t numbers_to_insert_size = numbers_to_insert.size();
std::cout << "TestInsertElementsIntoHashMap preallocated map size: " << map_size << " numbers to insert size: " << numbers_to_insert_size;
std::cout << std::endl;
HashMap<int, int> hash_map(preallocated ? map_size : 32);
Stopwatch watch;
for (size_t i = 0; i < numbers_to_insert_size; ++i)
hash_map.insert({ numbers_to_insert[i], numbers_to_insert[i] });
std::cout << "Inserted in " << watch.elapsedMilliseconds() << " milliseconds" << std::endl;
UInt64 summ = 0;
for (size_t i = 0; i < numbers_to_insert_size; ++i)
{
auto it = hash_map.find(numbers_to_insert[i]);
if (it)
summ += it->getMapped();
}
std::cout << "Calculated summ: " << summ << " in " << watch.elapsedMilliseconds() << " milliseconds" << std::endl;
}
void testInsertElementsIntoStandardMap(size_t map_size, const std::vector<UInt64> & numbers_to_insert, bool preallocated)
{
size_t numbers_to_insert_size = numbers_to_insert.size();
std::cout << "TestInsertElementsIntoStandardMap map size: " << map_size << " numbers to insert size: " << numbers_to_insert_size;
std::cout << std::endl;
std::unordered_map<int, int> hash_map(preallocated ? map_size : 32);
Stopwatch watch;
for (size_t i = 0; i < numbers_to_insert_size; ++i)
hash_map.insert({ numbers_to_insert[i], numbers_to_insert[i] });
std::cout << "Inserted in " << watch.elapsedMilliseconds() << " milliseconds" << std::endl;
UInt64 summ = 0;
for (size_t i = 0; i < numbers_to_insert_size; ++i)
{
auto it = hash_map.find(numbers_to_insert[i]);
if (it != hash_map.end())
summ += it->second;
}
std::cout << "Calculated summ: " << summ << " in " << watch.elapsedMilliseconds() << " milliseconds" << std::endl;
}
template<typename LRUCache>
UInt64 testInsertIntoEmptyCache(size_t map_size, const std::vector<UInt64> & numbers_to_insert, bool preallocated)
{
size_t numbers_to_insert_size = numbers_to_insert.size();
std::cout << "Test testInsertPreallocated preallocated map size: " << map_size << " numbers to insert size: " << numbers_to_insert_size;
std::cout << std::endl;
LRUCache cache(map_size, preallocated);
Stopwatch watch;
for (size_t i = 0; i < numbers_to_insert_size; ++i)
{
cache.insert(numbers_to_insert[i], numbers_to_insert[i]);
}
std::cout << "Inserted in " << watch.elapsedMilliseconds() << " milliseconds" << std::endl;
UInt64 summ = 0;
for (size_t i = 0; i < numbers_to_insert_size; ++i)
if (cache.contains(numbers_to_insert[i]))
summ += cache.get(numbers_to_insert[i]);
std::cout << "Calculated summ: " << summ << " in " << watch.elapsedMilliseconds() << " milliseconds" << std::endl;
return summ;
}
int main(int argc, char ** argv)
{
(void)(argc);
(void)(argv);
size_t hash_map_size = 1200000;
size_t numbers_to_insert_size = 12000000;
std::vector<UInt64> numbers = generateNumbersToInsert(numbers_to_insert_size);
std::cout << "Test insert into HashMap preallocated=0" << std::endl;
testInsertElementsIntoHashMap(hash_map_size, numbers, true);
std::cout << std::endl;
std::cout << "Test insert into HashMap preallocated=1" << std::endl;
testInsertElementsIntoHashMap(hash_map_size, numbers, true);
std::cout << std::endl;
std::cout << "Test LRUHashMap preallocated=0" << std::endl;
testInsertIntoEmptyCache<LRUHashMap<UInt64, UInt64>>(hash_map_size, numbers, false);
std::cout << std::endl;
std::cout << "Test LRUHashMap preallocated=1" << std::endl;
testInsertIntoEmptyCache<LRUHashMap<UInt64, UInt64>>(hash_map_size, numbers, true);
std::cout << std::endl;
std::cout << "Test LRUHashMapBasic preallocated=0" << std::endl;
testInsertIntoEmptyCache<LRUHashMapBasic<UInt64, UInt64>>(hash_map_size, numbers, false);
std::cout << std::endl;
std::cout << "Test LRUHashMapBasic preallocated=1" << std::endl;
testInsertIntoEmptyCache<LRUHashMapBasic<UInt64, UInt64>>(hash_map_size, numbers, true);
std::cout << std::endl;
return 0;
}