aboutsummaryrefslogtreecommitdiffstats
path: root/searchcore/src/vespa
diff options
context:
space:
mode:
Diffstat (limited to 'searchcore/src/vespa')
-rw-r--r--searchcore/src/vespa/searchcore/proton/bucketdb/bucketdb.cpp18
-rw-r--r--searchcore/src/vespa/searchcore/proton/bucketdb/bucketdb.h4
-rw-r--r--searchcore/src/vespa/searchcore/proton/bucketdb/remove_batch_entry.h36
-rw-r--r--searchcore/src/vespa/searchcore/proton/documentmetastore/documentmetastore.cpp12
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/partial_result.cpp6
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/result_processor.cpp6
-rw-r--r--searchcore/src/vespa/searchcore/proton/persistenceengine/persistenceengine.cpp10
-rw-r--r--searchcore/src/vespa/searchcore/proton/persistenceengine/transport_latch.cpp8
-rw-r--r--searchcore/src/vespa/searchcore/proton/persistenceengine/transport_latch.h6
9 files changed, 84 insertions, 22 deletions
diff --git a/searchcore/src/vespa/searchcore/proton/bucketdb/bucketdb.cpp b/searchcore/src/vespa/searchcore/proton/bucketdb/bucketdb.cpp
index 998da6b5789..c7a3103af22 100644
--- a/searchcore/src/vespa/searchcore/proton/bucketdb/bucketdb.cpp
+++ b/searchcore/src/vespa/searchcore/proton/bucketdb/bucketdb.cpp
@@ -1,14 +1,18 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
#include "bucketdb.h"
+#include "remove_batch_entry.h"
#include <cassert>
#include <algorithm>
+#include <optional>
using document::GlobalId;
using storage::spi::BucketChecksum;
namespace proton {
+using bucketdb::RemoveBatchEntry;
+
BucketDB::BucketDB()
: _map(),
_cachedBucketId(),
@@ -65,6 +69,20 @@ BucketDB::remove(const GlobalId &gid,
}
void
+BucketDB::remove_batch(const std::vector<RemoveBatchEntry> &removed, SubDbType sub_db_type)
+{
+ std::optional<BucketId> prev_bucket_id;
+ BucketState* state = nullptr;
+ for (auto &entry : removed) {
+ if (!prev_bucket_id.has_value() || prev_bucket_id.value() != entry.get_bucket_id()) {
+ state = &_map[entry.get_bucket_id()];
+ prev_bucket_id = entry.get_bucket_id();
+ }
+ state->remove(entry.get_gid(), entry.get_timestamp(), entry.get_doc_size(), sub_db_type);
+ }
+}
+
+void
BucketDB::modify(const GlobalId &gid,
const BucketId &oldBucketId, const Timestamp &oldTimestamp, uint32_t oldDocSize,
const BucketId &newBucketId, const Timestamp &newTimestamp, uint32_t newDocSize,
diff --git a/searchcore/src/vespa/searchcore/proton/bucketdb/bucketdb.h b/searchcore/src/vespa/searchcore/proton/bucketdb/bucketdb.h
index 1723609e48e..2ea7594bde1 100644
--- a/searchcore/src/vespa/searchcore/proton/bucketdb/bucketdb.h
+++ b/searchcore/src/vespa/searchcore/proton/bucketdb/bucketdb.h
@@ -7,6 +7,8 @@
#include <vespa/persistence/spi/result.h>
#include <map>
+namespace proton::bucketdb { class RemoveBatchEntry; }
+
namespace proton {
class BucketDB
@@ -42,6 +44,8 @@ public:
const BucketId &bucketId, const Timestamp &timestamp, uint32_t docSize,
SubDbType subDbType);
+ void remove_batch(const std::vector<bucketdb::RemoveBatchEntry> &removed, SubDbType sub_db_type);
+
void modify(const GlobalId &gid,
const BucketId &oldBucketId, const Timestamp &oldTimestamp, uint32_t oldDocSize,
const BucketId &newBucketId, const Timestamp &newTimestamp, uint32_t newDocSize,
diff --git a/searchcore/src/vespa/searchcore/proton/bucketdb/remove_batch_entry.h b/searchcore/src/vespa/searchcore/proton/bucketdb/remove_batch_entry.h
new file mode 100644
index 00000000000..1ab1adb1add
--- /dev/null
+++ b/searchcore/src/vespa/searchcore/proton/bucketdb/remove_batch_entry.h
@@ -0,0 +1,36 @@
+// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#pragma once
+
+#include <vespa/document/base/globalid.h>
+#include <vespa/document/bucket/bucketid.h>
+#include <persistence/spi/types.h>
+
+
+namespace proton::bucketdb {
+
+/*
+ * Class containing meta data for a single document being removed from
+ * bucket db.
+ */
+class RemoveBatchEntry {
+ document::GlobalId _gid;
+ document::BucketId _bucket_id;
+ storage::spi::Timestamp _timestamp;
+ uint32_t _doc_size;
+public:
+ RemoveBatchEntry(const document::GlobalId& gid, const document::BucketId& bucket_id, const storage::spi::Timestamp& timestamp, uint32_t doc_size) noexcept
+ : _gid(gid),
+ _bucket_id(bucket_id),
+ _timestamp(timestamp),
+ _doc_size(doc_size)
+ {
+ }
+
+ const document::GlobalId& get_gid() const noexcept { return _gid; }
+ const document::BucketId& get_bucket_id() const noexcept { return _bucket_id; }
+ const storage::spi::Timestamp& get_timestamp() const noexcept { return _timestamp; }
+ uint32_t get_doc_size() const noexcept { return _doc_size; }
+};
+
+}
diff --git a/searchcore/src/vespa/searchcore/proton/documentmetastore/documentmetastore.cpp b/searchcore/src/vespa/searchcore/proton/documentmetastore/documentmetastore.cpp
index 3d7e1c1c774..13d4a39c8b1 100644
--- a/searchcore/src/vespa/searchcore/proton/documentmetastore/documentmetastore.cpp
+++ b/searchcore/src/vespa/searchcore/proton/documentmetastore/documentmetastore.cpp
@@ -9,6 +9,7 @@
#include <vespa/persistence/spi/bucket_limits.h>
#include <vespa/searchcore/proton/bucketdb/bucketsessionbase.h>
#include <vespa/searchcore/proton/bucketdb/joinbucketssession.h>
+#include <vespa/searchcore/proton/bucketdb/remove_batch_entry.h>
#include <vespa/searchcore/proton/bucketdb/splitbucketsession.h>
#include <vespa/searchlib/attribute/load_utils.h>
#include <vespa/searchlib/attribute/readerbase.h>
@@ -30,6 +31,7 @@ LOG_SETUP(".proton.documentmetastore");
using document::BucketId;
using document::GlobalId;
using proton::bucketdb::BucketState;
+using proton::bucketdb::RemoveBatchEntry;
using proton::documentmetastore::GidToLidMapKey;
using search::AttributeVector;
using search::FileReader;
@@ -681,13 +683,15 @@ DocumentMetaStore::removeBatch(const std::vector<DocId> &lidsToRemove, const uin
remove_batch_internal_btree(removed);
_lidAlloc.unregister_lids(lidsToRemove);
{
- bucketdb::Guard bucketGuard = _bucketDB->takeGuard();
- // TODO: add remove_batch() method to BucketDB
+ std::vector<RemoveBatchEntry> bdb_removed;
+ bdb_removed.reserve(removed.size());
for (const auto& lid_and_meta : removed) {
auto& meta = lid_and_meta.second;
- bucketGuard->remove(meta.getGid(), meta.getBucketId().stripUnused(),
- meta.getTimestamp(), meta.getDocSize(), _subDbType);
+ bdb_removed.emplace_back(meta.getGid(), meta.getBucketId().stripUnused(),
+ meta.getTimestamp(), meta.getDocSize());
}
+ bucketdb::Guard bucketGuard = _bucketDB->takeGuard();
+ bucketGuard->remove_batch(bdb_removed, _subDbType);
}
incGeneration();
if (_op_listener) {
diff --git a/searchcore/src/vespa/searchcore/proton/matching/partial_result.cpp b/searchcore/src/vespa/searchcore/proton/matching/partial_result.cpp
index 6ae97a125ad..432752d69d0 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/partial_result.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/partial_result.cpp
@@ -7,10 +7,10 @@ namespace proton::matching {
namespace {
bool before(const search::RankedHit &a, const search::RankedHit &b) {
- if (a._rankValue != b._rankValue) {
- return (a._rankValue > b._rankValue);
+ if (a.getRank() != b.getRank()) {
+ return (a.getRank() > b.getRank());
}
- return (a._docId < b._docId);
+ return (a.getDocId() < b.getDocId());
}
void mergeHits(size_t maxHits,
diff --git a/searchcore/src/vespa/searchcore/proton/matching/result_processor.cpp b/searchcore/src/vespa/searchcore/proton/matching/result_processor.cpp
index 341bd3bb855..f332ca5ec26 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/result_processor.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/result_processor.cpp
@@ -108,7 +108,7 @@ ResultProcessor::extract_docid_ordering(const PartialResult &result) const
std::vector<std::pair<uint32_t,uint32_t>> list;
list.reserve(est_size);
for (size_t i = _offset; i < result.size(); ++i) {
- list.emplace_back(result.hit(i)._docId, list.size());
+ list.emplace_back(result.hit(i).getDocId(), list.size());
}
std::sort(list.begin(), list.end(), [](const auto &a, const auto &b){ return (a.first < b.first); });
return list;
@@ -142,11 +142,11 @@ ResultProcessor::makeReply(PartialResultUP full_result)
for (size_t i = 0; i < hitcnt; ++i) {
search::engine::SearchReply::Hit &dst = r.hits[i];
const search::RankedHit &src = result.hit(hitOffset + i);
- uint32_t docId = src._docId;
+ uint32_t docId = src.getDocId();
if (metaStore.getGidEvenIfMoved(docId, gid)) {
dst.gid = gid;
}
- dst.metric = src._rankValue;
+ dst.metric = src.getRank();
LOG(debug, "convertLidToGid: hit[%zu]: lid(%u) -> gid(%s)", i, docId, dst.gid.toString().c_str());
}
if (result.hasSortData() && (hitcnt > 0)) {
diff --git a/searchcore/src/vespa/searchcore/proton/persistenceengine/persistenceengine.cpp b/searchcore/src/vespa/searchcore/proton/persistenceengine/persistenceengine.cpp
index 2e1fc74037c..04d91b9c028 100644
--- a/searchcore/src/vespa/searchcore/proton/persistenceengine/persistenceengine.cpp
+++ b/searchcore/src/vespa/searchcore/proton/persistenceengine/persistenceengine.cpp
@@ -364,7 +364,7 @@ PersistenceEngine::putAsync(const Bucket &bucket, Timestamp ts, storage::spi::Do
return onComplete->onComplete(std::make_unique<Result>(Result::ErrorType::PERMANENT_ERROR,
make_string("No handler for document type '%s'", docType.toString().c_str())));
}
- auto transportContext = std::make_shared<AsyncTranportContext>(1, std::move(onComplete));
+ auto transportContext = std::make_shared<AsyncTransportContext>(1, std::move(onComplete));
handler->handlePut(feedtoken::make(std::move(transportContext)), bucket, ts, std::move(doc));
}
@@ -384,7 +384,7 @@ PersistenceEngine::removeAsync(const Bucket& b, Timestamp t, const DocumentId& d
return onComplete->onComplete(std::make_unique<RemoveResult>(Result::ErrorType::PERMANENT_ERROR,
make_string("No handler for document type '%s'", docType.toString().c_str())));
}
- auto transportContext = std::make_shared<AsyncTranportContext>(1, std::move(onComplete));
+ auto transportContext = std::make_shared<AsyncTransportContext>(1, std::move(onComplete));
handler->handleRemove(feedtoken::make(std::move(transportContext)), b, t, did);
}
@@ -436,7 +436,7 @@ PersistenceEngine::updateAsync(const Bucket& b, Timestamp t, DocumentUpdate::SP
if (handler == nullptr) {
return onComplete->onComplete(std::make_unique<UpdateResult>(Result::ErrorType::PERMANENT_ERROR, make_string("No handler for document type '%s'", docType.toString().c_str())));
}
- auto transportContext = std::make_shared<AsyncTranportContext>(1, std::move(onComplete));
+ auto transportContext = std::make_shared<AsyncTransportContext>(1, std::move(onComplete));
handler->handleUpdate(feedtoken::make(std::move(transportContext)), b, t, std::move(upd));
}
@@ -555,7 +555,7 @@ PersistenceEngine::createBucketAsync(const Bucket &b, Context &, OperationComple
LOG(spam, "createBucket(%s)", b.toString().c_str());
HandlerSnapshot snap = getHandlerSnapshot(rguard, b.getBucketSpace());
- auto transportContext = std::make_shared<AsyncTranportContext>(snap.size(), std::move(onComplete));
+ auto transportContext = std::make_shared<AsyncTransportContext>(snap.size(), std::move(onComplete));
while (snap.handlers().valid()) {
IPersistenceHandler *handler = snap.handlers().get();
snap.handlers().next();
@@ -575,7 +575,7 @@ PersistenceEngine::deleteBucketAsync(const Bucket& b, Context&, OperationComplet
LOG(spam, "deleteBucket(%s)", b.toString().c_str());
HandlerSnapshot snap = getHandlerSnapshot(rguard, b.getBucketSpace());
- auto transportContext = std::make_shared<AsyncTranportContext>(snap.size(), std::move(onComplete));
+ auto transportContext = std::make_shared<AsyncTransportContext>(snap.size(), std::move(onComplete));
while (snap.handlers().valid()) {
IPersistenceHandler *handler = snap.handlers().get();
snap.handlers().next();
diff --git a/searchcore/src/vespa/searchcore/proton/persistenceengine/transport_latch.cpp b/searchcore/src/vespa/searchcore/proton/persistenceengine/transport_latch.cpp
index 61c240f0d6a..8a0955b3147 100644
--- a/searchcore/src/vespa/searchcore/proton/persistenceengine/transport_latch.cpp
+++ b/searchcore/src/vespa/searchcore/proton/persistenceengine/transport_latch.cpp
@@ -67,7 +67,7 @@ TransportLatch::send(ResultUP result, bool documentWasFound)
_latch.countDown();
}
-AsyncTranportContext::AsyncTranportContext(uint32_t cnt, OperationComplete::UP onComplete)
+AsyncTransportContext::AsyncTransportContext(uint32_t cnt, OperationComplete::UP onComplete)
: TransportMerger(cnt > 1),
_countDown(cnt),
_onComplete(std::move(onComplete))
@@ -78,17 +78,17 @@ AsyncTranportContext::AsyncTranportContext(uint32_t cnt, OperationComplete::UP o
}
void
-AsyncTranportContext::completeIfDone() {
+AsyncTransportContext::completeIfDone() {
_countDown--;
if (_countDown == 0) {
_onComplete->onComplete(std::move(_result));
}
}
-AsyncTranportContext::~AsyncTranportContext() = default;
+AsyncTransportContext::~AsyncTransportContext() = default;
void
-AsyncTranportContext::send(ResultUP result, bool documentWasFound)
+AsyncTransportContext::send(ResultUP result, bool documentWasFound)
{
mergeResult(std::move(result), documentWasFound);
}
diff --git a/searchcore/src/vespa/searchcore/proton/persistenceengine/transport_latch.h b/searchcore/src/vespa/searchcore/proton/persistenceengine/transport_latch.h
index 084ad0e8d10..91cadfedcd5 100644
--- a/searchcore/src/vespa/searchcore/proton/persistenceengine/transport_latch.h
+++ b/searchcore/src/vespa/searchcore/proton/persistenceengine/transport_latch.h
@@ -63,7 +63,7 @@ public:
* Implementation of FeedToken::ITransport for async handling of the async reply for an operation.
* Uses an internal count to keep track the number of the outstanding replies.
*/
-class AsyncTranportContext : public TransportMerger {
+class AsyncTransportContext : public TransportMerger {
private:
using Result = storage::spi::Result;
using OperationComplete = storage::spi::OperationComplete;
@@ -72,8 +72,8 @@ private:
OperationComplete::UP _onComplete;
void completeIfDone() override;
public:
- AsyncTranportContext(uint32_t cnt, OperationComplete::UP);
- ~AsyncTranportContext() override;
+ AsyncTransportContext(uint32_t cnt, OperationComplete::UP);
+ ~AsyncTransportContext() override;
void send(ResultUP result, bool documentWasFound) override;
};