summaryrefslogtreecommitdiffstats
path: root/searchlib
diff options
context:
space:
mode:
authorTor Egge <Tor.Egge@online.no>2024-02-14 19:05:23 +0100
committerTor Egge <Tor.Egge@online.no>2024-02-14 19:05:23 +0100
commitab47f5e966b7a4a3f0c6410852790cc5d7743b79 (patch)
treed57586fca257048e69f9a75540e97034965efe5a /searchlib
parent5c5a0a43b2eee22b621ea17dfff39691abc28be5 (diff)
Add search::predicate::SimpleIndexSaver.
Remove extra logging when saving predicate attribute.
Diffstat (limited to 'searchlib')
-rw-r--r--searchlib/src/tests/predicate/predicate_index_test.cpp6
-rw-r--r--searchlib/src/tests/predicate/simple_index_test.cpp16
-rw-r--r--searchlib/src/vespa/searchlib/attribute/predicate_attribute.cpp20
-rw-r--r--searchlib/src/vespa/searchlib/predicate/CMakeLists.txt1
-rw-r--r--searchlib/src/vespa/searchlib/predicate/document_features_store_saver.cpp10
-rw-r--r--searchlib/src/vespa/searchlib/predicate/nbo_write.h21
-rw-r--r--searchlib/src/vespa/searchlib/predicate/predicate_index.cpp33
-rw-r--r--searchlib/src/vespa/searchlib/predicate/predicate_index.h15
-rw-r--r--searchlib/src/vespa/searchlib/predicate/predicate_interval.h11
-rw-r--r--searchlib/src/vespa/searchlib/predicate/simple_index.h25
-rw-r--r--searchlib/src/vespa/searchlib/predicate/simple_index.hpp35
-rw-r--r--searchlib/src/vespa/searchlib/predicate/simple_index_saver.cpp15
-rw-r--r--searchlib/src/vespa/searchlib/predicate/simple_index_saver.h30
-rw-r--r--searchlib/src/vespa/searchlib/predicate/simple_index_saver.hpp40
14 files changed, 163 insertions, 115 deletions
diff --git a/searchlib/src/tests/predicate/predicate_index_test.cpp b/searchlib/src/tests/predicate/predicate_index_test.cpp
index 8e72fc1dbd9..40b650e489a 100644
--- a/searchlib/src/tests/predicate/predicate_index_test.cpp
+++ b/searchlib/src/tests/predicate/predicate_index_test.cpp
@@ -292,8 +292,7 @@ TEST("require that PredicateIndex can be (de)serialized") {
index.commit();
vespalib::DataBuffer buffer;
- PredicateIndex::SerializeStats dummy_stats;
- index.serialize(buffer, dummy_stats);
+ index.serialize(buffer);
uint32_t doc_id_limit;
DocIdLimitFinder finder(doc_id_limit);
PredicateIndex index2(generation_holder, dummy_provider, simple_index_config,
@@ -337,8 +336,7 @@ TEST("require that DocumentFeaturesStore is restored on deserialization") {
EXPECT_FALSE(index.getIntervalIndex().lookup(hash).valid());
indexFeature(index, doc_id, min_feature, {{hash, interval}}, {{hash2, bounds}});
vespalib::DataBuffer buffer;
- PredicateIndex::SerializeStats dummy_stats;
- index.serialize(buffer, dummy_stats);
+ index.serialize(buffer);
uint32_t doc_id_limit;
DocIdLimitFinder finder(doc_id_limit);
PredicateIndex index2(generation_holder, dummy_provider, simple_index_config,
diff --git a/searchlib/src/tests/predicate/simple_index_test.cpp b/searchlib/src/tests/predicate/simple_index_test.cpp
index c37d377e668..9b99ff8e809 100644
--- a/searchlib/src/tests/predicate/simple_index_test.cpp
+++ b/searchlib/src/tests/predicate/simple_index_test.cpp
@@ -2,6 +2,9 @@
// Unit tests for simple_index.
#include <vespa/searchlib/predicate/simple_index.hpp>
+#include <vespa/searchlib/predicate/simple_index_saver.hpp>
+#include <vespa/searchlib/predicate/nbo_write.h>
+#include <vespa/searchlib/util/data_buffer_writer.h>
#include <vespa/vespalib/testkit/testapp.h>
#include <vespa/searchlib/attribute/predicate_attribute.h>
#include <vespa/vespalib/btree/btree.hpp>
@@ -31,9 +34,9 @@ struct MyData {
}
};
-struct MyDataSerializer : PostingSerializer<MyData> {
- void serialize(const MyData &data, vespalib::DataBuffer& buffer) const override {
- buffer.writeInt32(data.data);
+struct MyDataSaver : PostingSaver<MyData> {
+ void save(const MyData &data, BufferWriter& writer) const override {
+ nbo_write<uint32_t>(writer, data.data);
}
};
@@ -176,8 +179,11 @@ TEST_FF("require that SimpleIndex can be serialized and deserialized.", Fixture,
}
f1.commit();
vespalib::DataBuffer buffer;
- SimpleIndex<MyData>::SerializeStats dummy_stats;
- f1.index().serialize(buffer, MyDataSerializer(), dummy_stats);
+ {
+ DataBufferWriter writer(buffer);
+ f1.index().make_saver(std::make_unique<MyDataSaver>())->save(writer);
+ writer.flush();
+ }
MyObserver observer;
MyDataDeserializer deserializer;
f2.index().deserialize(buffer, deserializer, observer, PredicateAttribute::PREDICATE_ATTRIBUTE_VERSION);
diff --git a/searchlib/src/vespa/searchlib/attribute/predicate_attribute.cpp b/searchlib/src/vespa/searchlib/attribute/predicate_attribute.cpp
index 1f596699bca..ddf71063306 100644
--- a/searchlib/src/vespa/searchlib/attribute/predicate_attribute.cpp
+++ b/searchlib/src/vespa/searchlib/attribute/predicate_attribute.cpp
@@ -141,11 +141,8 @@ PredicateAttribute::before_inc_generation(generation_t current_gen)
void
PredicateAttribute::onSave(IAttributeSaveTarget &saveTarget) {
LOG(info, "Saving predicate attribute version %d", getVersion());
- vespalib::string name(getBaseFileName());
- PredicateIndex::SerializeStats stats;
IAttributeSaveTarget::Buffer buffer(saveTarget.datWriter().allocBuf(4_Ki));
- _index->serialize(*buffer, stats);
- size_t predicate_index_len = buffer->getDataLen();
+ _index->serialize(*buffer);
uint32_t highest_doc_id = static_cast<uint32_t>(_min_feature.size() - 1);
buffer->writeInt32(highest_doc_id);
for (size_t i = 1; i <= highest_doc_id; ++i) {
@@ -155,21 +152,6 @@ PredicateAttribute::onSave(IAttributeSaveTarget &saveTarget) {
buffer->writeInt16(_interval_range_vector[i]);
}
buffer->writeInt16(_max_interval_range);
- auto min_feature_and_interval_range_vector_len = buffer->getDataLen() - predicate_index_len;
- auto total_len = buffer->getDataLen();
- LOG(info, "Serialized predicate attribute %s: "
- "{features=%zu, zeros=%zu, "
- "interval={dictionary=%zu, btrees=%zu, bytes=%zu}, "
- "interval_with_bounds={dictionary %zu, btrees=%zu, bytes=%zu}, "
- "predicate-index_len=%zu, "
- "min_feature and interval_range_vector=%zu, total=%zu}",
- name.c_str(),
- stats._features_len, stats._zeroes_len,
- stats._interval._dictionary_size, stats._interval._btree_count, stats._interval._bytes,
- stats._interval_with_bounds._dictionary_size, stats._interval_with_bounds._btree_count, stats._interval_with_bounds._bytes,
- predicate_index_len,
- min_feature_and_interval_range_vector_len,
- total_len);
saveTarget.datWriter().writeBuf(std::move(buffer));
}
diff --git a/searchlib/src/vespa/searchlib/predicate/CMakeLists.txt b/searchlib/src/vespa/searchlib/predicate/CMakeLists.txt
index acb0391143d..48e79648675 100644
--- a/searchlib/src/vespa/searchlib/predicate/CMakeLists.txt
+++ b/searchlib/src/vespa/searchlib/predicate/CMakeLists.txt
@@ -11,6 +11,7 @@ vespa_add_library(searchlib_predicate OBJECT
predicate_tree_annotator.cpp
predicate_zero_constraint_posting_list.cpp
simple_index.cpp
+ simple_index_saver.cpp
common.cpp
DEPENDS
)
diff --git a/searchlib/src/vespa/searchlib/predicate/document_features_store_saver.cpp b/searchlib/src/vespa/searchlib/predicate/document_features_store_saver.cpp
index 08a3592da14..fcdb1afc652 100644
--- a/searchlib/src/vespa/searchlib/predicate/document_features_store_saver.cpp
+++ b/searchlib/src/vespa/searchlib/predicate/document_features_store_saver.cpp
@@ -1,8 +1,7 @@
// Copyright Vespa.ai. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
#include "document_features_store_saver.h"
-#include <vespa/searchlib/util/bufferwriter.h>
-#include <vespa/vespalib/objects/nbo.h>
+#include "nbo_write.h"
using vespalib::datastore::EntryRef;
using search::BufferWriter;
@@ -22,13 +21,6 @@ DocumentFeaturesStoreSaver::~DocumentFeaturesStoreSaver() = default;
namespace {
-template <typename T>
-void nbo_write(BufferWriter& writer, T value)
-{
- auto value_nbo = vespalib::nbo::n2h(value);
- writer.write(&value_nbo, sizeof(value_nbo));
-}
-
template <typename RefsVector, typename RangesStore>
void
find_used_words(const RefsVector& refs, const RangesStore& ranges,
diff --git a/searchlib/src/vespa/searchlib/predicate/nbo_write.h b/searchlib/src/vespa/searchlib/predicate/nbo_write.h
new file mode 100644
index 00000000000..fc5724a6012
--- /dev/null
+++ b/searchlib/src/vespa/searchlib/predicate/nbo_write.h
@@ -0,0 +1,21 @@
+// Copyright Vespa.ai. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#pragma once
+
+#include <vespa/searchlib/util/bufferwriter.h>
+#include <vespa/vespalib/objects/nbo.h>
+
+namespace search::predicate {
+
+/*
+ * Utility function for writing a scalar value
+ * in network byte order via an BufferWriter.
+ */
+template <typename T>
+void nbo_write(BufferWriter& writer, T value)
+{
+ auto value_nbo = vespalib::nbo::n2h(value);
+ writer.write(&value_nbo, sizeof(value_nbo));
+}
+
+}
diff --git a/searchlib/src/vespa/searchlib/predicate/predicate_index.cpp b/searchlib/src/vespa/searchlib/predicate/predicate_index.cpp
index 324e3fd99e8..296a6ff5c2e 100644
--- a/searchlib/src/vespa/searchlib/predicate/predicate_index.cpp
+++ b/searchlib/src/vespa/searchlib/predicate/predicate_index.cpp
@@ -3,6 +3,7 @@
#include "predicate_index.h"
#include "document_features_store_saver.h"
#include "predicate_hash.h"
+#include "simple_index_saver.h"
#include <vespa/searchlib/util/data_buffer_writer.h>
#include <vespa/vespalib/datastore/buffer_type.hpp>
#include <vespa/vespalib/btree/btree.hpp>
@@ -11,7 +12,6 @@
#include <vespa/vespalib/btree/btreestore.hpp>
#include <vespa/vespalib/btree/btreenodeallocator.hpp>
-
using vespalib::datastore::EntryRef;
using vespalib::DataBuffer;
@@ -44,20 +44,20 @@ PredicateIndex::indexDocumentFeatures(uint32_t doc_id, const PredicateIndex::Fea
namespace {
constexpr double THRESHOLD_USE_BIT_VECTOR_CACHE = 0.1;
-// PostingSerializer that writes intervals from interval store based
-// on the EntryRef that is to be serialized.
+// PostingSaver that writes intervals from interval store based
+// on the EntryRef that is to be saved.
template <typename IntervalT>
-class IntervalSerializer : public PostingSerializer<EntryRef> {
+class IntervalSaver : public PostingSaver<EntryRef> {
const PredicateIntervalStore &_store;
public:
- IntervalSerializer(const PredicateIntervalStore &store) : _store(store) {}
- void serialize(const EntryRef &ref, DataBuffer &buffer) const override {
+ IntervalSaver(const PredicateIntervalStore &store) : _store(store) {}
+ void save(const EntryRef &ref, BufferWriter& writer) const override {
uint32_t size;
IntervalT single_buf;
const IntervalT *interval = _store.get(ref, size, &single_buf);
- buffer.writeInt16(size);
+ nbo_write<uint16_t>(writer, size);
for (uint32_t i = 0; i < size; ++i) {
- interval[i].serialize(buffer);
+ interval[i].save(writer);
}
}
};
@@ -128,25 +128,26 @@ PredicateIndex::PredicateIndex(GenerationHolder &genHolder,
PredicateIndex::~PredicateIndex() = default;
void
-PredicateIndex::serialize(DataBuffer &buffer, SerializeStats& stats) const {
+PredicateIndex::serialize(DataBuffer &buffer) const {
{
auto saver = _features_store.make_saver();
DataBufferWriter writer(buffer);
saver->save(writer);
writer.flush();
}
- stats._features_len = buffer.getDataLen();
- auto old_len = buffer.getDataLen();
buffer.writeInt16(_arity);
buffer.writeInt32(_zero_constraint_docs.size());
for (auto it = _zero_constraint_docs.begin(); it.valid(); ++it) {
buffer.writeInt32(it.getKey());
}
- stats._zeroes_len = buffer.getDataLen() - old_len;
- IntervalSerializer<Interval> interval_serializer(_interval_store);
- _interval_index.serialize(buffer, interval_serializer, stats._interval);
- IntervalSerializer<IntervalWithBounds> bounds_serializer(_interval_store);
- _bounds_index.serialize(buffer, bounds_serializer, stats._interval_with_bounds);
+ {
+ DataBufferWriter writer(buffer);
+ auto interval_saver = _interval_index.make_saver(std::make_unique<IntervalSaver<Interval>>(_interval_store));
+ interval_saver->save(writer);
+ auto bounds_saver = _bounds_index.make_saver(std::make_unique<IntervalSaver<IntervalWithBounds>>(_interval_store));
+ bounds_saver->save(writer);
+ writer.flush();
+ }
}
void
diff --git a/searchlib/src/vespa/searchlib/predicate/predicate_index.h b/searchlib/src/vespa/searchlib/predicate/predicate_index.h
index 351fa3a1a9f..439187bccd7 100644
--- a/searchlib/src/vespa/searchlib/predicate/predicate_index.h
+++ b/searchlib/src/vespa/searchlib/predicate/predicate_index.h
@@ -37,19 +37,6 @@ public:
using GenerationHolder = vespalib::GenerationHolder;
using BTreeIterator = SimpleIndex<vespalib::datastore::EntryRef>::BTreeIterator;
using VectorIterator = SimpleIndex<vespalib::datastore::EntryRef>::VectorIterator;
- struct SerializeStats {
- size_t _features_len;
- size_t _zeroes_len;
- IntervalIndex::SerializeStats _interval;
- BoundsIndex::SerializeStats _interval_with_bounds;
- SerializeStats()
- : _features_len(0),
- _zeroes_len(0),
- _interval(),
- _interval_with_bounds()
- {
- }
- };
private:
uint32_t _arity;
const DocIdLimitProvider &_limit_provider;
@@ -79,7 +66,7 @@ public:
SimpleIndexDeserializeObserver<> & observer, uint32_t version);
~PredicateIndex() override;
- void serialize(vespalib::DataBuffer &buffer, SerializeStats& stats) const;
+ void serialize(vespalib::DataBuffer &buffer) const;
void onDeserializationCompleted();
void indexEmptyDocument(uint32_t doc_id);
diff --git a/searchlib/src/vespa/searchlib/predicate/predicate_interval.h b/searchlib/src/vespa/searchlib/predicate/predicate_interval.h
index 28cb17abd79..2023ec8560c 100644
--- a/searchlib/src/vespa/searchlib/predicate/predicate_interval.h
+++ b/searchlib/src/vespa/searchlib/predicate/predicate_interval.h
@@ -2,6 +2,7 @@
#pragma once
+#include "nbo_write.h"
#include <vespa/vespalib/data/databuffer.h>
namespace search::predicate {
@@ -15,8 +16,8 @@ struct Interval {
Interval() : interval(0) {}
Interval(uint32_t interval_) : interval(interval_) {}
- void serialize(vespalib::DataBuffer &buffer) const {
- buffer.writeInt32(interval);
+ void save(BufferWriter& writer) const {
+ nbo_write<uint32_t>(writer, interval);
}
static Interval deserialize(vespalib::DataBuffer &buffer) {
return Interval{buffer.readInt32()};
@@ -41,9 +42,9 @@ struct IntervalWithBounds {
IntervalWithBounds() : interval(0), bounds(0) {}
IntervalWithBounds(uint32_t interval_, uint32_t bounds_) : interval(interval_), bounds(bounds_) {}
- void serialize(vespalib::DataBuffer &buffer) const {
- buffer.writeInt32(interval);
- buffer.writeInt32(bounds);
+ void save(BufferWriter& writer) const {
+ nbo_write<uint32_t>(writer, interval);
+ nbo_write<uint32_t>(writer, bounds);
}
static IntervalWithBounds deserialize(vespalib::DataBuffer &buffer) {
uint32_t interval = buffer.readInt32();
diff --git a/searchlib/src/vespa/searchlib/predicate/simple_index.h b/searchlib/src/vespa/searchlib/predicate/simple_index.h
index 0fb6ce6e9db..1acfda24eeb 100644
--- a/searchlib/src/vespa/searchlib/predicate/simple_index.h
+++ b/searchlib/src/vespa/searchlib/predicate/simple_index.h
@@ -8,8 +8,12 @@
#include <vespa/vespalib/util/rcuvector.h>
#include <optional>
+namespace search { class BufferWriter; }
+
namespace search::predicate {
+template <typename, typename, typename> class SimpleIndexSaver;
+
template <typename Key = uint64_t, typename DocId = uint32_t>
struct SimpleIndexDeserializeObserver {
virtual ~SimpleIndexDeserializeObserver() {}
@@ -17,9 +21,9 @@ struct SimpleIndexDeserializeObserver {
};
template <typename Posting>
-struct PostingSerializer {
- virtual ~PostingSerializer() {}
- virtual void serialize(const Posting &posting, vespalib::DataBuffer &buffer) const = 0;
+struct PostingSaver {
+ virtual ~PostingSaver() {}
+ virtual void save(const Posting &posting, BufferWriter& writer) const = 0;
};
template <typename Posting>
@@ -136,17 +140,6 @@ public:
using PostingVector = vespalib::RcuVectorBase<Posting>;
using VectorStore = vespalib::btree::BTree<Key, std::shared_ptr<PostingVector>, vespalib::btree::NoAggregated>;
using VectorIterator = PostingVectorIterator<Posting, Key, DocId>;
- struct SerializeStats {
- size_t _dictionary_size;
- size_t _btree_count;
- size_t _bytes;
- SerializeStats()
- : _dictionary_size(0),
- _btree_count(0),
- _bytes(0)
- {
- }
- };
private:
using GenerationHolder = vespalib::GenerationHolder;
@@ -186,8 +179,6 @@ public:
: _generation_holder(generation_holder), _config(config), _limit_provider(provider) {}
~SimpleIndex();
- void serialize(vespalib::DataBuffer &buffer,
- const PostingSerializer<Posting> &serializer, SerializeStats& stats) const;
void deserialize(vespalib::DataBuffer &buffer,
PostingDeserializer<Posting> &deserializer,
SimpleIndexDeserializeObserver<Key, DocId> &observer, uint32_t version);
@@ -226,6 +217,8 @@ public:
return optional<VectorIterator>();
}
+
+ std::unique_ptr<SimpleIndexSaver<Posting, Key, DocId>> make_saver(std::unique_ptr<PostingSaver<Posting>> subsaver) const;
};
template<typename Posting, typename Key, typename DocId>
diff --git a/searchlib/src/vespa/searchlib/predicate/simple_index.hpp b/searchlib/src/vespa/searchlib/predicate/simple_index.hpp
index 0b5c8cbdb62..b0a65622d86 100644
--- a/searchlib/src/vespa/searchlib/predicate/simple_index.hpp
+++ b/searchlib/src/vespa/searchlib/predicate/simple_index.hpp
@@ -2,6 +2,7 @@
#pragma once
#include "simple_index.h"
+#include "simple_index_saver.h"
#include <vespa/vespalib/util/stringfmt.h>
namespace search::predicate {
@@ -69,33 +70,6 @@ SimpleIndex<Posting, Key, DocId>::~SimpleIndex() {
template <typename Posting, typename Key, typename DocId>
void
-SimpleIndex<Posting, Key, DocId>::serialize(vespalib::DataBuffer &buffer, const PostingSerializer<Posting> &serializer, SerializeStats& stats) const {
- assert(sizeof(Key) <= sizeof(uint64_t));
- assert(sizeof(DocId) <= sizeof(uint32_t));
- stats = SerializeStats();
- stats._dictionary_size = _dictionary.size();
- auto old_size = buffer.getDataLen();
- buffer.writeInt32(_dictionary.size());
- for (auto it = _dictionary.begin(); it.valid(); ++it) {
- vespalib::datastore::EntryRef ref = it.getData();
- buffer.writeInt32(_btree_posting_lists.size(ref)); // 0 if !valid()
- auto posting_it = _btree_posting_lists.begin(ref);
- if (!posting_it.valid())
- continue;
- if (posting_it.size() > 8u) {
- ++stats._btree_count;
- }
- buffer.writeInt64(it.getKey()); // Key
- for (; posting_it.valid(); ++posting_it) {
- buffer.writeInt32(posting_it.getKey()); // DocId
- serializer.serialize(posting_it.getData(), buffer);
- }
- }
- stats._bytes = buffer.getDataLen() - old_size;
-}
-
-template <typename Posting, typename Key, typename DocId>
-void
SimpleIndex<Posting, Key, DocId>::deserialize(vespalib::DataBuffer &buffer, PostingDeserializer<Posting> &deserializer,
SimpleIndexDeserializeObserver<Key, DocId> &observer, uint32_t version)
{
@@ -326,4 +300,11 @@ SimpleIndex<Posting, Key, DocId>::getMemoryUsage() const {
return combined;
};
+template <typename Posting, typename Key, typename DocId>
+std::unique_ptr<SimpleIndexSaver<Posting, Key, DocId>>
+SimpleIndex<Posting, Key, DocId>::make_saver(std::unique_ptr<PostingSaver<Posting>> subsaver) const
+{
+ return std::make_unique<SimpleIndexSaver<Posting, Key, DocId>>(_dictionary, _btree_posting_lists, std::move(subsaver));
+}
+
}
diff --git a/searchlib/src/vespa/searchlib/predicate/simple_index_saver.cpp b/searchlib/src/vespa/searchlib/predicate/simple_index_saver.cpp
new file mode 100644
index 00000000000..9b080281347
--- /dev/null
+++ b/searchlib/src/vespa/searchlib/predicate/simple_index_saver.cpp
@@ -0,0 +1,15 @@
+// Copyright Vespa.ai. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#include "simple_index_saver.hpp"
+#include <vespa/vespalib/btree/btree.hpp>
+#include <vespa/vespalib/btree/btreeroot.hpp>
+#include <vespa/vespalib/btree/btreeiterator.hpp>
+#include <vespa/vespalib/btree/btreestore.hpp>
+#include <vespa/vespalib/btree/btreenodeallocator.hpp>
+#include <vespa/vespalib/datastore/buffer_type.hpp>
+
+namespace search::predicate {
+
+template class SimpleIndexSaver<vespalib::datastore::EntryRef>;
+
+}
diff --git a/searchlib/src/vespa/searchlib/predicate/simple_index_saver.h b/searchlib/src/vespa/searchlib/predicate/simple_index_saver.h
new file mode 100644
index 00000000000..e312f42931d
--- /dev/null
+++ b/searchlib/src/vespa/searchlib/predicate/simple_index_saver.h
@@ -0,0 +1,30 @@
+// Copyright Vespa.ai. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#pragma once
+
+#include "simple_index.h"
+
+namespace search::predicate {
+
+/*
+ * Class used to save a SimpleIndex instance, streaming the serialized
+ * data via a BufferWriter.
+ */
+template <typename Posting,
+ typename Key = uint64_t, typename DocId = uint32_t>
+class SimpleIndexSaver
+{
+ using Source = SimpleIndex<Posting,Key,DocId>;
+ using Dictionary = Source::Dictionary;
+ using BTreeStore = Source::BTreeStore;
+
+ const Dictionary& _dictionary;
+ const BTreeStore& _btree_posting_lists;
+ std::unique_ptr<PostingSaver<Posting>> _subsaver;
+public:
+ SimpleIndexSaver(const Dictionary& dictionary, const BTreeStore& btree_posting_lists, std::unique_ptr<PostingSaver<Posting>> _subsaver);
+ ~SimpleIndexSaver();
+ void save(BufferWriter& writer) const;
+};
+
+}
diff --git a/searchlib/src/vespa/searchlib/predicate/simple_index_saver.hpp b/searchlib/src/vespa/searchlib/predicate/simple_index_saver.hpp
new file mode 100644
index 00000000000..a3db21ca6cb
--- /dev/null
+++ b/searchlib/src/vespa/searchlib/predicate/simple_index_saver.hpp
@@ -0,0 +1,40 @@
+// Copyright Vespa.ai. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#include "simple_index_saver.h"
+#include "nbo_write.h"
+
+namespace search::predicate {
+
+template <typename Posting, typename Key, typename DocId>
+SimpleIndexSaver<Posting, Key, DocId>::SimpleIndexSaver(const Dictionary& dictionary, const BTreeStore& btree_posting_lists, std::unique_ptr<PostingSaver<Posting>> subsaver)
+ : _dictionary(dictionary),
+ _btree_posting_lists(btree_posting_lists),
+ _subsaver(std::move(subsaver))
+{
+}
+
+template <typename Posting, typename Key, typename DocId>
+SimpleIndexSaver<Posting, Key, DocId>::~SimpleIndexSaver() = default;
+
+template <typename Posting, typename Key, typename DocId>
+void
+SimpleIndexSaver<Posting, Key, DocId>::save(BufferWriter& writer) const
+{
+ assert(sizeof(Key) <= sizeof(uint64_t));
+ assert(sizeof(DocId) <= sizeof(uint32_t));
+ nbo_write<uint32_t>(writer, _dictionary.size());
+ for (auto it = _dictionary.begin(); it.valid(); ++it) {
+ vespalib::datastore::EntryRef ref = it.getData();
+ auto posting_it = _btree_posting_lists.begin(ref);
+ nbo_write<uint32_t>(writer, posting_it.size()); // 0 if !valid()
+ if (!posting_it.valid())
+ continue;
+ nbo_write<uint64_t>(writer, it.getKey()); // Key
+ for (; posting_it.valid(); ++posting_it) {
+ nbo_write<uint32_t>(writer, posting_it.getKey()); // DocId
+ _subsaver->save(posting_it.getData(), writer);
+ }
+ }
+}
+
+}