summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorHenning Baldersheim <balder@yahoo-inc.com>2021-03-10 21:36:43 +0100
committerGitHub <noreply@github.com>2021-03-10 21:36:43 +0100
commit95e943313575d8bc5f9b85b1b0d4ba61c0124ca6 (patch)
tree37fea09ee6ce4eeb8b994b0de40807deb83cbfd9
parent511afbfc58b08ce0e729511c8f2df8c81bfe37a6 (diff)
parentcbe17e3249a6f93a68d80eb6a9a78b65680c0122 (diff)
Merge pull request #16887 from vespa-engine/toregge/add-lock-free-hash-map
Add lock free hash map.
-rw-r--r--searchlib/src/vespa/searchlib/attribute/reference.h4
-rw-r--r--vespalib/CMakeLists.txt1
-rw-r--r--vespalib/src/tests/datastore/simple_hash_map/CMakeLists.txt9
-rw-r--r--vespalib/src/tests/datastore/simple_hash_map/simple_hash_map_test.cpp232
-rw-r--r--vespalib/src/tests/datastore/unique_store_dictionary/unique_store_dictionary_test.cpp3
-rw-r--r--vespalib/src/vespa/vespalib/datastore/CMakeLists.txt2
-rw-r--r--vespalib/src/vespa/vespalib/datastore/atomic_entry_ref.h3
-rw-r--r--vespalib/src/vespa/vespalib/datastore/entry_comparator.h1
-rw-r--r--vespalib/src/vespa/vespalib/datastore/fixed_size_hash_map.cpp174
-rw-r--r--vespalib/src/vespa/vespalib/datastore/fixed_size_hash_map.h119
-rw-r--r--vespalib/src/vespa/vespalib/datastore/simple_hash_map.cpp139
-rw-r--r--vespalib/src/vespa/vespalib/datastore/simple_hash_map.h57
-rw-r--r--vespalib/src/vespa/vespalib/datastore/unique_store_comparator.h19
-rw-r--r--vespalib/src/vespa/vespalib/datastore/unique_store_string_comparator.h5
14 files changed, 768 insertions, 0 deletions
diff --git a/searchlib/src/vespa/searchlib/attribute/reference.h b/searchlib/src/vespa/searchlib/attribute/reference.h
index 0df7edef5b2..426ce9ea314 100644
--- a/searchlib/src/vespa/searchlib/attribute/reference.h
+++ b/searchlib/src/vespa/searchlib/attribute/reference.h
@@ -40,6 +40,10 @@ public:
EntryRef revMapIdx() const { return _revMapIdx; }
void setLid(uint32_t targetLid) const { _lid = targetLid; }
void setRevMapIdx(EntryRef newRevMapIdx) const { _revMapIdx = newRevMapIdx; }
+ size_t hash() const noexcept {
+ GlobalId::hash hasher;
+ return hasher(_gid);
+ }
};
}
diff --git a/vespalib/CMakeLists.txt b/vespalib/CMakeLists.txt
index 09a03180fbf..cc82091568e 100644
--- a/vespalib/CMakeLists.txt
+++ b/vespalib/CMakeLists.txt
@@ -42,6 +42,7 @@ vespa_define_module(
src/tests/datastore/array_store_config
src/tests/datastore/buffer_type
src/tests/datastore/datastore
+ src/tests/datastore/simple_hash_map
src/tests/datastore/unique_store
src/tests/datastore/unique_store_dictionary
src/tests/datastore/unique_store_string_allocator
diff --git a/vespalib/src/tests/datastore/simple_hash_map/CMakeLists.txt b/vespalib/src/tests/datastore/simple_hash_map/CMakeLists.txt
new file mode 100644
index 00000000000..c790481ebbc
--- /dev/null
+++ b/vespalib/src/tests/datastore/simple_hash_map/CMakeLists.txt
@@ -0,0 +1,9 @@
+# Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+vespa_add_executable(vespalib_simple_hash_map_test_app
+ SOURCES
+ simple_hash_map_test.cpp
+ DEPENDS
+ vespalib
+ GTest::GTest
+)
+vespa_add_test(NAME vespalib_simple_hash_map_test_app COMMAND vespalib_simple_hash_map_test_app)
diff --git a/vespalib/src/tests/datastore/simple_hash_map/simple_hash_map_test.cpp b/vespalib/src/tests/datastore/simple_hash_map/simple_hash_map_test.cpp
new file mode 100644
index 00000000000..ad4ac1518b2
--- /dev/null
+++ b/vespalib/src/tests/datastore/simple_hash_map/simple_hash_map_test.cpp
@@ -0,0 +1,232 @@
+// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#include <vespa/vespalib/datastore/simple_hash_map.h>
+#include <vespa/vespalib/datastore/unique_store_allocator.h>
+#include <vespa/vespalib/datastore/unique_store_comparator.h>
+
+#include <vespa/vespalib/util/lambdatask.h>
+#include <vespa/vespalib/util/rand48.h>
+#include <vespa/vespalib/util/size_literals.h>
+#include <vespa/vespalib/util/threadstackexecutor.h>
+#include <vespa/vespalib/gtest/gtest.h>
+
+#include <vespa/vespalib/datastore/unique_store_allocator.hpp>
+
+#include <vespa/log/log.h>
+LOG_SETUP("vespalib_datastore_simple_hash_test");
+
+using vespalib::datastore::EntryRef;
+using RefT = vespalib::datastore::EntryRefT<22>;
+using MyAllocator = vespalib::datastore::UniqueStoreAllocator<uint32_t, RefT>;
+using MyDataStore = vespalib::datastore::DataStoreT<RefT>;
+using MyCompare = vespalib::datastore::UniqueStoreComparator<uint32_t, RefT>;
+using MyHashMap = vespalib::datastore::SimpleHashMap;
+using GenerationHandler = vespalib::GenerationHandler;
+using vespalib::makeLambdaTask;
+
+struct DataStoreSimpleHashTest : public ::testing::Test
+{
+ GenerationHandler _generationHandler;
+ MyAllocator _allocator;
+ MyDataStore& _store;
+ MyHashMap _hash_map;
+ vespalib::ThreadStackExecutor _writer; // 1 write thread
+ vespalib::ThreadStackExecutor _readers; // multiple reader threads
+ vespalib::Rand48 _rnd;
+ uint32_t _keyLimit;
+ std::atomic<long> _read_seed;
+ std::atomic<long> _done_write_work;
+ std::atomic<long> _done_read_work;
+ std::atomic<long> _found_count;
+ std::atomic<int> _stop_read;
+ bool _report_work;
+
+ DataStoreSimpleHashTest();
+ ~DataStoreSimpleHashTest();
+ void commit();
+ void insert(uint32_t key);
+ void remove(uint32_t key);
+
+ void read_work(uint32_t cnt);
+ void read_work();
+ void write_work(uint32_t cnt);
+};
+
+
+DataStoreSimpleHashTest::DataStoreSimpleHashTest()
+ : _generationHandler(),
+ _allocator(),
+ _store(_allocator.get_data_store()),
+ _hash_map(std::make_unique<MyCompare>(_store)),
+ _writer(1, 128_Ki),
+ _readers(4, 128_Ki),
+ _rnd(),
+ _keyLimit(1000000),
+ _read_seed(50),
+ _done_write_work(0),
+ _done_read_work(0),
+ _found_count(0),
+ _stop_read(0),
+ _report_work(false)
+{
+ _rnd.srand48(32);
+}
+
+
+DataStoreSimpleHashTest::~DataStoreSimpleHashTest()
+{
+ _readers.sync();
+ _readers.shutdown();
+ _writer.sync();
+ _writer.shutdown();
+ commit();
+ if (_report_work) {
+ LOG(info,
+ "read_work=%ld, write_work=%ld, found_count=%ld",
+ _done_read_work.load(), _done_write_work.load(), _found_count.load());
+ }
+}
+
+
+void
+DataStoreSimpleHashTest::commit()
+{
+ _store.transferHoldLists(_generationHandler.getCurrentGeneration());
+ _hash_map.transfer_hold_lists(_generationHandler.getCurrentGeneration());
+ _generationHandler.incGeneration();
+ _store.trimHoldLists(_generationHandler.getFirstUsedGeneration());
+ _hash_map.trim_hold_lists(_generationHandler.getFirstUsedGeneration());
+}
+
+void
+DataStoreSimpleHashTest::insert(uint32_t key)
+{
+ MyCompare comp(_store, key);
+std::function<EntryRef(void)> insert_entry([this, key]() -> EntryRef { return _allocator.allocate(key); });
+ auto& result = _hash_map.add(comp, insert_entry);
+ auto ref = result.first.load_relaxed();
+ auto &wrapped_entry = _allocator.get_wrapped(ref);
+ EXPECT_EQ(key, wrapped_entry.value());
+}
+
+void
+DataStoreSimpleHashTest::remove(uint32_t key)
+{
+ MyCompare comp(_store, key);
+ auto result = _hash_map.remove(comp, EntryRef());
+ if (result != nullptr) {
+ auto ref = result->first.load_relaxed();
+ auto &wrapped_entry = _allocator.get_wrapped(ref);
+ EXPECT_EQ(key, wrapped_entry.value());
+ _allocator.hold(ref);
+ }
+}
+
+
+void
+DataStoreSimpleHashTest::read_work(uint32_t cnt)
+{
+ vespalib::Rand48 rnd;
+ long found = 0;
+ rnd.srand48(++_read_seed);
+ uint32_t i;
+ for (i = 0; i < cnt && _stop_read.load() == 0; ++i) {
+ auto guard = _generationHandler.takeGuard();
+ uint32_t key = rnd.lrand48() % (_keyLimit + 1);
+ MyCompare comp(_store, key);
+ auto result = _hash_map.find(comp, EntryRef());
+ if (result != nullptr) {
+ auto ref = result->first.load_relaxed();
+ auto &wrapped_entry = _allocator.get_wrapped(ref);
+ EXPECT_EQ(key, wrapped_entry.value());
+ ++found;
+ }
+ }
+ _done_read_work += i;
+ _found_count += found;
+ LOG(info, "done %u read work", i);
+}
+
+
+void
+DataStoreSimpleHashTest::read_work()
+{
+ read_work(std::numeric_limits<uint32_t>::max());
+}
+
+
+void
+DataStoreSimpleHashTest::write_work(uint32_t cnt)
+{
+ vespalib::Rand48 &rnd(_rnd);
+ for (uint32_t i = 0; i < cnt; ++i) {
+ uint32_t key = rnd.lrand48() % _keyLimit;
+ if ((rnd.lrand48() & 1) == 0) {
+ insert(key);
+ } else {
+ remove(key);
+ }
+ commit();
+ }
+ _done_write_work += cnt;
+ _stop_read = 1;
+ LOG(info, "done %u write work", cnt);
+}
+
+
+TEST_F(DataStoreSimpleHashTest, smoke_test)
+{
+ EXPECT_EQ(0, _hash_map.size());
+ insert(1);
+ EXPECT_EQ(1, _hash_map.size());
+ remove(2);
+ EXPECT_EQ(1, _hash_map.size());
+ insert(1);
+ EXPECT_EQ(1, _hash_map.size());
+ insert(5);
+ EXPECT_EQ(2, _hash_map.size());
+ insert(4);
+ EXPECT_EQ(3, _hash_map.size());
+ remove(3);
+ EXPECT_EQ(3, _hash_map.size());
+ remove(5);
+ EXPECT_EQ(2, _hash_map.size());
+ commit();
+ MyCompare comp3(_store, 3);
+ auto result3 = _hash_map.find(comp3, EntryRef());
+ EXPECT_TRUE(result3 == nullptr);
+ MyCompare comp4(_store, 4);
+ auto result4 = _hash_map.find(comp4, EntryRef());
+ EXPECT_TRUE(result4 != nullptr);
+ auto ref4 = result4->first.load_relaxed();
+ auto& wrapped_entry4 = _allocator.get_wrapped(ref4);
+ EXPECT_EQ(4, wrapped_entry4.value());
+}
+
+TEST_F(DataStoreSimpleHashTest, single_threaded_reader_without_updates)
+{
+ _report_work = true;
+ write_work(10);
+ _stop_read = 0;
+ read_work(10);
+}
+
+TEST_F(DataStoreSimpleHashTest, single_threaded_reader_during_updates)
+{
+ uint32_t cnt = 1000000;
+ _report_work = true;
+ _writer.execute(makeLambdaTask([this, cnt]() { write_work(cnt); }));
+ _readers.execute(makeLambdaTask([this]() { read_work(); }));
+}
+
+TEST_F(DataStoreSimpleHashTest, multi_threaded_reader_during_updates)
+{
+ uint32_t cnt = 1000000;
+ _report_work = true;
+ _writer.execute(makeLambdaTask([this, cnt]() { write_work(cnt); }));
+ for (size_t i = 0; i < 4; ++i) {
+ _readers.execute(makeLambdaTask([this]() { read_work(); }));
+ }
+}
+
+GTEST_MAIN_RUN_ALL_TESTS()
diff --git a/vespalib/src/tests/datastore/unique_store_dictionary/unique_store_dictionary_test.cpp b/vespalib/src/tests/datastore/unique_store_dictionary/unique_store_dictionary_test.cpp
index 08b80917f45..71cb1864ce7 100644
--- a/vespalib/src/tests/datastore/unique_store_dictionary/unique_store_dictionary_test.cpp
+++ b/vespalib/src/tests/datastore/unique_store_dictionary/unique_store_dictionary_test.cpp
@@ -31,6 +31,9 @@ public:
bool equal(const EntryRef lhs, const EntryRef rhs) const override {
return resolve(lhs).ref() == resolve(rhs).ref();
}
+ size_t hash(const EntryRef rhs) const override {
+ return rhs.ref();
+ }
};
struct DictionaryReadTest : public ::testing::Test {
diff --git a/vespalib/src/vespa/vespalib/datastore/CMakeLists.txt b/vespalib/src/vespa/vespalib/datastore/CMakeLists.txt
index 16c5d3973e8..1ee945f1b8f 100644
--- a/vespalib/src/vespa/vespalib/datastore/CMakeLists.txt
+++ b/vespalib/src/vespa/vespalib/datastore/CMakeLists.txt
@@ -8,6 +8,8 @@ vespa_add_library(vespalib_vespalib_datastore OBJECT
datastore.cpp
datastorebase.cpp
entryref.cpp
+ fixed_size_hash_map.cpp
+ simple_hash_map.cpp
unique_store.cpp
unique_store_string_allocator.cpp
DEPENDS
diff --git a/vespalib/src/vespa/vespalib/datastore/atomic_entry_ref.h b/vespalib/src/vespa/vespalib/datastore/atomic_entry_ref.h
index ed45723b6ef..3ec2d6b163e 100644
--- a/vespalib/src/vespa/vespalib/datastore/atomic_entry_ref.h
+++ b/vespalib/src/vespa/vespalib/datastore/atomic_entry_ref.h
@@ -37,6 +37,9 @@ public:
EntryRef load_acquire() const noexcept {
return EntryRef(_ref.load(std::memory_order_acquire));
}
+ EntryRef load_relaxed() const noexcept {
+ return EntryRef(_ref.load(std::memory_order_relaxed));
+ }
};
}
diff --git a/vespalib/src/vespa/vespalib/datastore/entry_comparator.h b/vespalib/src/vespa/vespalib/datastore/entry_comparator.h
index 41ecf229d8f..bd0eb318b18 100644
--- a/vespalib/src/vespa/vespalib/datastore/entry_comparator.h
+++ b/vespalib/src/vespa/vespalib/datastore/entry_comparator.h
@@ -21,6 +21,7 @@ public:
*/
virtual bool less(const EntryRef lhs, const EntryRef rhs) const = 0;
virtual bool equal(const EntryRef lhs, const EntryRef rhs) const = 0;
+ virtual size_t hash(const EntryRef rhs) const = 0;
};
}
diff --git a/vespalib/src/vespa/vespalib/datastore/fixed_size_hash_map.cpp b/vespalib/src/vespa/vespalib/datastore/fixed_size_hash_map.cpp
new file mode 100644
index 00000000000..d852cd40b78
--- /dev/null
+++ b/vespalib/src/vespa/vespalib/datastore/fixed_size_hash_map.cpp
@@ -0,0 +1,174 @@
+// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#include "fixed_size_hash_map.h"
+#include "entry_comparator.h"
+#include <vespa/vespalib/util/array.hpp>
+#include <cassert>
+#include <stdexcept>
+
+namespace vespalib::datastore {
+
+FixedSizeHashMap::Node::Node(Node&&)
+{
+ throw std::runtime_error("vespalib::datastore::FixedSizeHashMap::Node move constructor should never be called");
+}
+
+void
+FixedSizeHashMap::Node::on_free()
+{
+ _kv = std::make_pair(AtomicEntryRef(), AtomicEntryRef());
+}
+
+FixedSizeHashMap::FixedSizeHashMap(uint32_t modulo, uint32_t capacity, uint32_t num_stripes)
+ : _chain_heads(modulo),
+ _nodes(),
+ _modulo(modulo),
+ _count(0u),
+ _free_head(no_node_idx),
+ _free_count(0u),
+ _hold_count(0u),
+ _hold_1_list(),
+ _hold_2_list(),
+ _num_stripes(num_stripes)
+{
+ _nodes.reserve(capacity);
+}
+
+FixedSizeHashMap::FixedSizeHashMap(uint32_t modulo, uint32_t capacity, uint32_t num_stripes, const FixedSizeHashMap &orig, const EntryComparator& comp)
+ : FixedSizeHashMap(modulo, capacity, num_stripes)
+{
+ for (const auto &chain_head : orig._chain_heads) {
+ for (uint32_t node_idx = chain_head.load_relaxed(); node_idx != no_node_idx;) {
+ auto& node = orig._nodes[node_idx];
+ force_add(comp, node.get_kv());
+ node_idx = node.get_next().load(std::memory_order_relaxed);
+ }
+ }
+}
+
+FixedSizeHashMap::~FixedSizeHashMap() = default;
+
+void
+FixedSizeHashMap::force_add(const EntryComparator& comp, const KvType& kv)
+{
+ size_t hash_idx = comp.hash(kv.first.load_relaxed()) / _num_stripes;
+ hash_idx %= _modulo;
+ auto& chain_head = _chain_heads[hash_idx];
+ assert(_nodes.size() < _nodes.capacity());
+ uint32_t node_idx = _nodes.size();
+ new (_nodes.push_back_fast()) Node(kv, chain_head.load_relaxed());
+ chain_head.set(node_idx);
+ ++_count;
+}
+
+FixedSizeHashMap::KvType&
+FixedSizeHashMap::add(const EntryComparator& comp, std::function<EntryRef(void)>& insert_entry)
+{
+ size_t hash_idx = comp.hash(EntryRef()) / _num_stripes;
+ hash_idx %= _modulo;
+ auto& chain_head = _chain_heads[hash_idx];
+ uint32_t node_idx = chain_head.load_relaxed();
+ while (node_idx != no_node_idx) {
+ auto& node = _nodes[node_idx];
+ if (comp.equal(EntryRef(), node.get_kv().first.load_relaxed())) {
+ return node.get_kv();
+ }
+ node_idx = node.get_next().load(std::memory_order_relaxed);
+ }
+ if (_free_head != no_node_idx) {
+ node_idx = _free_head;
+ auto& node = _nodes[node_idx];
+ _free_head = node.get_next().load(std::memory_order_relaxed);
+ --_free_count;
+ node.get_kv().first.store_release(insert_entry());
+ node.get_next().store(chain_head.load_relaxed());
+ chain_head.set(node_idx);
+ ++_count;
+ return node.get_kv();
+ }
+ assert(_nodes.size() < _nodes.capacity());
+ node_idx = _nodes.size();
+ new (_nodes.push_back_fast()) Node(std::make_pair(AtomicEntryRef(insert_entry()), AtomicEntryRef()), chain_head.load_relaxed());
+ chain_head.set(node_idx);
+ ++_count;
+ return _nodes[node_idx].get_kv();
+}
+
+void
+FixedSizeHashMap::transfer_hold_lists_slow(generation_t generation)
+{
+ auto &hold_2_list = _hold_2_list;
+ for (uint32_t node_idx : _hold_1_list) {
+ hold_2_list.push_back(std::make_pair(generation, node_idx));
+ }
+ _hold_1_list.clear();
+
+}
+
+
+void
+FixedSizeHashMap::trim_hold_lists_slow(generation_t usedGen)
+{
+ while (!_hold_2_list.empty()) {
+ auto& first = _hold_2_list.front();
+ if (static_cast<sgeneration_t>(first.first - usedGen) >= 0) {
+ break;
+ }
+ uint32_t node_idx = first.second;
+ auto& node = _nodes[node_idx];
+ node.get_next().store(_free_head, std::memory_order_relaxed);
+ _free_head = node_idx;
+ ++_free_count;
+ --_hold_count;
+ node.on_free();
+ _hold_2_list.erase(_hold_2_list.begin());
+ }
+}
+
+FixedSizeHashMap::KvType*
+FixedSizeHashMap::remove(const EntryComparator& comp, EntryRef key_ref)
+{
+ size_t hash_idx = comp.hash(key_ref) / _num_stripes;
+ hash_idx %= _modulo;
+ auto& chain_head = _chain_heads[hash_idx];
+ uint32_t node_idx = chain_head.load_relaxed();
+ uint32_t prev_node_idx = no_node_idx;
+ while (node_idx != no_node_idx) {
+ auto &node = _nodes[node_idx];
+ uint32_t next = node.get_next().load(std::memory_order_relaxed);
+ if (comp.equal(key_ref, node.get_kv().first.load_relaxed())) {
+ if (prev_node_idx != no_node_idx) {
+ _nodes[prev_node_idx].get_next().store(next, std::memory_order_release);
+ } else {
+ chain_head.set(next);
+ }
+ --_count;
+ ++_hold_count;
+ _hold_1_list.push_back(node_idx);
+ return &_nodes[node_idx].get_kv();
+ }
+ node_idx = next;
+ }
+ return nullptr;
+}
+
+const FixedSizeHashMap::KvType*
+FixedSizeHashMap::find(const EntryComparator& comp, EntryRef key_ref) const
+{
+ size_t hash_idx = comp.hash(key_ref) / _num_stripes;
+ hash_idx %= _modulo;
+ auto& chain_head = _chain_heads[hash_idx];
+ uint32_t node_idx = chain_head.load_acquire();
+ while (node_idx != no_node_idx) {
+ auto &node = _nodes[node_idx];
+ EntryRef node_key_ref = node.get_kv().first.load_acquire();
+ if (node_key_ref.valid() && comp.equal(key_ref, node_key_ref)) {
+ return &_nodes[node_idx].get_kv();
+ }
+ uint32_t next = node.get_next().load(std::memory_order_acquire);
+ node_idx = next;
+ }
+ return nullptr;
+}
+
+}
diff --git a/vespalib/src/vespa/vespalib/datastore/fixed_size_hash_map.h b/vespalib/src/vespa/vespalib/datastore/fixed_size_hash_map.h
new file mode 100644
index 00000000000..bafcf642a8d
--- /dev/null
+++ b/vespalib/src/vespa/vespalib/datastore/fixed_size_hash_map.h
@@ -0,0 +1,119 @@
+// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#pragma once
+
+#include "atomic_entry_ref.h"
+#include <vespa/vespalib/util/array.h>
+#include <vespa/vespalib/util/arrayref.h>
+#include <vespa/vespalib/util/generationhandler.h>
+#include <limits>
+#include <atomic>
+#include <deque>
+#include <functional>
+
+namespace vespalib { class GenerationHolder; }
+namespace vespalib::datastore {
+
+class EntryComparator;
+
+/*
+ * Fixed sized hash map over keys in data store, meant to support a faster
+ * dictionary for unique store with relation to lookups.
+ *
+ * Currently hardcoded key and data types, where key references an entry
+ * in a UniqueStore and value references a posting list
+ * (cf. search::attribute::PostingStore).
+ *
+ * This structure supports one writer and many readers.
+ *
+ * A reader must own an appropriate GenerationHandler::Guard to ensure
+ * that memory is held while it can be accessed by reader.
+ *
+ * The writer must update generation and call transfer_hold_lists and
+ * trim_hold_lists as needed to free up memory no longer needed by any
+ * readers.
+ */
+class FixedSizeHashMap {
+public:
+ static constexpr uint32_t no_node_idx = std::numeric_limits<uint32_t>::max();
+ using KvType = std::pair<AtomicEntryRef, AtomicEntryRef>;
+ using generation_t = GenerationHandler::generation_t;
+ using sgeneration_t = GenerationHandler::sgeneration_t;
+
+private:
+ class ChainHead {
+ std::atomic<uint32_t> _node_idx;
+
+ public:
+ ChainHead()
+ : _node_idx(no_node_idx)
+ {
+ }
+ // Writer thread
+ uint32_t load_relaxed() const noexcept { return _node_idx.load(std::memory_order_relaxed); }
+ void set(uint32_t node_idx) { _node_idx.store(node_idx, std::memory_order_release); }
+
+ // Reader thread
+ uint32_t load_acquire() const noexcept { return _node_idx.load(std::memory_order_acquire); }
+ };
+ class Node {
+ KvType _kv;
+ std::atomic<uint32_t> _next;
+ public:
+ Node()
+ : Node(std::make_pair(AtomicEntryRef(), AtomicEntryRef()), no_node_idx)
+ {
+ }
+ Node(KvType kv, uint32_t next)
+ : _kv(kv),
+ _next(next)
+ {
+ }
+ Node(Node &&rhs); // Must be defined, but must never be used.
+ void on_free();
+ std::atomic<uint32_t>& get_next() noexcept { return _next; }
+ const std::atomic<uint32_t>& get_next() const noexcept { return _next; }
+ KvType& get_kv() noexcept { return _kv; }
+ const KvType& get_kv() const noexcept { return _kv; }
+ };
+
+ Array<ChainHead> _chain_heads;
+ Array<Node> _nodes;
+ uint32_t _modulo;
+ uint32_t _count;
+ uint32_t _free_head;
+ uint32_t _free_count;
+ uint32_t _hold_count;
+ Array<uint32_t> _hold_1_list;
+ std::deque<std::pair<generation_t, uint32_t>> _hold_2_list;
+ uint32_t _num_stripes;
+
+ void transfer_hold_lists_slow(generation_t generation);
+ void trim_hold_lists_slow(generation_t usedGen);
+ void force_add(const EntryComparator& comp, const KvType& kv);
+public:
+ FixedSizeHashMap(uint32_t module, uint32_t capacity, uint32_t num_stripes);
+ FixedSizeHashMap(uint32_t module, uint32_t capacity, uint32_t num_stripes, const FixedSizeHashMap &orig, const EntryComparator& comp);
+ ~FixedSizeHashMap();
+
+ KvType& add(const EntryComparator& comp, std::function<EntryRef(void)>& insert_entry);
+ KvType* remove(const EntryComparator& comp, EntryRef key_ref);
+ const KvType* find(const EntryComparator& comp, EntryRef key_ref) const;
+
+ void transfer_hold_lists(generation_t generation) {
+ if (!_hold_1_list.empty()) {
+ transfer_hold_lists_slow(generation);
+ }
+ }
+
+ void trim_hold_lists(generation_t usedGen) {
+ if (!_hold_2_list.empty() && static_cast<sgeneration_t>(_hold_2_list.front().first - usedGen) < 0) {
+ trim_hold_lists_slow(usedGen);
+ }
+ }
+
+ bool full() const noexcept { return _nodes.size() == _nodes.capacity() && _free_count == 0u; }
+ size_t size() const noexcept { return _count; }
+};
+
+}
diff --git a/vespalib/src/vespa/vespalib/datastore/simple_hash_map.cpp b/vespalib/src/vespa/vespalib/datastore/simple_hash_map.cpp
new file mode 100644
index 00000000000..90e1bc60e06
--- /dev/null
+++ b/vespalib/src/vespa/vespalib/datastore/simple_hash_map.cpp
@@ -0,0 +1,139 @@
+// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#include "simple_hash_map.h"
+#include "fixed_size_hash_map.h"
+#include "entry_comparator.h"
+
+namespace vespalib::datastore {
+
+class SimpleHashMapStripeHeld : public GenerationHeldBase
+{
+ std::unique_ptr<const FixedSizeHashMap> _data;
+public:
+ SimpleHashMapStripeHeld(size_t size, std::unique_ptr<const FixedSizeHashMap> data);
+ ~SimpleHashMapStripeHeld();
+};
+
+SimpleHashMapStripeHeld::SimpleHashMapStripeHeld(size_t size, std::unique_ptr<const FixedSizeHashMap> data)
+ : GenerationHeldBase(size),
+ _data(std::move(data))
+{
+}
+
+SimpleHashMapStripeHeld::~SimpleHashMapStripeHeld() = default;
+
+SimpleHashMap::SimpleHashMap(std::unique_ptr<const EntryComparator> comp)
+ : _gen_holder(),
+ _maps(),
+ _comp(std::move(comp))
+{
+}
+
+SimpleHashMap::~SimpleHashMap()
+{
+ for (size_t i = 0; i < num_stripes; ++i) {
+ auto map = _maps[i].load(std::memory_order_relaxed);
+ delete map;
+ }
+}
+
+size_t
+SimpleHashMap::get_stripe(const EntryComparator& comp, EntryRef key_ref) const
+{
+ return comp.hash(key_ref) % num_stripes;
+}
+
+void
+SimpleHashMap::alloc_stripe(size_t stripe)
+{
+ auto map = _maps[stripe].load(std::memory_order_relaxed);
+ if (map == nullptr) {
+ auto umap = std::make_unique<FixedSizeHashMap>(2u, 3u, num_stripes);
+ _maps[stripe].store(umap.release(), std::memory_order_release);
+ } else {
+ auto umap = std::make_unique<FixedSizeHashMap>(map->size() * 2 + 2, map->size() * 3 + 3, num_stripes, *map, *_comp);
+ _maps[stripe].store(umap.release(), std::memory_order_release);
+ hold_stripe(std::unique_ptr<const FixedSizeHashMap>(map));
+ }
+}
+
+void
+SimpleHashMap::hold_stripe(std::unique_ptr<const FixedSizeHashMap> map)
+{
+ // TODO: Provider proper held size
+ auto hold = std::make_unique<SimpleHashMapStripeHeld>(0, std::move(map));
+ _gen_holder.hold(std::move(hold));
+}
+
+SimpleHashMap::KvType&
+SimpleHashMap::add(const EntryComparator& comp, std::function<EntryRef(void)>& insert_entry)
+{
+ size_t stripe = get_stripe(comp, EntryRef());
+ auto map = _maps[stripe].load(std::memory_order_relaxed);
+ if (map == nullptr || map->full()) {
+ alloc_stripe(stripe);
+ map = _maps[stripe].load(std::memory_order_relaxed);
+ }
+ return map->add(comp, insert_entry);
+}
+
+SimpleHashMap::KvType*
+SimpleHashMap::remove(const EntryComparator& comp, EntryRef key_ref)
+{
+ size_t stripe = get_stripe(comp, key_ref);
+ auto map = _maps[stripe].load(std::memory_order_relaxed);
+ if (map == nullptr) {
+ return nullptr;
+ }
+ return map->remove(comp, key_ref);
+}
+
+const SimpleHashMap::KvType*
+SimpleHashMap::find(const EntryComparator& comp, EntryRef key_ref) const
+{
+ size_t stripe = get_stripe(comp, key_ref);
+ auto map = _maps[stripe].load(std::memory_order_relaxed);
+ if (map == nullptr) {
+ return nullptr;
+ }
+ return map->find(comp, key_ref);
+}
+
+void
+SimpleHashMap::transfer_hold_lists(generation_t generation)
+{
+ for (size_t i = 0; i < num_stripes; ++i) {
+ auto map = _maps[i].load(std::memory_order_relaxed);
+ if (map != nullptr) {
+ map->transfer_hold_lists(generation);
+ }
+ }
+ _gen_holder.transferHoldLists(generation);
+}
+
+void
+SimpleHashMap::trim_hold_lists(generation_t used_gen)
+{
+ for (size_t i = 0; i < num_stripes; ++i) {
+ auto map = _maps[i].load(std::memory_order_relaxed);
+ if (map != nullptr) {
+ map->trim_hold_lists(used_gen);
+ }
+ }
+ _gen_holder.trimHoldLists(used_gen);
+}
+
+size_t
+SimpleHashMap::size() const noexcept
+{
+ size_t result = 0;
+ for (size_t i = 0; i < num_stripes; ++i) {
+ auto map = _maps[i].load(std::memory_order_relaxed);
+ if (map != nullptr) {
+ result += map->size();
+ }
+ }
+ return result;
+}
+
+}
diff --git a/vespalib/src/vespa/vespalib/datastore/simple_hash_map.h b/vespalib/src/vespa/vespalib/datastore/simple_hash_map.h
new file mode 100644
index 00000000000..506c1a3ea3f
--- /dev/null
+++ b/vespalib/src/vespa/vespalib/datastore/simple_hash_map.h
@@ -0,0 +1,57 @@
+// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#pragma once
+
+#include "atomic_entry_ref.h"
+#include <atomic>
+#include <vespa/vespalib/util/generationholder.h>
+#include <functional>
+
+namespace vespalib::datastore {
+
+class FixedSizeHashMap;
+class EntryComparator;
+
+/*
+ * Hash map over keys in data store, meant to support a faster
+ * dictionary for unique store with relation to lookups.
+ *
+ * Currently hardcoded key and data types, where key references an entry
+ * in a UniqueStore and value references a posting list
+ * (cf. search::attribute::PostingStore).
+ *
+ * This structure supports one writer and many readers.
+ *
+ * A reader must own an appropriate GenerationHandler::Guard to ensure
+ * that memory is held while it can be accessed by reader.
+ *
+ * The writer must update generation and call transfer_hold_lists and
+ * trim_hold_lists as needed to free up memory no longer needed by any
+ * readers.
+ */
+class SimpleHashMap {
+public:
+ using KvType = std::pair<AtomicEntryRef, AtomicEntryRef>;
+ using generation_t = GenerationHandler::generation_t;
+ using sgeneration_t = GenerationHandler::sgeneration_t;
+private:
+ GenerationHolder _gen_holder;
+ static constexpr size_t num_stripes = 1;
+ std::atomic<FixedSizeHashMap *> _maps[num_stripes];
+ std::unique_ptr<const EntryComparator> _comp;
+
+ size_t get_stripe(const EntryComparator& comp, EntryRef key_ref) const;
+ void alloc_stripe(size_t stripe);
+ void hold_stripe(std::unique_ptr<const FixedSizeHashMap> map);
+public:
+ SimpleHashMap(std::unique_ptr<const EntryComparator> comp);
+ ~SimpleHashMap();
+ KvType& add(const EntryComparator& comp, std::function<EntryRef(void)> &insert_entry);
+ KvType* remove(const EntryComparator& comp, EntryRef key_ref);
+ const KvType* find(const EntryComparator& comp, EntryRef key_ref) const;
+ void transfer_hold_lists(generation_t generation);
+ void trim_hold_lists(generation_t used_gen);
+ size_t size() const noexcept;
+};
+
+}
diff --git a/vespalib/src/vespa/vespalib/datastore/unique_store_comparator.h b/vespalib/src/vespa/vespalib/datastore/unique_store_comparator.h
index 630c53e7b08..5280b8712e5 100644
--- a/vespalib/src/vespa/vespalib/datastore/unique_store_comparator.h
+++ b/vespalib/src/vespa/vespalib/datastore/unique_store_comparator.h
@@ -5,6 +5,7 @@
#include "entry_comparator.h"
#include "unique_store_entry.h"
#include "datastore.h"
+#include <vespa/vespalib/stllike/hash_fun.h>
#include <cmath>
namespace vespalib::datastore {
@@ -21,6 +22,10 @@ public:
static bool equal(const EntryT& lhs, const EntryT& rhs) {
return lhs == rhs;
}
+ static size_t hash(const EntryT& rhs) {
+ vespalib::hash<EntryT> hasher;
+ return hasher(rhs);
+ }
};
/**
@@ -49,6 +54,16 @@ public:
return (lhs == rhs);
}
}
+ static size_t hash(EntryT rhs) {
+ if (std::isnan(rhs)) {
+ return 0;
+ } else {
+ union U { EntryT f; std::conditional_t<std::is_same_v<double, EntryT>, uint64_t, uint32_t> i; };
+ U t;
+ t.f = rhs;
+ return t.i;
+ }
+ }
};
/**
@@ -115,6 +130,10 @@ public:
const EntryType &rhsValue = get(rhs);
return UniqueStoreComparatorHelper<EntryT>::equal(lhsValue, rhsValue);
}
+ size_t hash(const EntryRef rhs) const override {
+ const EntryType &rhsValue = get(rhs);
+ return UniqueStoreComparatorHelper<EntryT>::hash(rhsValue);
+ }
};
}
diff --git a/vespalib/src/vespa/vespalib/datastore/unique_store_string_comparator.h b/vespalib/src/vespa/vespalib/datastore/unique_store_string_comparator.h
index 0be0e3e8d9d..9acacc0073f 100644
--- a/vespalib/src/vespa/vespalib/datastore/unique_store_string_comparator.h
+++ b/vespalib/src/vespa/vespalib/datastore/unique_store_string_comparator.h
@@ -54,6 +54,11 @@ public:
const char *rhs_value = get(rhs);
return (strcmp(lhs_value, rhs_value) == 0);
}
+ size_t hash(const EntryRef rhs) const override {
+ const char *rhs_value = get(rhs);
+ vespalib::hash<const char *> hasher;
+ return hasher(rhs_value);
+ }
};
}