summaryrefslogtreecommitdiffstats
path: root/searchcore
diff options
context:
space:
mode:
authorHenning Baldersheim <balder@yahoo-inc.com>2020-08-23 13:18:06 +0000
committerHenning Baldersheim <balder@yahoo-inc.com>2020-08-23 13:18:06 +0000
commit797ebaa0751f8cc1c943abef749ee1ed50259fcc (patch)
tree574c4797e3fd8bfa7c1540f1135985cb84d46ec7 /searchcore
parent282e9afaf6ddfd4d7bbd042d060895380dfd9601 (diff)
Track pending uncommitted lids.
Diffstat (limited to 'searchcore')
-rw-r--r--searchcore/src/tests/proton/document_iterator/document_iterator_test.cpp4
-rw-r--r--searchcore/src/tests/proton/documentdb/document_subdbs/document_subdbs_test.cpp4
-rw-r--r--searchcore/src/tests/proton/documentdb/maintenancecontroller/maintenancecontroller_test.cpp4
-rw-r--r--searchcore/src/vespa/searchcore/proton/common/icommitable.h6
-rw-r--r--searchcore/src/vespa/searchcore/proton/common/pendinglidtracker.cpp15
-rw-r--r--searchcore/src/vespa/searchcore/proton/common/pendinglidtracker.h39
-rw-r--r--searchcore/src/vespa/searchcore/proton/persistenceengine/commit_and_wait_document_retriever.cpp2
-rw-r--r--searchcore/src/vespa/searchcore/proton/persistenceengine/commit_and_wait_document_retriever.h4
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/combiningfeedview.cpp2
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/combiningfeedview.h2
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.cpp2
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/ifeedview.h4
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/putdonecontext.cpp4
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/putdonecontext.h16
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/storeonlyfeedview.cpp46
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/storeonlyfeedview.h4
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/updatedonecontext.cpp3
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/updatedonecontext.h4
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/visibilityhandler.cpp4
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/visibilityhandler.h4
-rw-r--r--searchcore/src/vespa/searchcore/proton/test/dummy_feed_view.cpp4
-rw-r--r--searchcore/src/vespa/searchcore/proton/test/dummy_feed_view.h2
22 files changed, 117 insertions, 62 deletions
diff --git a/searchcore/src/tests/proton/document_iterator/document_iterator_test.cpp b/searchcore/src/tests/proton/document_iterator/document_iterator_test.cpp
index d8da0b5eb4c..b106233a13e 100644
--- a/searchcore/src/tests/proton/document_iterator/document_iterator_test.cpp
+++ b/searchcore/src/tests/proton/document_iterator/document_iterator_test.cpp
@@ -280,10 +280,10 @@ struct Committer : public ICommitable {
Committer() : _commitCount(0), _commitAndWaitCount(0) { }
void commit() override { _commitCount++; }
void commitAndWait() override { _commitAndWaitCount++; }
- void commitAndWait(PendingLidTracker & , uint32_t ) override {
+ void commitAndWait(IPendingLidTracker &, uint32_t ) override {
commitAndWait();
}
- void commitAndWait(PendingLidTracker & , const std::vector<uint32_t> & ) override {
+ void commitAndWait(IPendingLidTracker &, const std::vector<uint32_t> & ) override {
commitAndWait();
}
};
diff --git a/searchcore/src/tests/proton/documentdb/document_subdbs/document_subdbs_test.cpp b/searchcore/src/tests/proton/documentdb/document_subdbs/document_subdbs_test.cpp
index 1d6e0074ae5..eeb6fc1966f 100644
--- a/searchcore/src/tests/proton/documentdb/document_subdbs/document_subdbs_test.cpp
+++ b/searchcore/src/tests/proton/documentdb/document_subdbs/document_subdbs_test.cpp
@@ -268,10 +268,10 @@ struct Committer : public ICommitable {
Committer() : _commitCount(0), _commitAndWaitCount(0) { }
void commit() override { _commitCount++; }
void commitAndWait() override { _commitAndWaitCount++; }
- void commitAndWait(PendingLidTracker & , uint32_t ) override {
+ void commitAndWait(IPendingLidTracker &, uint32_t ) override {
commitAndWait();
}
- void commitAndWait(PendingLidTracker & , const std::vector<uint32_t> & ) override {
+ void commitAndWait(IPendingLidTracker &, const std::vector<uint32_t> & ) override {
commitAndWait();
}
};
diff --git a/searchcore/src/tests/proton/documentdb/maintenancecontroller/maintenancecontroller_test.cpp b/searchcore/src/tests/proton/documentdb/maintenancecontroller/maintenancecontroller_test.cpp
index 2f35a0c04b5..daae29dabdc 100644
--- a/searchcore/src/tests/proton/documentdb/maintenancecontroller/maintenancecontroller_test.cpp
+++ b/searchcore/src/tests/proton/documentdb/maintenancecontroller/maintenancecontroller_test.cpp
@@ -387,10 +387,10 @@ public:
void syncSubDBs();
void commit() override { }
void commitAndWait() override { }
- void commitAndWait(PendingLidTracker & , uint32_t ) override {
+ void commitAndWait(IPendingLidTracker &, uint32_t ) override {
commitAndWait();
}
- void commitAndWait(PendingLidTracker & , const std::vector<uint32_t> & ) override {
+ void commitAndWait(IPendingLidTracker &, const std::vector<uint32_t> & ) override {
commitAndWait();
}
void performSyncSubDBs();
diff --git a/searchcore/src/vespa/searchcore/proton/common/icommitable.h b/searchcore/src/vespa/searchcore/proton/common/icommitable.h
index 8fcaafb548f..3623e6d60d1 100644
--- a/searchcore/src/vespa/searchcore/proton/common/icommitable.h
+++ b/searchcore/src/vespa/searchcore/proton/common/icommitable.h
@@ -5,7 +5,7 @@
#include <vector>
namespace proton {
-class PendingLidTracker;
+class IPendingLidTracker;
/**
* Interface for anyone that needs to commit.
@@ -14,8 +14,8 @@ class ICommitable {
public:
virtual void commit() = 0;
virtual void commitAndWait() = 0;
- virtual void commitAndWait(PendingLidTracker & uncommittedLidTracker, uint32_t lid) = 0;
- virtual void commitAndWait(PendingLidTracker & uncommittedLidTracker, const std::vector<uint32_t> & lid) = 0;
+ virtual void commitAndWait(IPendingLidTracker &uncommittedLidTracker, uint32_t lid) = 0;
+ virtual void commitAndWait(IPendingLidTracker &uncommittedLidTracker, const std::vector<uint32_t> & lid) = 0;
protected:
virtual ~ICommitable() = default;
};
diff --git a/searchcore/src/vespa/searchcore/proton/common/pendinglidtracker.cpp b/searchcore/src/vespa/searchcore/proton/common/pendinglidtracker.cpp
index 4616f256863..8a4c01a4560 100644
--- a/searchcore/src/vespa/searchcore/proton/common/pendinglidtracker.cpp
+++ b/searchcore/src/vespa/searchcore/proton/common/pendinglidtracker.cpp
@@ -6,17 +6,28 @@
namespace proton {
-PendingLidTracker::Token::Token(uint32_t lid, PendingLidTracker &tracker)
+IPendingLidTracker::Token::Token(uint32_t lid, IPendingLidTracker &tracker)
: _tracker(&tracker),
_lid(lid)
{}
-PendingLidTracker::Token::~Token() {
+IPendingLidTracker::Token::Token()
+ : _tracker(nullptr),
+ _lid(0u)
+{}
+
+
+IPendingLidTracker::Token::~Token() {
if (_tracker != nullptr) {
_tracker->consume(_lid);
}
}
+IPendingLidTracker::Token
+NoopLidTracker::produce(uint32_t) {
+ return Token();
+}
+
PendingLidTracker::PendingLidTracker()
: _mutex(),
_cond(),
diff --git a/searchcore/src/vespa/searchcore/proton/common/pendinglidtracker.h b/searchcore/src/vespa/searchcore/proton/common/pendinglidtracker.h
index 7f06240c78f..15dd0a9a4c3 100644
--- a/searchcore/src/vespa/searchcore/proton/common/pendinglidtracker.h
+++ b/searchcore/src/vespa/searchcore/proton/common/pendinglidtracker.h
@@ -8,31 +8,52 @@
namespace proton {
-class PendingLidTracker {
+class IPendingLidTracker {
public:
class Token {
public:
- Token(uint32_t lid, PendingLidTracker & tracker);
+ Token();
+ Token(uint32_t lid, IPendingLidTracker & tracker);
Token(const Token &) = delete;
Token & operator = (const Token &) = delete;
Token & operator = (Token &&) = delete;
Token(Token && rhs) noexcept
- : _tracker(rhs._tracker),
- _lid(rhs._lid)
+ : _tracker(rhs._tracker),
+ _lid(rhs._lid)
{
rhs._tracker = nullptr;
}
~Token();
private:
- PendingLidTracker * _tracker;
+ IPendingLidTracker * _tracker;
uint32_t _lid;
};
+ virtual ~IPendingLidTracker() = default;
+ virtual Token produce(uint32_t lid) = 0;
+ virtual void waitForConsumedLid(uint32_t lid) = 0;
+private:
+ virtual void consume(uint32_t lid) = 0;
+ std::mutex _mutex;
+ std::condition_variable _cond;
+ vespalib::hash_map<uint32_t, uint32_t> _pending;
+};
+
+class NoopLidTracker : public IPendingLidTracker {
+public:
+ Token produce(uint32_t lid) override;
+ void waitForConsumedLid(uint32_t ) override { }
+private:
+ void consume(uint32_t ) override { }
+};
+
+class PendingLidTracker : public IPendingLidTracker {
+public:
PendingLidTracker();
- ~PendingLidTracker();
- Token produce(uint32_t lid);
- void waitForConsumedLid(uint32_t lid);
+ ~PendingLidTracker() override;
+ Token produce(uint32_t lid) override;
+ void waitForConsumedLid(uint32_t lid) override;
private:
- void consume(uint32_t lid);
+ void consume(uint32_t lid) override;
std::mutex _mutex;
std::condition_variable _cond;
vespalib::hash_map<uint32_t, uint32_t> _pending;
diff --git a/searchcore/src/vespa/searchcore/proton/persistenceengine/commit_and_wait_document_retriever.cpp b/searchcore/src/vespa/searchcore/proton/persistenceengine/commit_and_wait_document_retriever.cpp
index cf76f098dc9..f6a52ad7b44 100644
--- a/searchcore/src/vespa/searchcore/proton/persistenceengine/commit_and_wait_document_retriever.cpp
+++ b/searchcore/src/vespa/searchcore/proton/persistenceengine/commit_and_wait_document_retriever.cpp
@@ -6,7 +6,7 @@
namespace proton {
CommitAndWaitDocumentRetriever::CommitAndWaitDocumentRetriever(IDocumentRetriever::SP retriever, ICommitable &commit,
- PendingLidTracker & unCommittedLidTracker)
+ IPendingLidTracker & unCommittedLidTracker)
: _retriever(std::move(retriever)),
_commit(commit),
_uncommittedLidsTracker(unCommittedLidTracker)
diff --git a/searchcore/src/vespa/searchcore/proton/persistenceengine/commit_and_wait_document_retriever.h b/searchcore/src/vespa/searchcore/proton/persistenceengine/commit_and_wait_document_retriever.h
index 93778c56126..33258f767dc 100644
--- a/searchcore/src/vespa/searchcore/proton/persistenceengine/commit_and_wait_document_retriever.h
+++ b/searchcore/src/vespa/searchcore/proton/persistenceengine/commit_and_wait_document_retriever.h
@@ -17,10 +17,10 @@ class CommitAndWaitDocumentRetriever : public IDocumentRetriever
{
IDocumentRetriever::SP _retriever;
ICommitable &_commit;
- PendingLidTracker &_uncommittedLidsTracker;
+ IPendingLidTracker &_uncommittedLidsTracker;
using Bucket = storage::spi::Bucket;
public:
- CommitAndWaitDocumentRetriever(IDocumentRetriever::SP retriever, ICommitable &commit, PendingLidTracker & unCommittedLidTracker);
+ CommitAndWaitDocumentRetriever(IDocumentRetriever::SP retriever, ICommitable &commit, IPendingLidTracker & unCommittedLidTracker);
~CommitAndWaitDocumentRetriever() override;
const document::DocumentTypeRepo &getDocumentTypeRepo() const override;
diff --git a/searchcore/src/vespa/searchcore/proton/server/combiningfeedview.cpp b/searchcore/src/vespa/searchcore/proton/server/combiningfeedview.cpp
index 3a7597659ae..09dbace0bdc 100644
--- a/searchcore/src/vespa/searchcore/proton/server/combiningfeedview.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/combiningfeedview.cpp
@@ -233,7 +233,7 @@ CombiningFeedView::forceCommit(search::SerialNum serialNum)
}
}
-PendingLidTracker &
+IPendingLidTracker &
CombiningFeedView::getUncommittedLidsTracker() {
LOG_ABORT("CombiningFeedView::getUncommittedLidsTracker should never be called.");
}
diff --git a/searchcore/src/vespa/searchcore/proton/server/combiningfeedview.h b/searchcore/src/vespa/searchcore/proton/server/combiningfeedview.h
index 9d6686867b0..aa730e104aa 100644
--- a/searchcore/src/vespa/searchcore/proton/server/combiningfeedview.h
+++ b/searchcore/src/vespa/searchcore/proton/server/combiningfeedview.h
@@ -77,7 +77,7 @@ public:
void sync() override;
void handlePruneRemovedDocuments(const PruneRemovedDocumentsOperation &pruneOp) override;
void handleCompactLidSpace(const CompactLidSpaceOperation &op) override;
- PendingLidTracker & getUncommittedLidsTracker() override;
+ IPendingLidTracker & getUncommittedLidsTracker() override;
// Called by document db executor
void setCalculator(const IBucketStateCalculator::SP &newCalc);
diff --git a/searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.cpp b/searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.cpp
index db53c5812c2..54363c5b197 100644
--- a/searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.cpp
@@ -130,7 +130,7 @@ DocumentSubDBCollection::createRetrievers()
namespace {
IDocumentRetriever::SP
-wrapRetriever(IDocumentRetriever::SP retriever, ICommitable &commit, PendingLidTracker & unCommitedLidsTracker)
+wrapRetriever(IDocumentRetriever::SP retriever, ICommitable &commit, IPendingLidTracker & unCommitedLidsTracker)
{
return std::make_shared<CommitAndWaitDocumentRetriever>(std::move(retriever), commit, unCommitedLidsTracker);
}
diff --git a/searchcore/src/vespa/searchcore/proton/server/ifeedview.h b/searchcore/src/vespa/searchcore/proton/server/ifeedview.h
index 1413f5aa3e3..11e95937ec9 100644
--- a/searchcore/src/vespa/searchcore/proton/server/ifeedview.h
+++ b/searchcore/src/vespa/searchcore/proton/server/ifeedview.h
@@ -4,6 +4,7 @@
#include <vespa/searchcore/proton/common/feedtoken.h>
#include <vespa/searchlib/common/serialnum.h>
+#include <vespa/searchcore/proton/common/pendinglidtracker.h>
namespace document { class DocumentTypeRepo; }
@@ -19,7 +20,6 @@ class PruneRemovedDocumentsOperation;
class PutOperation;
class RemoveOperation;
class UpdateOperation;
-class PendingLidTracker;
/**
* Interface for a feed view as seen from a feed handler.
@@ -62,7 +62,7 @@ public:
virtual void forceCommit(search::SerialNum serialNum) = 0;
virtual void handlePruneRemovedDocuments(const PruneRemovedDocumentsOperation & pruneOp) = 0;
virtual void handleCompactLidSpace(const CompactLidSpaceOperation &op) = 0;
- virtual PendingLidTracker & getUncommittedLidsTracker() = 0;
+ virtual IPendingLidTracker & getUncommittedLidsTracker() = 0;
};
} // namespace proton
diff --git a/searchcore/src/vespa/searchcore/proton/server/putdonecontext.cpp b/searchcore/src/vespa/searchcore/proton/server/putdonecontext.cpp
index 0af9101f826..20ffe203235 100644
--- a/searchcore/src/vespa/searchcore/proton/server/putdonecontext.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/putdonecontext.cpp
@@ -9,11 +9,13 @@ using document::Document;
namespace proton {
-PutDoneContext::PutDoneContext(FeedToken token, IGidToLidChangeHandler &gidToLidChangeHandler,
+PutDoneContext::PutDoneContext(FeedToken token, IPendingLidTracker::Token uncommitted,
+ IGidToLidChangeHandler &gidToLidChangeHandler,
std::shared_ptr<const Document> doc,
const document::GlobalId &gid, uint32_t lid,
search::SerialNum serialNum, bool enableNotifyPut)
: OperationDoneContext(std::move(token)),
+ _uncommitted(std::move(uncommitted)),
_lid(lid),
_docIdLimit(nullptr),
_gidToLidChangeHandler(gidToLidChangeHandler),
diff --git a/searchcore/src/vespa/searchcore/proton/server/putdonecontext.h b/searchcore/src/vespa/searchcore/proton/server/putdonecontext.h
index a2d60c856ba..024876c6c3d 100644
--- a/searchcore/src/vespa/searchcore/proton/server/putdonecontext.h
+++ b/searchcore/src/vespa/searchcore/proton/server/putdonecontext.h
@@ -3,6 +3,7 @@
#pragma once
#include "operationdonecontext.h"
+#include <vespa/searchcore/proton/common/pendinglidtracker.h>
#include <vespa/document/base/globalid.h>
#include <vespa/searchlib/common/serialnum.h>
@@ -22,16 +23,17 @@ class IGidToLidChangeHandler;
*/
class PutDoneContext : public OperationDoneContext
{
- uint32_t _lid;
- DocIdLimit *_docIdLimit;
- IGidToLidChangeHandler &_gidToLidChangeHandler;
- document::GlobalId _gid;
- search::SerialNum _serialNum;
- bool _enableNotifyPut;
+ IPendingLidTracker::Token _uncommitted;
+ uint32_t _lid;
+ DocIdLimit *_docIdLimit;
+ IGidToLidChangeHandler &_gidToLidChangeHandler;
+ document::GlobalId _gid;
+ search::SerialNum _serialNum;
+ bool _enableNotifyPut;
std::shared_ptr<const document::Document> _doc;
public:
- PutDoneContext(FeedToken token, IGidToLidChangeHandler &gidToLidChangeHandler,
+ PutDoneContext(FeedToken token, IPendingLidTracker::Token uncommitted, IGidToLidChangeHandler &gidToLidChangeHandler,
std::shared_ptr<const document::Document> doc,
const document::GlobalId &gid, uint32_t lid, search::SerialNum serialNum, bool enableNotifyPut);
~PutDoneContext() override;
diff --git a/searchcore/src/vespa/searchcore/proton/server/storeonlyfeedview.cpp b/searchcore/src/vespa/searchcore/proton/server/storeonlyfeedview.cpp
index 6d98f99db51..c5799f9082c 100644
--- a/searchcore/src/vespa/searchcore/proton/server/storeonlyfeedview.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/storeonlyfeedview.cpp
@@ -49,18 +49,19 @@ private:
IDestructorCallback::SP _moveDoneCtx;
public:
- PutDoneContextForMove(FeedToken token, IGidToLidChangeHandler &gidToLidChangeHandler,
+ PutDoneContextForMove(FeedToken token, IPendingLidTracker::Token uncommitted,
+ IGidToLidChangeHandler &gidToLidChangeHandler,
std::shared_ptr<const Document> doc,
const document::GlobalId &gid, uint32_t lid, search::SerialNum serialNum,
bool enableNotifyPut, IDestructorCallback::SP moveDoneCtx)
- : PutDoneContext(std::move(token), gidToLidChangeHandler, std::move(doc), gid, lid, serialNum, enableNotifyPut),
+ : PutDoneContext(std::move(token), std::move(uncommitted), gidToLidChangeHandler, std::move(doc), gid, lid, serialNum, enableNotifyPut),
_moveDoneCtx(std::move(moveDoneCtx))
{}
~PutDoneContextForMove() override = default;
};
std::shared_ptr<PutDoneContext>
-createPutDoneContext(FeedToken token,
+createPutDoneContext(FeedToken token, IPendingLidTracker::Token uncommitted,
IGidToLidChangeHandler &gidToLidChangeHandler,
std::shared_ptr<const Document> doc,
const document::GlobalId &gid, uint32_t lid,
@@ -69,25 +70,29 @@ createPutDoneContext(FeedToken token,
{
std::shared_ptr<PutDoneContext> result;
if (moveDoneCtx) {
- result = std::make_shared<PutDoneContextForMove>(std::move(token), gidToLidChangeHandler, std::move(doc), gid, lid, serialNum, enableNotifyPut, std::move(moveDoneCtx));
+ result = std::make_shared<PutDoneContextForMove>(std::move(token), std::move(uncommitted), gidToLidChangeHandler,
+ std::move(doc), gid, lid, serialNum, enableNotifyPut, std::move(moveDoneCtx));
} else {
- result = std::make_shared<PutDoneContext>(std::move(token), gidToLidChangeHandler, std::move(doc), gid, lid, serialNum, enableNotifyPut);
+ result = std::make_shared<PutDoneContext>(std::move(token), std::move(uncommitted), gidToLidChangeHandler,
+ std::move(doc), gid, lid, serialNum, enableNotifyPut);
}
return result;
}
std::shared_ptr<PutDoneContext>
-createPutDoneContext(FeedToken token, IGidToLidChangeHandler &gidToLidChangeHandler,
+createPutDoneContext(FeedToken token, IPendingLidTracker::Token uncommitted,
+ IGidToLidChangeHandler &gidToLidChangeHandler,
std::shared_ptr<const Document> doc,
const document::GlobalId &gid, uint32_t lid, SerialNum serialNum, bool enableNotifyPut)
{
- return createPutDoneContext(std::move(token), gidToLidChangeHandler, std::move(doc), gid, lid, serialNum, enableNotifyPut, IDestructorCallback::SP());
+ return createPutDoneContext(std::move(token), std::move(uncommitted), gidToLidChangeHandler, std::move(doc), gid,
+ lid, serialNum, enableNotifyPut, IDestructorCallback::SP());
}
std::shared_ptr<UpdateDoneContext>
-createUpdateDoneContext(FeedToken token, const DocumentUpdate::SP &upd)
+createUpdateDoneContext(FeedToken token, IPendingLidTracker::Token uncommitted, const DocumentUpdate::SP &upd)
{
- return std::make_shared<UpdateDoneContext>(std::move(token), upd);
+ return std::make_shared<UpdateDoneContext>(std::move(token), std::move(uncommitted), upd);
}
void setPrev(DocumentOperation &op, const documentmetastore::IStore::Result &result,
@@ -187,6 +192,15 @@ moveMetaData(documentmetastore::IStore &meta_store, const DocumentId & doc_id, c
meta_store.move(op.getPrevLid(), op.getLid());
}
+std::unique_ptr<IPendingLidTracker>
+createUncommitedLidTracker(bool needImmediateCommit) {
+ if (needImmediateCommit) {
+ return std::make_unique<NoopLidTracker>();
+ } else {
+ return std::make_unique<PendingLidTracker>();
+ }
+}
+
} // namespace
StoreOnlyFeedView::StoreOnlyFeedView(const Context &ctx, const PersistentParams &params)
@@ -198,7 +212,7 @@ StoreOnlyFeedView::StoreOnlyFeedView(const Context &ctx, const PersistentParams
_docType(nullptr),
_lidReuseDelayer(ctx._writeService, _documentMetaStoreContext->get(), ctx._lidReuseDelayerConfig),
_pendingLidsForDocStore(),
- _pendingLidsForCommit(),
+ _pendingLidsForCommit(createUncommitedLidTracker(_lidReuseDelayer.getImmediateCommit())),
_schema(ctx._schema),
_writeService(ctx._writeService),
_params(params),
@@ -216,9 +230,9 @@ StoreOnlyFeedView::sync()
_writeService.summary().sync();
}
-PendingLidTracker &
+IPendingLidTracker &
StoreOnlyFeedView::getUncommittedLidsTracker() {
- return _pendingLidsForCommit;
+ return *_pendingLidsForCommit;
}
void
StoreOnlyFeedView::forceCommit(SerialNum serialNum)
@@ -294,7 +308,8 @@ StoreOnlyFeedView::internalPut(FeedToken token, const PutOperation &putOp)
bool immediateCommit = needCommit();
const document::GlobalId &gid = docId.getGlobalId();
std::shared_ptr<PutDoneContext> onWriteDone =
- createPutDoneContext(std::move(token), _gidToLidChangeHandler, doc, gid, putOp.getLid(), serialNum,
+ createPutDoneContext(std::move(token), _pendingLidsForCommit->produce(putOp.getLid()),
+ _gidToLidChangeHandler, doc, gid, putOp.getLid(), serialNum,
putOp.changedDbdId() && useDocumentMetaStore(serialNum));
putSummary(serialNum, putOp.getLid(), doc, onWriteDone);
putAttributes(serialNum, putOp.getLid(), *doc, immediateCommit, onWriteDone);
@@ -446,7 +461,7 @@ StoreOnlyFeedView::internalUpdate(FeedToken token, const UpdateOperation &updOp)
considerEarlyAck(token);
bool immediateCommit = needCommit();
- auto onWriteDone = createUpdateDoneContext(std::move(token), updOp.getUpdate());
+ auto onWriteDone = createUpdateDoneContext(std::move(token), _pendingLidsForCommit->produce(updOp.getLid()), updOp.getUpdate());
UpdateScope updateScope(*_schema, upd);
updateAttributes(serialNum, lid, upd, immediateCommit, onWriteDone, updateScope);
@@ -772,7 +787,8 @@ StoreOnlyFeedView::handleMove(const MoveOperation &moveOp, IDestructorCallback::
bool immediateCommit = needCommit();
const document::GlobalId &gid = docId.getGlobalId();
std::shared_ptr<PutDoneContext> onWriteDone =
- createPutDoneContext(FeedToken(), _gidToLidChangeHandler, doc, gid, moveOp.getLid(), serialNum,
+ createPutDoneContext(FeedToken(), _pendingLidsForCommit->produce(moveOp.getLid()),
+ _gidToLidChangeHandler, doc, gid, moveOp.getLid(), serialNum,
moveOp.changedDbdId() && useDocumentMetaStore(serialNum), doneCtx);
putSummary(serialNum, moveOp.getLid(), doc, onWriteDone);
putAttributes(serialNum, moveOp.getLid(), *doc, immediateCommit, onWriteDone);
diff --git a/searchcore/src/vespa/searchcore/proton/server/storeonlyfeedview.h b/searchcore/src/vespa/searchcore/proton/server/storeonlyfeedview.h
index 87569d2876f..94ed2cf1b4c 100644
--- a/searchcore/src/vespa/searchcore/proton/server/storeonlyfeedview.h
+++ b/searchcore/src/vespa/searchcore/proton/server/storeonlyfeedview.h
@@ -144,7 +144,7 @@ private:
const document::DocumentType *_docType;
LidReuseDelayer _lidReuseDelayer;
PendingLidTracker _pendingLidsForDocStore;
- PendingLidTracker _pendingLidsForCommit;
+ std::unique_ptr<IPendingLidTracker> _pendingLidsForCommit;
protected:
const search::index::Schema::SP _schema;
@@ -263,7 +263,7 @@ public:
*/
void handlePruneRemovedDocuments(const PruneRemovedDocumentsOperation &pruneOp) override;
void handleCompactLidSpace(const CompactLidSpaceOperation &op) override;
- PendingLidTracker & getUncommittedLidsTracker() override;
+ IPendingLidTracker & getUncommittedLidsTracker() override;
};
}
diff --git a/searchcore/src/vespa/searchcore/proton/server/updatedonecontext.cpp b/searchcore/src/vespa/searchcore/proton/server/updatedonecontext.cpp
index 9f9857165c1..3feaba15c00 100644
--- a/searchcore/src/vespa/searchcore/proton/server/updatedonecontext.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/updatedonecontext.cpp
@@ -7,8 +7,9 @@ using document::Document;
namespace proton {
-UpdateDoneContext::UpdateDoneContext(FeedToken token, const document::DocumentUpdate::SP &upd)
+UpdateDoneContext::UpdateDoneContext(FeedToken token, IPendingLidTracker::Token uncommitted, const document::DocumentUpdate::SP &upd)
: OperationDoneContext(std::move(token)),
+ _uncommitted(std::move(uncommitted)),
_upd(upd),
_doc()
{
diff --git a/searchcore/src/vespa/searchcore/proton/server/updatedonecontext.h b/searchcore/src/vespa/searchcore/proton/server/updatedonecontext.h
index 63a8a477424..ba49317922b 100644
--- a/searchcore/src/vespa/searchcore/proton/server/updatedonecontext.h
+++ b/searchcore/src/vespa/searchcore/proton/server/updatedonecontext.h
@@ -3,6 +3,7 @@
#pragma once
#include "operationdonecontext.h"
+#include <vespa/searchcore/proton/common/pendinglidtracker.h>
#include <vespa/document/update/documentupdate.h>
#include <future>
@@ -19,10 +20,11 @@ namespace proton {
*/
class UpdateDoneContext : public OperationDoneContext
{
+ IPendingLidTracker::Token _uncommitted;
document::DocumentUpdate::SP _upd;
std::shared_future<std::unique_ptr<const document::Document>> _doc;
public:
- UpdateDoneContext(FeedToken token, const document::DocumentUpdate::SP &upd);
+ UpdateDoneContext(FeedToken token, IPendingLidTracker::Token uncommitted, const document::DocumentUpdate::SP &upd);
~UpdateDoneContext() override;
const document::DocumentUpdate &getUpdate() { return *_upd; }
diff --git a/searchcore/src/vespa/searchcore/proton/server/visibilityhandler.cpp b/searchcore/src/vespa/searchcore/proton/server/visibilityhandler.cpp
index d8eb0bc6553..c582733a983 100644
--- a/searchcore/src/vespa/searchcore/proton/server/visibilityhandler.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/visibilityhandler.cpp
@@ -56,10 +56,10 @@ VisibilityHandler::commitAndWait()
}
void
-VisibilityHandler::commitAndWait(PendingLidTracker & , uint32_t ) {
+VisibilityHandler::commitAndWait(IPendingLidTracker &, uint32_t ) {
commitAndWait();
}
-void VisibilityHandler::commitAndWait(PendingLidTracker & , const std::vector<uint32_t> & ) {
+void VisibilityHandler::commitAndWait(IPendingLidTracker &, const std::vector<uint32_t> & ) {
commitAndWait();
}
diff --git a/searchcore/src/vespa/searchcore/proton/server/visibilityhandler.h b/searchcore/src/vespa/searchcore/proton/server/visibilityhandler.h
index 0bb5870e773..6e65b6f6257 100644
--- a/searchcore/src/vespa/searchcore/proton/server/visibilityhandler.h
+++ b/searchcore/src/vespa/searchcore/proton/server/visibilityhandler.h
@@ -30,8 +30,8 @@ public:
bool hasVisibilityDelay() const { return _visibilityDelay != vespalib::duration::zero(); }
void commit() override;
void commitAndWait() override;
- void commitAndWait(PendingLidTracker & , uint32_t ) override;
- void commitAndWait(PendingLidTracker & , const std::vector<uint32_t> & ) override;
+ void commitAndWait(IPendingLidTracker &, uint32_t ) override;
+ void commitAndWait(IPendingLidTracker &, const std::vector<uint32_t> & ) override;
private:
bool startCommit(const std::lock_guard<std::mutex> &unused, bool force);
void performCommit(bool force);
diff --git a/searchcore/src/vespa/searchcore/proton/test/dummy_feed_view.cpp b/searchcore/src/vespa/searchcore/proton/test/dummy_feed_view.cpp
index beb39c8e574..ba9dd7ecc39 100644
--- a/searchcore/src/vespa/searchcore/proton/test/dummy_feed_view.cpp
+++ b/searchcore/src/vespa/searchcore/proton/test/dummy_feed_view.cpp
@@ -17,8 +17,8 @@ DummyFeedView::DummyFeedView(std::shared_ptr<const document::DocumentTypeRepo> d
DummyFeedView::~DummyFeedView() = default;
-PendingLidTracker
-&DummyFeedView::getUncommittedLidsTracker() {
+IPendingLidTracker &
+DummyFeedView::getUncommittedLidsTracker() {
assert(false);
}
diff --git a/searchcore/src/vespa/searchcore/proton/test/dummy_feed_view.h b/searchcore/src/vespa/searchcore/proton/test/dummy_feed_view.h
index 0150f57ddef..122559f9e68 100644
--- a/searchcore/src/vespa/searchcore/proton/test/dummy_feed_view.h
+++ b/searchcore/src/vespa/searchcore/proton/test/dummy_feed_view.h
@@ -33,7 +33,7 @@ struct DummyFeedView : public IFeedView
void handlePruneRemovedDocuments(const PruneRemovedDocumentsOperation &) override {}
void handleCompactLidSpace(const CompactLidSpaceOperation &) override {}
void forceCommit(search::SerialNum) override { }
- PendingLidTracker &getUncommittedLidsTracker() override;
+ IPendingLidTracker & getUncommittedLidsTracker() override;
};
}