From ae3542f8ab39b66ffdbc100b2e914c87a90c465d Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Tue, 29 Jun 2021 01:26:05 +0000 Subject: [PATCH 01/63] add stream struct in Tlog --- fdbserver/TLogInterface.h | 42 ++++++++++++++++++++++++++++++++++ fdbserver/TLogServer.actor.cpp | 4 ++++ 2 files changed, 46 insertions(+) diff --git a/fdbserver/TLogInterface.h b/fdbserver/TLogInterface.h index e9e5b20b0d..202aa38e18 100644 --- a/fdbserver/TLogInterface.h +++ b/fdbserver/TLogInterface.h @@ -38,6 +38,7 @@ struct TLogInterface { UID sharedTLogID; RequestStream peekMessages; + RequestStream peekStreamMessages; // request establish a peek stream with the TLog server RequestStream popMessages; RequestStream commit; @@ -70,6 +71,7 @@ struct TLogInterface { void initEndpoints() { std::vector> streams; streams.push_back(peekMessages.getReceiver(TaskPriority::TLogPeek)); + streams.push_back(peekStreamMessages.getReceiver(TaskPriority::ReadSocket)); streams.push_back(popMessages.getReceiver(TaskPriority::TLogPop)); streams.push_back(commit.getReceiver(TaskPriority::TLogCommit)); streams.push_back(lock.getReceiver()); @@ -106,6 +108,7 @@ struct TLogInterface { enablePopRequest = RequestStream(peekMessages.getEndpoint().getAdjustedEndpoint(9)); snapRequest = RequestStream(peekMessages.getEndpoint().getAdjustedEndpoint(10)); + peekStreamMessages = RequestStream(peekMessages.getEndpoint().getAdjustedEndpoint(11)); } } }; @@ -209,6 +212,45 @@ struct TLogPeekRequest { } }; +struct TLogPeekStreamReply: public ReplyPromiseStreamReply { + constexpr static FileIdentifier file_identifier = 10072848; + Arena arena; + StringRef messages; + Version end; + Optional popped; + Version maxKnownVersion; + Version minKnownCommittedVersion; + Optional begin; + + int expectedSize() const { + return messages.expectedSize() + sizeof(TLogPeekStreamReply); + } + + template + void serialize(Ar& ar) { + serializer(ar, ReplyPromiseStreamReply::acknowledgeToken, arena, messages, end, popped, maxKnownVersion, + minKnownCommittedVersion, begin); + } +}; + +struct TLogPeekStreamRequest { + constexpr static FileIdentifier file_identifier = 10072821; + Arena arena; + Version begin; + Tag tag; + int limit, limitBytes; + ReplyPromiseStream reply; + + TLogPeekStreamRequest() {} + TLogPeekStreamRequest(Version version, Tag tag, int limit, int limitBytes) + : begin(version), tag(tag), limit(limit), limitBytes(limitBytes) {} + + template + void serialize(Ar& ar) { + serializer(ar, arena, begin, tag, limit, limitBytes, reply); + } +}; + struct TLogPopRequest { constexpr static FileIdentifier file_identifier = 5556423; Arena arena; diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 3d2d90c3f7..802bb3c7ec 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -2398,6 +2398,9 @@ ACTOR Future serveTLogInterface(TLogData* self, } else { logData->logSystem->set(Reference()); } + } + when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { + } when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) { logData->addActor.send(tLogPeekMessages(self, req, logData)); @@ -3100,6 +3103,7 @@ ACTOR Future tLogStart(TLogData* self, InitializeTLogRequest req, Locality recruited.initEndpoints(); DUMPTOKEN(recruited.peekMessages); + DUMPTOKEN(recruited.peekStreamMessages); DUMPTOKEN(recruited.popMessages); DUMPTOKEN(recruited.commit); DUMPTOKEN(recruited.lock); From b50fda6b4bee524cad6fecc35ddac8117f88f32a Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Thu, 1 Jul 2021 04:32:30 +0000 Subject: [PATCH 02/63] add simple streaming peek functions --- fdbclient/ServerKnobs.cpp | 3 +- fdbclient/ServerKnobs.h | 3 +- fdbserver/LogSystem.h | 2 + fdbserver/LogSystemPeekCursor.actor.cpp | 101 ++++++++++++++++++------ fdbserver/TLogInterface.h | 47 ++++++----- fdbserver/TLogServer.actor.cpp | 19 +++-- fdbserver/worker.actor.cpp | 1 + 7 files changed, 120 insertions(+), 56 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 8cbb77defb..e4dcd9d38a 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -66,7 +66,8 @@ void ServerKnobs::initialize(Randomize _randomize, ClientKnobs* clientKnobs, IsS init( TLOG_MESSAGE_BLOCK_BYTES, 10e6 ); init( TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR, double(TLOG_MESSAGE_BLOCK_BYTES) / (TLOG_MESSAGE_BLOCK_BYTES - MAX_MESSAGE_SIZE) ); //1.0121466709838096006362758832473 init( PEEK_TRACKER_EXPIRATION_TIME, 600 ); if( randomize && BUGGIFY ) PEEK_TRACKER_EXPIRATION_TIME = deterministicRandom()->coinflip() ? 0.1 : 120; - init( PARALLEL_GET_MORE_REQUESTS, 32 ); if( randomize && BUGGIFY ) PARALLEL_GET_MORE_REQUESTS = 2; + init( PEEK_USEING_STREAMING, true ); + init( PARALLEL_GET_MORE_REQUESTS, 32 ); if( randomize && BUGGIFY ) PARALLEL_GET_MORE_REQUESTS = 2; init( MULTI_CURSOR_PRE_FETCH_LIMIT, 10 ); init( MAX_QUEUE_COMMIT_BYTES, 15e6 ); if( randomize && BUGGIFY ) MAX_QUEUE_COMMIT_BYTES = 5000; init( DESIRED_OUTSTANDING_MESSAGES, 5000 ); if( randomize && BUGGIFY ) DESIRED_OUTSTANDING_MESSAGES = deterministicRandom()->randomInt(0,100); diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index 30468c7e84..d7dda18e82 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -40,7 +40,8 @@ public: // often, so that versions always advance smoothly // TLogs - double TLOG_TIMEOUT; // tlog OR commit proxy failure - master's reaction time + bool PEEK_USEING_STREAMING; + double TLOG_TIMEOUT; // tlog OR commit proxy failure - master's reaction time double TLOG_SLOW_REJOIN_WARN_TIMEOUT_SECS; // Warns if a tlog takes too long to rejoin double RECOVERY_TLOG_SMART_QUORUM_DELAY; // smaller might be better for bug amplification double TLOG_STORAGE_MIN_UPDATE_INTERVAL; diff --git a/fdbserver/LogSystem.h b/fdbserver/LogSystem.h index da2fbcf5f2..a8b7e10d75 100644 --- a/fdbserver/LogSystem.h +++ b/fdbserver/LogSystem.h @@ -433,9 +433,11 @@ struct ILogSystem { bool onlySpilled; bool parallelGetMore; + bool usePeekStream; int sequence; Deque> futureResults; Future interfaceChanged; + Optional> peekReplyStream; double lastReset; Future resetCheck; diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 26287919cd..89b7e63c97 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -25,6 +25,18 @@ #include "fdbrpc/ReplicationUtils.h" #include "flow/actorcompiler.h" // has to be last include +// create a peek stream for cursor when it's possible +void tryEstablishPeekStream(ILogSystem::ServerPeekCursor* self) { + if (self->peekReplyStream.present()) + return; + else if (!self->interf || !self->interf->get().present()) { + self->peekReplyStream.reset(); + return; + } + self->peekReplyStream = self->interf->get().interf().peekStreamMessages.getReplyStream( + TLogPeekStreamRequest(self->messageVersion.version, self->tag, std::numeric_limits::max())); +} + ILogSystem::ServerPeekCursor::ServerPeekCursor(Reference>> const& interf, Tag tag, Version begin, @@ -35,9 +47,12 @@ ILogSystem::ServerPeekCursor::ServerPeekCursor(ReferencerandomUniqueID()), poppedVersion(0), returnIfBlocked(returnIfBlocked), sequence(0), onlySpilled(false), parallelGetMore(parallelGetMore), lastReset(0), slowReplies(0), fastReplies(0), unknownReplies(0), - resetCheck(Void()) { + resetCheck(Void()), usePeekStream(SERVER_KNOBS->PEEK_USEING_STREAMING) { this->results.maxKnownVersion = 0; this->results.minKnownCommittedVersion = 0; + if (usePeekStream) { + tryEstablishPeekStream(this); + } //TraceEvent("SPC_Starting", randomID).detail("Tag", tag.toString()).detail("Begin", begin).detail("End", end).backtrace(); } @@ -51,7 +66,8 @@ ILogSystem::ServerPeekCursor::ServerPeekCursor(TLogPeekReply const& results, : results(results), tag(tag), rd(results.arena, results.messages, Unversioned()), messageVersion(messageVersion), end(end), messageAndTags(message), hasMsg(hasMsg), randomID(deterministicRandom()->randomUniqueID()), poppedVersion(poppedVersion), returnIfBlocked(false), sequence(0), onlySpilled(false), parallelGetMore(false), - lastReset(0), slowReplies(0), fastReplies(0), unknownReplies(0), resetCheck(Void()) { + lastReset(0), slowReplies(0), fastReplies(0), unknownReplies(0), resetCheck(Void()), + usePeekStream(SERVER_KNOBS->PEEK_USEING_STREAMING) { //TraceEvent("SPC_Clone", randomID); this->results.maxKnownVersion = 0; this->results.minKnownCommittedVersion = 0; @@ -154,6 +170,20 @@ void ILogSystem::ServerPeekCursor::advanceTo(LogMessageVersion n) { } } +// This function is called after the cursor received one TLogPeekReply to update its members, which is the common logic +// in getMore helper functions. +void updateCursorWithReply(ILogSystem::ServerPeekCursor* self, const TLogPeekReply& res) { + self->results = res; + self->onlySpilled = res.onlySpilled; + if (res.popped.present()) + self->poppedVersion = std::min(std::max(self->poppedVersion, res.popped.get()), self->end.version); + self->rd = ArenaReader(self->results.arena, self->results.messages, Unversioned()); + LogMessageVersion skipSeq = self->messageVersion; + self->hasMsg = true; + self->nextMessage(); + self->advanceTo(skipSeq); +} + ACTOR Future resetChecker(ILogSystem::ServerPeekCursor* self, NetworkAddress addr) { self->slowReplies = 0; self->unknownReplies = 0; @@ -209,7 +239,7 @@ ACTOR Future recordRequestMetrics(ILogSystem::ServerPeekCursor* s } ACTOR Future serverPeekParallelGetMore(ILogSystem::ServerPeekCursor* self, TaskPriority taskID) { - if (!self->interf || self->messageVersion >= self->end) { + if (!self->interf || self->isExhausted()) { if (self->hasMessage()) return Void(); wait(Future(Never())); @@ -254,16 +284,7 @@ ACTOR Future serverPeekParallelGetMore(ILogSystem::ServerPeekCursor* self, } expectedBegin = res.end; self->futureResults.pop_front(); - self->results = res; - self->onlySpilled = res.onlySpilled; - if (res.popped.present()) - self->poppedVersion = - std::min(std::max(self->poppedVersion, res.popped.get()), self->end.version); - self->rd = ArenaReader(self->results.arena, self->results.messages, Unversioned()); - LogMessageVersion skipSeq = self->messageVersion; - self->hasMsg = true; - self->nextMessage(); - self->advanceTo(skipSeq); + updateCursorWithReply(self, res); //TraceEvent("SPC_GetMoreB", self->randomID).detail("Has", self->hasMessage()).detail("End", res.end).detail("Popped", res.popped.present() ? res.popped.get() : 0); return Void(); } @@ -297,8 +318,49 @@ ACTOR Future serverPeekParallelGetMore(ILogSystem::ServerPeekCursor* self, } } +ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, TaskPriority taskID) { + if (self->isExhausted()) { + if (self->hasMessage()) + return Void(); + wait(Future(Never())); + throw internal_error(); + } + + try { + tryEstablishPeekStream(self); + loop { + try { + choose { + when(wait(self->interf->onChange())) { + self->onlySpilled = false; + self->peekReplyStream.reset(); + tryEstablishPeekStream(self); + } + when(TLogPeekStreamReply res = wait(self->peekReplyStream.present() + ? waitAndForward(self->peekReplyStream.get().getFuture()) + : Never())) { + updateCursorWithReply(self, res.rep); + return Void(); + } + } + } catch (Error& e) { + if (e.code() == error_code_connection_failed) { + self->peekReplyStream.reset(); + } + throw; + } + } + } catch (Error& e) { + if (e.code() == error_code_end_of_stream) { + self->end.reset(self->messageVersion.version); + return Void(); + } + throw; + } +} + ACTOR Future serverPeekGetMore(ILogSystem::ServerPeekCursor* self, TaskPriority taskID) { - if (!self->interf || self->messageVersion >= self->end) { + if (!self->interf || self->isExhausted()) { wait(Future(Never())); throw internal_error(); } @@ -314,16 +376,7 @@ ACTOR Future serverPeekGetMore(ILogSystem::ServerPeekCursor* self, TaskPri self->onlySpilled), taskID)) : Never())) { - self->results = res; - self->onlySpilled = res.onlySpilled; - if (res.popped.present()) - self->poppedVersion = - std::min(std::max(self->poppedVersion, res.popped.get()), self->end.version); - self->rd = ArenaReader(self->results.arena, self->results.messages, Unversioned()); - LogMessageVersion skipSeq = self->messageVersion; - self->hasMsg = true; - self->nextMessage(); - self->advanceTo(skipSeq); + updateCursorWithReply(self, res); //TraceEvent("SPC_GetMoreB", self->randomID).detail("Has", self->hasMessage()).detail("End", res.end).detail("Popped", res.popped.present() ? res.popped.get() : 0); return Void(); } diff --git a/fdbserver/TLogInterface.h b/fdbserver/TLogInterface.h index 202aa38e18..ba8bc10d64 100644 --- a/fdbserver/TLogInterface.h +++ b/fdbserver/TLogInterface.h @@ -38,7 +38,8 @@ struct TLogInterface { UID sharedTLogID; RequestStream peekMessages; - RequestStream peekStreamMessages; // request establish a peek stream with the TLog server + RequestStream + peekStreamMessages; // request establish a peek stream with the TLog server RequestStream popMessages; RequestStream commit; @@ -69,9 +70,9 @@ struct TLogInterface { NetworkAddressList addresses() const { return peekMessages.getEndpoint().addresses; } void initEndpoints() { + // NOTE: the adding order should be the same as the hardcoded indices in serialize() std::vector> streams; streams.push_back(peekMessages.getReceiver(TaskPriority::TLogPeek)); - streams.push_back(peekStreamMessages.getReceiver(TaskPriority::ReadSocket)); streams.push_back(popMessages.getReceiver(TaskPriority::TLogPop)); streams.push_back(commit.getReceiver(TaskPriority::TLogCommit)); streams.push_back(lock.getReceiver()); @@ -82,7 +83,8 @@ struct TLogInterface { streams.push_back(disablePopRequest.getReceiver()); streams.push_back(enablePopRequest.getReceiver()); streams.push_back(snapRequest.getReceiver()); - FlowTransport::transport().addEndpoints(streams); + streams.push_back(peekStreamMessages.getReceiver(TaskPriority::ReadSocket)); + FlowTransport::transport().addEndpoints(streams); } template @@ -108,7 +110,8 @@ struct TLogInterface { enablePopRequest = RequestStream(peekMessages.getEndpoint().getAdjustedEndpoint(9)); snapRequest = RequestStream(peekMessages.getEndpoint().getAdjustedEndpoint(10)); - peekStreamMessages = RequestStream(peekMessages.getEndpoint().getAdjustedEndpoint(11)); + peekStreamMessages = + RequestStream(peekMessages.getEndpoint().getAdjustedEndpoint(11)); } } }; @@ -212,25 +215,19 @@ struct TLogPeekRequest { } }; -struct TLogPeekStreamReply: public ReplyPromiseStreamReply { - constexpr static FileIdentifier file_identifier = 10072848; - Arena arena; - StringRef messages; - Version end; - Optional popped; - Version maxKnownVersion; - Version minKnownCommittedVersion; - Optional begin; +struct TLogPeekStreamReply : public ReplyPromiseStreamReply { + constexpr static FileIdentifier file_identifier = 10072848; + TLogPeekReply rep; - int expectedSize() const { - return messages.expectedSize() + sizeof(TLogPeekStreamReply); + TLogPeekStreamReply() = default; + explicit TLogPeekStreamReply(const TLogPeekReply& rep) : rep(rep) {} + + int expectedSize() const { return rep.messages.expectedSize() + sizeof(TLogPeekStreamReply); } + + template + void serialize(Ar& ar) { + serializer(ar, ReplyPromiseStreamReply::acknowledgeToken, rep); } - - template - void serialize(Ar& ar) { - serializer(ar, ReplyPromiseStreamReply::acknowledgeToken, arena, messages, end, popped, maxKnownVersion, - minKnownCommittedVersion, begin); - } }; struct TLogPeekStreamRequest { @@ -238,16 +235,16 @@ struct TLogPeekStreamRequest { Arena arena; Version begin; Tag tag; - int limit, limitBytes; + int limitBytes; ReplyPromiseStream reply; TLogPeekStreamRequest() {} - TLogPeekStreamRequest(Version version, Tag tag, int limit, int limitBytes) - : begin(version), tag(tag), limit(limit), limitBytes(limitBytes) {} + TLogPeekStreamRequest(Version version, Tag tag, int limitBytes) + : begin(version), tag(tag), limitBytes(limitBytes) {} template void serialize(Ar& ar) { - serializer(ar, arena, begin, tag, limit, limitBytes, reply); + serializer(ar, arena, begin, tag, limitBytes, reply); } }; diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 802bb3c7ec..6ab8e2568c 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1568,6 +1568,14 @@ ACTOR Future> parseMessagesForTag(StringRef commitBlob, T return relevantMessages; } +ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { + if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { + req.tag.id = req.tag.id % logData->txsTags; + } + req.reply.setByteLimit(SERVER_KNOBS->MAXIMUM_PEEK_BYTES); + return Void(); +} + ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { state BinaryWriter messages(Unversioned()); state BinaryWriter messages2(Unversioned()); @@ -2400,7 +2408,7 @@ ACTOR Future serveTLogInterface(TLogData* self, } } when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { - + logData->addActor.send(tLogPeekStream(self, req, logData)); } when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) { logData->addActor.send(tLogPeekMessages(self, req, logData)); @@ -2658,7 +2666,7 @@ ACTOR Future tLogCore(TLogData* self, SERVER_KNOBS->STORAGE_LOGGING_DELAY, &logData->cc, logData->logId.toString() + "/TLogMetrics", - [self=self](TraceEvent& te) { + [self = self](TraceEvent& te) { StorageBytes sbTlog = self->persistentData->getStorageBytes(); te.detail("KvstoreBytesUsed", sbTlog.used); te.detail("KvstoreBytesFree", sbTlog.free); @@ -2842,6 +2850,7 @@ ACTOR Future restorePersistentState(TLogData* self, recruited.initEndpoints(); DUMPTOKEN(recruited.peekMessages); + DUMPTOKEN(recruited.peekStreamMessages); DUMPTOKEN(recruited.popMessages); DUMPTOKEN(recruited.commit); DUMPTOKEN(recruited.lock); @@ -2888,9 +2897,9 @@ ACTOR Future restorePersistentState(TLogData* self, logsByVersion.emplace_back(ver, id1); TraceEvent("TLogPersistentStateRestore", self->dbgid) - .detail("LogId", logData->logId) - .detail("Ver", ver) - .detail("RecoveryCount", logData->recoveryCount); + .detail("LogId", logData->logId) + .detail("Ver", ver) + .detail("RecoveryCount", logData->recoveryCount); // Restore popped keys. Pop operations that took place after the last (committed) updatePersistentDataVersion // might be lost, but that is fine because we will get the corresponding data back, too. tagKeys = prefixRange(rawId.withPrefix(persistTagPoppedKeys.begin)); diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index 6d107cff7f..8ff6208db6 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -1829,6 +1829,7 @@ ACTOR Future workerServer(Reference connFile, startRole(Role::LOG_ROUTER, recruited.id(), interf.id(), details); DUMPTOKEN(recruited.peekMessages); + DUMPTOKEN(recruited.peekStreamMessages); DUMPTOKEN(recruited.popMessages); DUMPTOKEN(recruited.commit); DUMPTOKEN(recruited.lock); From 9948b9d4ef39b39ced9bbec8c57b41eb213a82bd Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Mon, 5 Jul 2021 00:14:27 +0000 Subject: [PATCH 03/63] refactor TLog Peek code --- fdbserver/TLogServer.actor.cpp | 245 ++++++++++++++++++--------------- 1 file changed, 133 insertions(+), 112 deletions(-) diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 6ab8e2568c..6e92e31c0c 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1507,15 +1507,16 @@ std::deque>& getVersionMessages(Refe }; void peekMessagesFromMemory(Reference self, - TLogPeekRequest const& req, + Tag tag, + Version begin, BinaryWriter& messages, Version& endVersion) { ASSERT(!messages.getLength()); - auto& deque = getVersionMessages(self, req.tag); + auto& deque = getVersionMessages(self, tag); //TraceEvent("TLogPeekMem", self->dbgid).detail("Tag", req.tag1).detail("PDS", self->persistentDataSequence).detail("PDDS", self->persistentDataDurableSequence).detail("Oldest", map1.empty() ? 0 : map1.begin()->key ).detail("OldestMsgCount", map1.empty() ? 0 : map1.begin()->value.size()); - Version begin = std::max(req.begin, self->persistentDataDurableVersion + 1); + begin = std::max(begin, self->persistentDataDurableVersion + 1); auto it = std::lower_bound(deque.begin(), deque.end(), std::make_pair(begin, LengthPrefixedStringRef()), @@ -1542,7 +1543,7 @@ void peekMessagesFromMemory(Reference self, DEBUG_TAGS_AND_MESSAGE( "TLogPeek", currentVersion, StringRef((uint8_t*)data + offset, messages.getLength() - offset)) .detail("LogId", self->logId) - .detail("PeekTag", req.tag); + .detail("PeekTag", tag); } } @@ -1568,105 +1569,97 @@ ACTOR Future> parseMessagesForTag(StringRef commitBlob, T return relevantMessages; } -ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { - if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { - req.tag.id = req.tag.id % logData->txsTags; - } - req.reply.setByteLimit(SERVER_KNOBS->MAXIMUM_PEEK_BYTES); - return Void(); -} - -ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { +// Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request +ACTOR Future peekTLog( + TLogData* self, + Reference logData, + Version begin, + Tag tag, + bool returnIfBlocked = false, + bool reqOnlySpilled = false, + Optional> sequence = Optional>()) { state BinaryWriter messages(Unversioned()); state BinaryWriter messages2(Unversioned()); - state int sequence = -1; + state int sequenceNum = -1; state UID peekId; state double queueStart = now(); - if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { - req.tag.id = req.tag.id % logData->txsTags; + if (tag.locality == tagLocalityTxs && tag.id >= logData->txsTags && logData->txsTags > 0) { + tag.id = tag.id % logData->txsTags; } - if (req.sequence.present()) { - try { - peekId = req.sequence.get().first; - sequence = req.sequence.get().second; - if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - logData->peekTracker.find(peekId) == logData->peekTracker.end()) { - throw operation_obsolete(); - } - auto& trackerData = logData->peekTracker[peekId]; - if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - trackerData.tag = req.tag; - trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled)); - } - auto seqBegin = trackerData.sequence_version.begin(); - // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. - while (trackerData.sequence_version.size() && - seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { - if (seqBegin->second.canBeSet()) { - seqBegin->second.sendError(operation_obsolete()); - } - trackerData.sequence_version.erase(seqBegin); - seqBegin = trackerData.sequence_version.begin(); - } - - if (trackerData.sequence_version.size() && sequence < seqBegin->first) { - throw operation_obsolete(); - } - - Future> fPrevPeekData = trackerData.sequence_version[sequence].getFuture(); - if (fPrevPeekData.isReady()) { - trackerData.unblockedPeeks++; - double t = now() - trackerData.lastUpdate; - if (t > trackerData.idleMax) - trackerData.idleMax = t; - trackerData.idleTime += t; - } - trackerData.lastUpdate = now(); - std::pair prevPeekData = wait(fPrevPeekData); - req.begin = std::max(prevPeekData.first, req.begin); - req.onlySpilled = prevPeekData.second; - wait(yield()); - } catch (Error& e) { - if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { - req.reply.sendError(e); - return Void(); - } else { - throw; - } + // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence requests + // STEP: a. mark obsolete sequence requests; b. wait previous sequence requests are handled in order + if (sequence.present()) { + peekId = sequence.get().first; + sequenceNum = sequence.get().second; + if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + logData->peekTracker.find(peekId) == logData->peekTracker.end()) { + throw operation_obsolete(); } + auto& trackerData = logData->peekTracker[peekId]; + if (sequenceNum == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { + trackerData.tag = tag; + trackerData.sequence_version[0].send(std::make_pair(begin, reqOnlySpilled)); + } + auto seqBegin = trackerData.sequence_version.begin(); + // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. + while (trackerData.sequence_version.size() && + seqBegin->first <= sequenceNum - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + if (seqBegin->second.canBeSet()) { + seqBegin->second.sendError(operation_obsolete()); + } + trackerData.sequence_version.erase(seqBegin); + seqBegin = trackerData.sequence_version.begin(); + } + + if (trackerData.sequence_version.size() && sequenceNum < seqBegin->first) { + throw operation_obsolete(); + } + + Future> fPrevPeekData = trackerData.sequence_version[sequenceNum].getFuture(); + if (fPrevPeekData.isReady()) { + trackerData.unblockedPeeks++; + double t = now() - trackerData.lastUpdate; + if (t > trackerData.idleMax) + trackerData.idleMax = t; + trackerData.idleTime += t; + } + trackerData.lastUpdate = now(); + std::pair prevPeekData = wait(fPrevPeekData); + begin = std::max(prevPeekData.first, begin); + reqOnlySpilled = prevPeekData.second; + wait(yield()); } state double blockStart = now(); - if (req.returnIfBlocked && logData->version.get() < req.begin) { - req.reply.sendError(end_of_stream()); - if (req.sequence.present()) { + if (returnIfBlocked && logData->version.get() < begin) { + if (sequence.present()) { auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; trackerData.lastUpdate = now(); if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); + sequenceData.send(std::make_pair(begin, reqOnlySpilled)); } } - return Void(); + throw end_of_stream(); } //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); // Wait until we have something to return that the caller doesn't already have - if (logData->version.get() < req.begin) { - wait(logData->version.whenAtLeast(req.begin)); + if (logData->version.get() < begin) { + wait(logData->version.whenAtLeast(begin)); wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); } - if (logData->locality != tagLocalitySatellite && req.tag.locality == tagLocalityLogRouter) { + if (logData->locality != tagLocalitySatellite && tag.locality == tagLocalityLogRouter) { wait(self->concurrentLogRouterReads.take()); state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); wait(delay(0.0, TaskPriority::Low)); } - if (req.begin <= logData->persistentDataDurableVersion && req.tag.locality != tagLocalityTxs && req.tag != txsTag) { + if (begin <= logData->persistentDataDurableVersion && tag.locality != tagLocalityTxs && tag != txsTag) { // Reading spilled data will almost always imply that the storage server is >5s behind the rest // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up // slightly faster over keeping the rest of the cluster operating normally. @@ -1677,8 +1670,9 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen state double workStart = now(); - Version poppedVer = poppedVersion(logData, req.tag); - if (poppedVer > req.begin) { + Version poppedVer = poppedVersion(logData, tag); + if (poppedVer > begin) { + // reply with an empty message and let the next reply start from poppedVer TLogPeekReply rep; rep.maxKnownVersion = logData->version.get(); rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; @@ -1686,30 +1680,28 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen rep.end = poppedVer; rep.onlySpilled = false; - if (req.sequence.present()) { + // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence requests. + if (sequence.present()) { auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; trackerData.lastUpdate = now(); - if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - req.reply.sendError(operation_obsolete()); + if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { if (!sequenceData.isSet()) sequenceData.sendError(operation_obsolete()); - return Void(); + throw operation_obsolete(); } if (sequenceData.isSet()) { if (sequenceData.getFuture().get().first != rep.end) { - TEST(true); // tlog peek second attempt ended at a different version - req.reply.sendError(operation_obsolete()); - return Void(); + TEST(true); // 1 tlog peek second attempt ended at a different version + throw operation_obsolete(); } } else { sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); } - rep.begin = req.begin; + rep.begin = begin; } - req.reply.send(rep); - return Void(); + return rep; } state Version endVersion = logData->version.get() + 1; @@ -1717,23 +1709,23 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen // grab messages from disk //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - if (req.begin <= logData->persistentDataDurableVersion) { + if (begin <= logData->persistentDataDurableVersion) { // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if // an initial attempt to read from disk results in insufficient data and the required data is no longer in // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the // result? - if (req.onlySpilled) { + if (reqOnlySpilled) { endVersion = logData->persistentDataDurableVersion + 1; } else { - peekMessagesFromMemory(logData, req, messages2, endVersion); + peekMessagesFromMemory(logData, tag, begin, messages2, endVersion); } - if (logData->shouldSpillByValue(req.tag)) { + if (logData->shouldSpillByValue(tag)) { RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, req.tag, req.begin), - persistTagMessagesKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)), + KeyRangeRef(persistTagMessagesKey(logData->logId, tag, begin), + persistTagMessagesKey(logData->logId, tag, logData->persistentDataDurableVersion + 1)), SERVER_KNOBS->DESIRED_TOTAL_BYTES, SERVER_KNOBS->DESIRED_TOTAL_BYTES)); @@ -1752,9 +1744,8 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen } else { // FIXME: Limit to approximately DESIRED_TOTATL_BYTES somehow. RangeResult kvrefs = wait(self->persistentData->readRange( - KeyRangeRef( - persistTagMessageRefsKey(logData->logId, req.tag, req.begin), - persistTagMessageRefsKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)), + KeyRangeRef(persistTagMessageRefsKey(logData->logId, tag, begin), + persistTagMessageRefsKey(logData->logId, tag, logData->persistentDataDurableVersion + 1)), SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1)); //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); @@ -1774,7 +1765,7 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen earlyEnd = true; break; } - if (sd.version >= req.begin) { + if (sd.version >= begin) { firstVersion = std::min(firstVersion, sd.version); const IDiskQueue::location end = sd.start.lo + sd.length; commitLocations.emplace_back(sd.start, end); @@ -1816,13 +1807,13 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen messages << VERSION_HEADER << entry.version; std::vector rawMessages = - wait(parseMessagesForTag(entry.messages, req.tag, logData->logRouterTags)); + wait(parseMessagesForTag(entry.messages, tag, logData->logRouterTags)); for (const StringRef& msg : rawMessages) { messages.serializeBytes(msg); DEBUG_TAGS_AND_MESSAGE("TLogPeekFromDisk", entry.version, msg) .detail("UID", self->dbgid) .detail("LogId", logData->logId) - .detail("PeekTag", req.tag); + .detail("PeekTag", tag); } lastRefMessageVersion = entry.version; @@ -1840,10 +1831,10 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen } } } else { - if (req.onlySpilled) { + if (reqOnlySpilled) { endVersion = logData->persistentDataDurableVersion + 1; } else { - peekMessagesFromMemory(logData, req, messages, endVersion); + peekMessagesFromMemory(logData, tag, begin, messages, endVersion); } //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); @@ -1852,7 +1843,7 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen TLogPeekReply reply; reply.maxKnownVersion = logData->version.get(); reply.minKnownCommittedVersion = logData->minKnownCommittedVersion; - reply.messages = messages.toValue(); + reply.messages = StringRef(reply.arena, messages.toValue()); reply.end = endVersion; reply.onlySpilled = onlySpilled; @@ -1861,7 +1852,7 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen // detail("MsgBytes", reply.messages.expectedSize()). // detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); - if (req.sequence.present()) { + if (sequence.present()) { auto& trackerData = logData->peekTracker[peekId]; trackerData.lastUpdate = now(); @@ -1883,9 +1874,8 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen trackerData.blockTime += blockT; trackerData.workTime += workT; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; - if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - req.reply.sendError(operation_obsolete()); + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { if (!sequenceData.isSet()) { // It would technically be more correct to .send({req.begin, req.onlySpilled}), as the next // request might still be in the window of active requests, but LogSystemPeekCursor will @@ -1893,22 +1883,53 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen // response will probably be a waste of CPU. sequenceData.sendError(operation_obsolete()); } - return Void(); + throw operation_obsolete(); } if (sequenceData.isSet()) { trackerData.duplicatePeeks++; if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // tlog peek second attempt ended at a different version (2) - req.reply.sendError(operation_obsolete()); - return Void(); + TEST(true); // 1 tlog peek second attempt ended at a different version (2) + throw operation_obsolete(); } } else { sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); } - reply.begin = req.begin; + reply.begin = begin; + } + + return reply; +} + +// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover +ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { + if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { + req.tag.id = req.tag.id % logData->txsTags; + } + req.reply.setByteLimit(SERVER_KNOBS->MAXIMUM_PEEK_BYTES); + // loop { wait(req.reply.onReady()); }; + return Void(); +} + +ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { + state BinaryWriter messages(Unversioned()); + state BinaryWriter messages2(Unversioned()); + state int sequence = -1; + state UID peekId; + state double queueStart = now(); + + try { + TLogPeekReply reply = wait(peekTLog(self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); + req.reply.send(reply); + } catch (Error& e) { + if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || + e.code() == error_code_end_of_stream) { + req.reply.sendError(e); + return Void(); + } else { + throw; + } } - req.reply.send(reply); return Void(); } From b6d5c8a091b3fd662611a9ca69128d7c09722b76 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Mon, 5 Jul 2021 05:57:24 +0000 Subject: [PATCH 04/63] implement tLogPeekStream --- fdbserver/LogSystem.h | 2 +- fdbserver/LogSystemPeekCursor.actor.cpp | 38 +++++++++++++------------ fdbserver/TLogServer.actor.cpp | 23 +++++++++++++-- 3 files changed, 41 insertions(+), 22 deletions(-) diff --git a/fdbserver/LogSystem.h b/fdbserver/LogSystem.h index a8b7e10d75..d6d1190b84 100644 --- a/fdbserver/LogSystem.h +++ b/fdbserver/LogSystem.h @@ -423,7 +423,7 @@ struct ILogSystem { TLogPeekReply results; ArenaReader rd; - LogMessageVersion messageVersion, end; + LogMessageVersion messageVersion, end; // the version of current message; the intended end version of current cursor Version poppedVersion; TagsAndMessage messageAndTags; bool hasMsg; diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 89b7e63c97..64d1026ea8 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -50,9 +50,6 @@ ILogSystem::ServerPeekCursor::ServerPeekCursor(ReferencePEEK_USEING_STREAMING) { this->results.maxKnownVersion = 0; this->results.minKnownCommittedVersion = 0; - if (usePeekStream) { - tryEstablishPeekStream(this); - } //TraceEvent("SPC_Starting", randomID).detail("Tag", tag.toString()).detail("Begin", begin).detail("End", end).backtrace(); } @@ -66,8 +63,7 @@ ILogSystem::ServerPeekCursor::ServerPeekCursor(TLogPeekReply const& results, : results(results), tag(tag), rd(results.arena, results.messages, Unversioned()), messageVersion(messageVersion), end(end), messageAndTags(message), hasMsg(hasMsg), randomID(deterministicRandom()->randomUniqueID()), poppedVersion(poppedVersion), returnIfBlocked(false), sequence(0), onlySpilled(false), parallelGetMore(false), - lastReset(0), slowReplies(0), fastReplies(0), unknownReplies(0), resetCheck(Void()), - usePeekStream(SERVER_KNOBS->PEEK_USEING_STREAMING) { + lastReset(0), slowReplies(0), fastReplies(0), unknownReplies(0), resetCheck(Void()), usePeekStream(false) { //TraceEvent("SPC_Clone", randomID); this->results.maxKnownVersion = 0; this->results.minKnownCommittedVersion = 0; @@ -319,12 +315,12 @@ ACTOR Future serverPeekParallelGetMore(ILogSystem::ServerPeekCursor* self, } ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, TaskPriority taskID) { - if (self->isExhausted()) { - if (self->hasMessage()) - return Void(); - wait(Future(Never())); - throw internal_error(); - } + if (self->isExhausted()) { + if (self->hasMessage()) + return Void(); + wait(Future(Never())); + throw internal_error(); + } try { tryEstablishPeekStream(self); @@ -336,9 +332,10 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T self->peekReplyStream.reset(); tryEstablishPeekStream(self); } - when(TLogPeekStreamReply res = wait(self->peekReplyStream.present() - ? waitAndForward(self->peekReplyStream.get().getFuture()) - : Never())) { + when(TLogPeekStreamReply res = + wait(self->peekReplyStream.present() + ? brokenPromiseToNever(waitAndForward(self->peekReplyStream.get().getFuture())) + : Never())) { updateCursorWithReply(self, res.rep); return Void(); } @@ -346,15 +343,18 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T } catch (Error& e) { if (e.code() == error_code_connection_failed) { self->peekReplyStream.reset(); + } else { + throw; } - throw; } } } catch (Error& e) { - if (e.code() == error_code_end_of_stream) { + if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { self->end.reset(self->messageVersion.version); + self->peekReplyStream.reset(); return Void(); } + self->peekReplyStream.reset(); throw; } } @@ -397,7 +397,9 @@ Future ILogSystem::ServerPeekCursor::getMore(TaskPriority taskID) { if (hasMessage() && !parallelGetMore) return Void(); if (!more.isValid() || more.isReady()) { - if (parallelGetMore || onlySpilled || futureResults.size()) { + if (usePeekStream) { + more = serverPeekStreamGetMore(this, taskID); + } else if (parallelGetMore || onlySpilled || futureResults.size()) { more = serverPeekParallelGetMore(this, taskID); } else { more = serverPeekGetMore(this, taskID); @@ -427,7 +429,7 @@ Future ILogSystem::ServerPeekCursor::onFailed() { bool ILogSystem::ServerPeekCursor::isActive() const { if (!interf->get().present()) return false; - if (messageVersion >= end) + if (isExhausted()) return false; return IFailureMonitor::failureMonitor().getState(interf->get().interf().peekMessages.getEndpoint()).isAvailable(); } diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 6e92e31c0c..8f512572cc 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1902,12 +1902,29 @@ ACTOR Future peekTLog( // This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { + state Version begin = req.begin; + state bool onlySpilled = false; if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { req.tag.id = req.tag.id % logData->txsTags; } - req.reply.setByteLimit(SERVER_KNOBS->MAXIMUM_PEEK_BYTES); - // loop { wait(req.reply.onReady()); }; - return Void(); + req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); + loop { + state TLogPeekStreamReply reply; + try { + wait(req.reply.onReady() && store(reply.rep, peekTLog(self, logData, begin, req.tag, false, onlySpilled))); + req.reply.send(reply); + begin = reply.rep.end; + onlySpilled = reply.rep.onlySpilled; + + wait(delay(0.005, TaskPriority::TLogPeekReply)); + } catch (Error& e) { + if(e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { + req.reply.sendError(e); + return Void(); + } + throw; + } + } } ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { From 15347773d90ea2e5829196254d56e02376934e57 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Wed, 7 Jul 2021 22:55:49 +0000 Subject: [PATCH 05/63] fix double destruction memory bug --- fdbrpc/genericactors.actor.h | 2 +- fdbserver/LogSystemPeekCursor.actor.cpp | 65 +++++++++++-------------- 2 files changed, 29 insertions(+), 38 deletions(-) diff --git a/fdbrpc/genericactors.actor.h b/fdbrpc/genericactors.actor.h index 23bd3e97c2..46a79d29cf 100644 --- a/fdbrpc/genericactors.actor.h +++ b/fdbrpc/genericactors.actor.h @@ -197,7 +197,7 @@ struct PeerHolder { } }; -// Implements getRepyStream, this a void actor with the same lifetime as the input ReplyPromiseStream. +// Implements getReplyStream, this a void actor with the same lifetime as the input ReplyPromiseStream. // Because this actor holds a reference to the stream, normally it would be impossible to know when there are no other // references. To get around this, there is a SAV inside the stream that has one less promise reference than it should // (caused by getErrorFutureAndDelPromiseRef()). When that SAV gets a broken promise because no one besides this void diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 64d1026ea8..c13f9dfac8 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -238,8 +238,7 @@ ACTOR Future serverPeekParallelGetMore(ILogSystem::ServerPeekCursor* self, if (!self->interf || self->isExhausted()) { if (self->hasMessage()) return Void(); - wait(Future(Never())); - throw internal_error(); + return Never(); } if (!self->interfaceChanged.isValid()) { @@ -315,54 +314,46 @@ ACTOR Future serverPeekParallelGetMore(ILogSystem::ServerPeekCursor* self, } ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, TaskPriority taskID) { - if (self->isExhausted()) { + if (!self->interf || self->isExhausted()) { if (self->hasMessage()) return Void(); - wait(Future(Never())); - throw internal_error(); + return Never(); } - try { - tryEstablishPeekStream(self); - loop { - try { - choose { - when(wait(self->interf->onChange())) { - self->onlySpilled = false; - self->peekReplyStream.reset(); - tryEstablishPeekStream(self); - } - when(TLogPeekStreamReply res = - wait(self->peekReplyStream.present() - ? brokenPromiseToNever(waitAndForward(self->peekReplyStream.get().getFuture())) - : Never())) { - updateCursorWithReply(self, res.rep); - return Void(); - } - } - } catch (Error& e) { - if (e.code() == error_code_connection_failed) { + tryEstablishPeekStream(self); + loop { + try { + choose { + when(wait(self->interf->onChange())) { + self->onlySpilled = false; self->peekReplyStream.reset(); - } else { - throw; + tryEstablishPeekStream(self); + } + when(TLogPeekStreamReply res = wait(self->peekReplyStream.present() + ? brokenPromiseToNever(waitAndForward(self->peekReplyStream.get().getFuture())) + : Never())) { + updateCursorWithReply(self, res.rep); + return Void(); } } + } catch (Error& e) { + if (e.code() == error_code_connection_failed) { + self->peekReplyStream.reset(); + } + else if(e.code() == error_code_end_of_stream) { + self->end.reset(self->messageVersion.version); + return Void(); + } + else { + throw; + } } - } catch (Error& e) { - if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { - self->end.reset(self->messageVersion.version); - self->peekReplyStream.reset(); - return Void(); - } - self->peekReplyStream.reset(); - throw; } } ACTOR Future serverPeekGetMore(ILogSystem::ServerPeekCursor* self, TaskPriority taskID) { if (!self->interf || self->isExhausted()) { - wait(Future(Never())); - throw internal_error(); + return Never(); } try { loop { From 5a43a8c367836746d5706dcc1c38d7654a901c88 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Thu, 8 Jul 2021 19:32:58 +0000 Subject: [PATCH 06/63] add returnIfBlocked in stream request --- fdbserver/LogSystemPeekCursor.actor.cpp | 10 +++---- fdbserver/TLogInterface.h | 7 ++--- fdbserver/TLogServer.actor.cpp | 36 ++++++++++++------------- 3 files changed, 25 insertions(+), 28 deletions(-) diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index c13f9dfac8..ce7bc5aa1c 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -33,8 +33,8 @@ void tryEstablishPeekStream(ILogSystem::ServerPeekCursor* self) { self->peekReplyStream.reset(); return; } - self->peekReplyStream = self->interf->get().interf().peekStreamMessages.getReplyStream( - TLogPeekStreamRequest(self->messageVersion.version, self->tag, std::numeric_limits::max())); + self->peekReplyStream = self->interf->get().interf().peekStreamMessages.getReplyStream(TLogPeekStreamRequest( + self->messageVersion.version, self->tag, self->returnIfBlocked, std::numeric_limits::max())); } ILogSystem::ServerPeekCursor::ServerPeekCursor(Reference>> const& interf, @@ -339,12 +339,10 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T } catch (Error& e) { if (e.code() == error_code_connection_failed) { self->peekReplyStream.reset(); - } - else if(e.code() == error_code_end_of_stream) { + } else if (e.code() == error_code_end_of_stream) { self->end.reset(self->messageVersion.version); return Void(); - } - else { + } else { throw; } } diff --git a/fdbserver/TLogInterface.h b/fdbserver/TLogInterface.h index ba8bc10d64..892adb3730 100644 --- a/fdbserver/TLogInterface.h +++ b/fdbserver/TLogInterface.h @@ -235,16 +235,17 @@ struct TLogPeekStreamRequest { Arena arena; Version begin; Tag tag; + bool returnIfBlocked; int limitBytes; ReplyPromiseStream reply; TLogPeekStreamRequest() {} - TLogPeekStreamRequest(Version version, Tag tag, int limitBytes) - : begin(version), tag(tag), limitBytes(limitBytes) {} + TLogPeekStreamRequest(Version version, Tag tag, bool returnIfBlocked, int limitBytes) + : begin(version), tag(tag), returnIfBlocked(returnIfBlocked), limitBytes(limitBytes) {} template void serialize(Ar& ar) { - serializer(ar, arena, begin, tag, limitBytes, reply); + serializer(ar, arena, begin, tag, returnIfBlocked, limitBytes, reply); } }; diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 8f512572cc..bce1639dc4 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1570,14 +1570,13 @@ ACTOR Future> parseMessagesForTag(StringRef commitBlob, T } // Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request -ACTOR Future peekTLog( - TLogData* self, - Reference logData, - Version begin, - Tag tag, - bool returnIfBlocked = false, - bool reqOnlySpilled = false, - Optional> sequence = Optional>()) { +ACTOR Future peekTLog(TLogData* self, + Reference logData, + Version begin, + Tag tag, + bool returnIfBlocked = false, + bool reqOnlySpilled = false, + Optional> sequence = Optional>()) { state BinaryWriter messages(Unversioned()); state BinaryWriter messages2(Unversioned()); state int sequenceNum = -1; @@ -1680,7 +1679,8 @@ ACTOR Future peekTLog( rep.end = poppedVer; rep.onlySpilled = false; - // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence requests. + // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence + // requests. if (sequence.present()) { auto& trackerData = logData->peekTracker[peekId]; auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; @@ -1911,31 +1911,29 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref loop { state TLogPeekStreamReply reply; try { - wait(req.reply.onReady() && store(reply.rep, peekTLog(self, logData, begin, req.tag, false, onlySpilled))); + wait(req.reply.onReady() && + store(reply.rep, peekTLog(self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled))); req.reply.send(reply); begin = reply.rep.end; onlySpilled = reply.rep.onlySpilled; wait(delay(0.005, TaskPriority::TLogPeekReply)); + // return Void(); } catch (Error& e) { - if(e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { + if (e.code() == error_code_end_of_stream) { req.reply.sendError(e); return Void(); + } else { + throw; } - throw; } } } ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { - state BinaryWriter messages(Unversioned()); - state BinaryWriter messages2(Unversioned()); - state int sequence = -1; - state UID peekId; - state double queueStart = now(); - try { - TLogPeekReply reply = wait(peekTLog(self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); + TLogPeekReply reply = + wait(peekTLog(self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); req.reply.send(reply); } catch (Error& e) { if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || From 6d1c12899dcdb263cb112a5b5ec15e3a00bd9a9c Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Fri, 9 Jul 2021 22:46:16 +0000 Subject: [PATCH 07/63] catch exceptions --- fdbrpc/fdbrpc.h | 4 ++-- fdbserver/LogSystemPeekCursor.actor.cpp | 12 +++++++----- fdbserver/TLogInterface.h | 2 +- fdbserver/TLogServer.actor.cpp | 8 +++++--- 4 files changed, 15 insertions(+), 11 deletions(-) diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index bfc0d93c78..81c3bbd3b5 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -406,8 +406,6 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, template class ReplyPromiseStream { public: - // The endpoints of a ReplyPromiseStream must be initialized at Task::ReadSocket, because with lower priorities a - // delay(0) in FlowTransport deliver can cause out of order delivery. // stream.send( request ) // Unreliable at most once delivery: Delivers request unless there is a connection failure (zero or one times) @@ -477,6 +475,8 @@ public: errors->delPromiseRef(); } + // The endpoints of a ReplyPromiseStream must be initialized at Task::ReadSocket, because with lower priorities a + // delay(0) in FlowTransport deliver can cause out of order delivery. const Endpoint& getEndpoint() const { return queue->getEndpoint(TaskPriority::ReadSocket); } bool operator==(const ReplyPromiseStream& rhs) const { return queue == rhs.queue; } diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index ce7bc5aa1c..97d17c4d62 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -386,13 +386,15 @@ Future ILogSystem::ServerPeekCursor::getMore(TaskPriority taskID) { if (hasMessage() && !parallelGetMore) return Void(); if (!more.isValid() || more.isReady()) { - if (usePeekStream) { + if (usePeekStream && taskID == TaskPriority::TLogPeekReply) { more = serverPeekStreamGetMore(this, taskID); - } else if (parallelGetMore || onlySpilled || futureResults.size()) { - more = serverPeekParallelGetMore(this, taskID); - } else { - more = serverPeekGetMore(this, taskID); } +// if (parallelGetMore || onlySpilled || futureResults.size()) { +// more = serverPeekParallelGetMore(this, taskID); +// } + else { + more = serverPeekGetMore(this, taskID); + } } return more; } diff --git a/fdbserver/TLogInterface.h b/fdbserver/TLogInterface.h index 892adb3730..f92638d639 100644 --- a/fdbserver/TLogInterface.h +++ b/fdbserver/TLogInterface.h @@ -83,7 +83,7 @@ struct TLogInterface { streams.push_back(disablePopRequest.getReceiver()); streams.push_back(enablePopRequest.getReceiver()); streams.push_back(snapRequest.getReceiver()); - streams.push_back(peekStreamMessages.getReceiver(TaskPriority::ReadSocket)); + streams.push_back(peekStreamMessages.getReceiver(TaskPriority::TLogPeek)); FlowTransport::transport().addEndpoints(streams); } diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index bce1639dc4..5524612a2c 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1916,15 +1916,17 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref req.reply.send(reply); begin = reply.rep.end; onlySpilled = reply.rep.onlySpilled; - - wait(delay(0.005, TaskPriority::TLogPeekReply)); // return Void(); } catch (Error& e) { if (e.code() == error_code_end_of_stream) { req.reply.sendError(e); return Void(); + } + else if (e.code() == error_code_operation_obsolete) { + // reply stream is cancelled on the client + return Void(); } else { - throw; + throw; } } } From 066d5341947a0df60efb8e21c7511510371a8a01 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Wed, 14 Jul 2021 16:19:23 +0000 Subject: [PATCH 08/63] trivial changes --- fdbserver/LogSystemPeekCursor.actor.cpp | 31 +++++++++++++++++++------ fdbserver/TLogServer.actor.cpp | 3 ++- 2 files changed, 26 insertions(+), 8 deletions(-) diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 97d17c4d62..809b126e70 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -35,6 +35,7 @@ void tryEstablishPeekStream(ILogSystem::ServerPeekCursor* self) { } self->peekReplyStream = self->interf->get().interf().peekStreamMessages.getReplyStream(TLogPeekStreamRequest( self->messageVersion.version, self->tag, self->returnIfBlocked, std::numeric_limits::max())); + TraceEvent(SevDebug, "StreamCreated"); } ILogSystem::ServerPeekCursor::ServerPeekCursor(Reference>> const& interf, @@ -329,10 +330,15 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T self->peekReplyStream.reset(); tryEstablishPeekStream(self); } - when(TLogPeekStreamReply res = wait(self->peekReplyStream.present() + when(TLogPeekStreamReply res = + wait(self->peekReplyStream.present() ? brokenPromiseToNever(waitAndForward(self->peekReplyStream.get().getFuture())) : Never())) { updateCursorWithReply(self, res.rep); + TraceEvent("SPC_GetMoreB", self->randomID) + .detail("Has", self->hasMessage()) + .detail("End", res.rep.end) + .detail("Popped", res.rep.popped.present() ? res.rep.popped.get() : 0); return Void(); } } @@ -389,12 +395,12 @@ Future ILogSystem::ServerPeekCursor::getMore(TaskPriority taskID) { if (usePeekStream && taskID == TaskPriority::TLogPeekReply) { more = serverPeekStreamGetMore(this, taskID); } -// if (parallelGetMore || onlySpilled || futureResults.size()) { -// more = serverPeekParallelGetMore(this, taskID); -// } + // if (parallelGetMore || onlySpilled || futureResults.size()) { + // more = serverPeekParallelGetMore(this, taskID); + // } else { - more = serverPeekGetMore(this, taskID); - } + more = serverPeekGetMore(this, taskID); + } } return more; } @@ -408,6 +414,12 @@ ACTOR Future serverPeekOnFailed(ILogSystem::ServerPeekCursor* self) { : Never())) { return Void(); } + when(wait(self->interf->get().present() + ? IFailureMonitor::failureMonitor().onStateEqual( + self->interf->get().interf().peekStreamMessages.getEndpoint(), FailureStatus()) + : Never())) { + return Void(); + } when(wait(self->interf->onChange())) {} } } @@ -422,7 +434,12 @@ bool ILogSystem::ServerPeekCursor::isActive() const { return false; if (isExhausted()) return false; - return IFailureMonitor::failureMonitor().getState(interf->get().interf().peekMessages.getEndpoint()).isAvailable(); + return IFailureMonitor::failureMonitor() + .getState(interf->get().interf().peekMessages.getEndpoint()) + .isAvailable() && + IFailureMonitor::failureMonitor() + .getState(interf->get().interf().peekStreamMessages.getEndpoint()) + .isAvailable(); } bool ILogSystem::ServerPeekCursor::isExhausted() const { diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 5524612a2c..cdf8dacbad 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1916,7 +1916,8 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref req.reply.send(reply); begin = reply.rep.end; onlySpilled = reply.rep.onlySpilled; - // return Void(); + + wait(delay(.05, g_network->getCurrentTask())); } catch (Error& e) { if (e.code() == error_code_end_of_stream) { req.reply.sendError(e); From 227570357ab5f040dcea217d079f34f35bf42c5d Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Thu, 15 Jul 2021 21:30:14 +0000 Subject: [PATCH 09/63] trace log and reset changes; byteAcknownledge overflow --- fdbrpc/fdbrpc.h | 44 ++++++++++++++------- fdbserver/LogSystemPeekCursor.actor.cpp | 52 ++++++++++++++++--------- fdbserver/TLogServer.actor.cpp | 36 +++++++++-------- 3 files changed, 82 insertions(+), 50 deletions(-) diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index 403d8d4dc2..b0443d2a23 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -277,9 +277,9 @@ struct AcknowledgementReceiver final : FlowReceiver, FastAllocated::operator new; using FastAllocated::operator delete; - int64_t bytesSent; - int64_t bytesAcknowledged; - int64_t bytesLimit; + uint64_t bytesSent; + uint64_t bytesAcknowledged; + uint64_t bytesLimit; Promise ready; Future failures; @@ -358,11 +358,19 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, // send an ack immediately if (acknowledgements.getRawEndpoint().isValid()) { acknowledgements.bytesAcknowledged += message.get().asUnderlyingType().expectedSize(); - FlowTransport::transport().sendUnreliable( - SerializeSource>( - AcknowledgementReply(acknowledgements.bytesAcknowledged)), - acknowledgements.getEndpoint(TaskPriority::ReadSocket), - false); + // int64_t overflow: we need to reset this stream + if (acknowledgements.bytesAcknowledged > std::numeric_limits::max()) { + FlowTransport::transport().sendUnreliable( + SerializeSource>(operation_obsolete()), + acknowledgements.getEndpoint(TaskPriority::ReadSocket), + false); + } else { + FlowTransport::transport().sendUnreliable( + SerializeSource>( + AcknowledgementReply(acknowledgements.bytesAcknowledged)), + acknowledgements.getEndpoint(TaskPriority::ReadSocket), + false); + } } } @@ -376,10 +384,17 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, // A reply that has been queued up is being consumed, so send an ack to the server if (acknowledgements.getRawEndpoint().isValid()) { acknowledgements.bytesAcknowledged += res.expectedSize(); - FlowTransport::transport().sendUnreliable(SerializeSource>( - AcknowledgementReply(acknowledgements.bytesAcknowledged)), - acknowledgements.getEndpoint(TaskPriority::ReadSocket), - false); + if (acknowledgements.bytesAcknowledged > std::numeric_limits::max()) { + FlowTransport::transport().sendUnreliable( + SerializeSource>(operation_obsolete()), + acknowledgements.getEndpoint(TaskPriority::ReadSocket), + false); + } else { + FlowTransport::transport().sendUnreliable(SerializeSource>( + AcknowledgementReply(acknowledgements.bytesAcknowledged)), + acknowledgements.getEndpoint(TaskPriority::ReadSocket), + false); + } } return res; } @@ -406,7 +421,6 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, template class ReplyPromiseStream { public: - // stream.send( request ) // Unreliable at most once delivery: Delivers request unless there is a connection failure (zero or one times) @@ -475,8 +489,8 @@ public: errors->delPromiseRef(); } - // The endpoints of a ReplyPromiseStream must be initialized at Task::ReadSocket, because with lower priorities a - // delay(0) in FlowTransport deliver can cause out of order delivery. + // The endpoints of a ReplyPromiseStream must be initialized at Task::ReadSocket, because with lower priorities a + // delay(0) in FlowTransport deliver can cause out of order delivery. const Endpoint& getEndpoint() const { return queue->getEndpoint(TaskPriority::ReadSocket); } bool operator==(const ReplyPromiseStream& rhs) const { return queue == rhs.queue; } diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 809b126e70..154d870f88 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -35,7 +35,7 @@ void tryEstablishPeekStream(ILogSystem::ServerPeekCursor* self) { } self->peekReplyStream = self->interf->get().interf().peekStreamMessages.getReplyStream(TLogPeekStreamRequest( self->messageVersion.version, self->tag, self->returnIfBlocked, std::numeric_limits::max())); - TraceEvent(SevDebug, "StreamCreated"); + TraceEvent(SevDebug, "SPC_StreamCreated", self->randomID); } ILogSystem::ServerPeekCursor::ServerPeekCursor(Reference>> const& interf, @@ -51,7 +51,11 @@ ILogSystem::ServerPeekCursor::ServerPeekCursor(ReferencePEEK_USEING_STREAMING) { this->results.maxKnownVersion = 0; this->results.minKnownCommittedVersion = 0; - //TraceEvent("SPC_Starting", randomID).detail("Tag", tag.toString()).detail("Begin", begin).detail("End", end).backtrace(); + TraceEvent("SPC_Starting", randomID) + .detail("Tag", tag.toString()) + .detail("Begin", begin) + .detail("End", end) + .backtrace(); } ILogSystem::ServerPeekCursor::ServerPeekCursor(TLogPeekReply const& results, @@ -316,29 +320,34 @@ ACTOR Future serverPeekParallelGetMore(ILogSystem::ServerPeekCursor* self, ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, TaskPriority taskID) { if (!self->interf || self->isExhausted()) { + self->peekReplyStream.reset(); if (self->hasMessage()) return Void(); return Never(); } - tryEstablishPeekStream(self); loop { try { + tryEstablishPeekStream(self); + state Future fPeekReply = self->peekReplyStream.present() + ? map(waitAndForward(self->peekReplyStream.get().getFuture()), + [](const TLogPeekStreamReply& r) { return r.rep; }) + : Never(); choose { when(wait(self->interf->onChange())) { self->onlySpilled = false; self->peekReplyStream.reset(); - tryEstablishPeekStream(self); } - when(TLogPeekStreamReply res = + when(TLogPeekReply res = wait(self->peekReplyStream.present() - ? brokenPromiseToNever(waitAndForward(self->peekReplyStream.get().getFuture())) + ? recordRequestMetrics( + self, self->peekReplyStream.get().getEndpoint().getPrimaryAddress(), fPeekReply) : Never())) { - updateCursorWithReply(self, res.rep); + updateCursorWithReply(self, res); TraceEvent("SPC_GetMoreB", self->randomID) .detail("Has", self->hasMessage()) - .detail("End", res.rep.end) - .detail("Popped", res.rep.popped.present() ? res.rep.popped.get() : 0); + .detail("End", res.end) + .detail("Popped", res.popped.present() ? res.popped.get() : 0); return Void(); } } @@ -388,19 +397,24 @@ ACTOR Future serverPeekGetMore(ILogSystem::ServerPeekCursor* self, TaskPri } Future ILogSystem::ServerPeekCursor::getMore(TaskPriority taskID) { - //TraceEvent("SPC_GetMore", randomID).detail("HasMessage", hasMessage()).detail("More", !more.isValid() || more.isReady()).detail("MessageVersion", messageVersion.toString()).detail("End", end.toString()); + TraceEvent("SPC_GetMore", randomID) + .detail("HasMessage", hasMessage()) + .detail("More", !more.isValid() || more.isReady()) + .detail("MessageVersion", messageVersion.toString()) + .detail("End", end.toString()); if (hasMessage() && !parallelGetMore) return Void(); if (!more.isValid() || more.isReady()) { - if (usePeekStream && taskID == TaskPriority::TLogPeekReply) { - more = serverPeekStreamGetMore(this, taskID); - } - // if (parallelGetMore || onlySpilled || futureResults.size()) { - // more = serverPeekParallelGetMore(this, taskID); - // } - else { - more = serverPeekGetMore(this, taskID); - } + more = serverPeekStreamGetMore(this, taskID); +// if (usePeekStream && taskID == TaskPriority::TLogPeekReply) { +// more = serverPeekStreamGetMore(this, taskID); +// } +// if (parallelGetMore || onlySpilled || futureResults.size()) { +// more = serverPeekParallelGetMore(this, taskID); +// } +// else { +// more = serverPeekGetMore(this, taskID); +// } } return more; } diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index a46d22d856..9ad416986e 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -557,6 +557,7 @@ struct LogData : NonCopyable, public ReferenceCounted { TLogData* tLogData; Promise recoveryComplete, committingQueue; Version unrecoveredBefore, recoveredAt; + int activePeekStreams = 0; struct PeekTrackerData { std::map>> @@ -668,6 +669,7 @@ struct LogData : NonCopyable, public ReferenceCounted { specialCounter(cc, "PeekMemoryReserved", [tLogData]() { return tLogData->peekMemoryLimiter.activePermits(); }); specialCounter(cc, "PeekMemoryRequestsStalled", [tLogData]() { return tLogData->peekMemoryLimiter.waiters(); }); specialCounter(cc, "Generation", [this]() { return this->recoveryCount; }); + specialCounter(cc, "ActivePeekStreams", [this]() { return this->activePeekStreams; }); } ~LogData() { @@ -1167,17 +1169,19 @@ ACTOR Future tLogPopCore(TLogData* self, Tag inputTag, Version to, Referen } uint64_t PoppedVersionLag = logData->persistentDataDurableVersion - logData->queuePoppedVersion; - if ( SERVER_KNOBS->ENABLE_DETAILED_TLOG_POP_TRACE && - (logData->queuePoppedVersion > 0) && //avoid generating massive events at beginning - (tagData->unpoppedRecovered || PoppedVersionLag >= SERVER_KNOBS->TLOG_POPPED_VER_LAG_THRESHOLD_FOR_TLOGPOP_TRACE)) { //when recovery or long lag + if (SERVER_KNOBS->ENABLE_DETAILED_TLOG_POP_TRACE && + (logData->queuePoppedVersion > 0) && // avoid generating massive events at beginning + (tagData->unpoppedRecovered || + PoppedVersionLag >= + SERVER_KNOBS->TLOG_POPPED_VER_LAG_THRESHOLD_FOR_TLOGPOP_TRACE)) { // when recovery or long lag TraceEvent("TLogPopDetails", logData->logId) - .detail("Tag", tagData->tag.toString()) - .detail("UpTo", upTo) - .detail("PoppedVersionLag", PoppedVersionLag) - .detail("MinPoppedTag", logData->minPoppedTag.toString()) - .detail("QueuePoppedVersion", logData->queuePoppedVersion) - .detail("UnpoppedRecovered", tagData->unpoppedRecovered ? "True" : "False") - .detail("NothingPersistent", tagData->nothingPersistent ? "True" : "False"); + .detail("Tag", tagData->tag.toString()) + .detail("UpTo", upTo) + .detail("PoppedVersionLag", PoppedVersionLag) + .detail("MinPoppedTag", logData->minPoppedTag.toString()) + .detail("QueuePoppedVersion", logData->queuePoppedVersion) + .detail("UnpoppedRecovered", tagData->unpoppedRecovered ? "True" : "False") + .detail("NothingPersistent", tagData->nothingPersistent ? "True" : "False"); } if (upTo > logData->persistentDataDurableVersion) wait(tagData->eraseMessagesBefore(upTo, self, logData, TaskPriority::TLogPop)); @@ -1915,6 +1919,7 @@ ACTOR Future peekTLog(TLogData* self, // This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { + logData->activePeekStreams ++; state Version begin = req.begin; state bool onlySpilled = false; if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { @@ -1929,18 +1934,17 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref req.reply.send(reply); begin = reply.rep.end; onlySpilled = reply.rep.onlySpilled; - - wait(delay(.05, g_network->getCurrentTask())); + wait(delay(0, g_network->getCurrentTask())); } catch (Error& e) { + logData->activePeekStreams --; if (e.code() == error_code_end_of_stream) { req.reply.sendError(e); return Void(); - } - else if (e.code() == error_code_operation_obsolete) { + } else if (e.code() == error_code_operation_obsolete) { // reply stream is cancelled on the client - return Void(); + return Void(); } else { - throw; + throw; } } } From f3667ce91adc18dbc49fb3f7d3e609b1f46f78d6 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Mon, 19 Jul 2021 18:43:51 +0000 Subject: [PATCH 10/63] more debug logs; let tryEstablishStream wait until the connection is good --- fdbserver/LogSystemPeekCursor.actor.cpp | 45 ++++++++++++++----------- fdbserver/TLogServer.actor.cpp | 11 +++--- 2 files changed, 33 insertions(+), 23 deletions(-) diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 154d870f88..7473d7525a 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -26,16 +26,20 @@ #include "flow/actorcompiler.h" // has to be last include // create a peek stream for cursor when it's possible -void tryEstablishPeekStream(ILogSystem::ServerPeekCursor* self) { +ACTOR Future tryEstablishPeekStream(ILogSystem::ServerPeekCursor* self) { if (self->peekReplyStream.present()) - return; + return Void(); else if (!self->interf || !self->interf->get().present()) { self->peekReplyStream.reset(); - return; + return Never(); } + wait(IFailureMonitor::failureMonitor().onStateEqual(self->interf->get().interf().peekStreamMessages.getEndpoint(), + FailureStatus(false))); self->peekReplyStream = self->interf->get().interf().peekStreamMessages.getReplyStream(TLogPeekStreamRequest( self->messageVersion.version, self->tag, self->returnIfBlocked, std::numeric_limits::max())); - TraceEvent(SevDebug, "SPC_StreamCreated", self->randomID); + TraceEvent(SevDebug, "SPC_StreamCreated", self->randomID) + .detail("PeerAddress", self->interf->get().interf().peekStreamMessages.getEndpoint().getPrimaryAddress()); + return Void(); } ILogSystem::ServerPeekCursor::ServerPeekCursor(Reference>> const& interf, @@ -328,21 +332,23 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T loop { try { - tryEstablishPeekStream(self); state Future fPeekReply = self->peekReplyStream.present() ? map(waitAndForward(self->peekReplyStream.get().getFuture()), [](const TLogPeekStreamReply& r) { return r.rep; }) : Never(); choose { + when(wait(self->peekReplyStream.present() ? Never() : tryEstablishPeekStream(self))) {} when(wait(self->interf->onChange())) { self->onlySpilled = false; self->peekReplyStream.reset(); } - when(TLogPeekReply res = - wait(self->peekReplyStream.present() - ? recordRequestMetrics( - self, self->peekReplyStream.get().getEndpoint().getPrimaryAddress(), fPeekReply) - : Never())) { + when(TLogPeekReply res = wait( + self->peekReplyStream.present() + ? recordRequestMetrics( + self, + self->interf->get().interf().peekStreamMessages.getEndpoint().getPrimaryAddress(), + fPeekReply) + : Never())) { updateCursorWithReply(self, res); TraceEvent("SPC_GetMoreB", self->randomID) .detail("Has", self->hasMessage()) @@ -352,6 +358,7 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T } } } catch (Error& e) { + TraceEvent(SevDebug, "SPC_GetMoreB_Error", self->randomID).detail("Error", e.what()); if (e.code() == error_code_connection_failed) { self->peekReplyStream.reset(); } else if (e.code() == error_code_end_of_stream) { @@ -406,15 +413,15 @@ Future ILogSystem::ServerPeekCursor::getMore(TaskPriority taskID) { return Void(); if (!more.isValid() || more.isReady()) { more = serverPeekStreamGetMore(this, taskID); -// if (usePeekStream && taskID == TaskPriority::TLogPeekReply) { -// more = serverPeekStreamGetMore(this, taskID); -// } -// if (parallelGetMore || onlySpilled || futureResults.size()) { -// more = serverPeekParallelGetMore(this, taskID); -// } -// else { -// more = serverPeekGetMore(this, taskID); -// } + // if (usePeekStream && taskID == TaskPriority::TLogPeekReply) { + // more = serverPeekStreamGetMore(this, taskID); + // } + // if (parallelGetMore || onlySpilled || futureResults.size()) { + // more = serverPeekParallelGetMore(this, taskID); + // } + // else { + // more = serverPeekGetMore(this, taskID); + // } } return more; } diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 9ad416986e..f437f24637 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -341,6 +341,7 @@ struct TLogData : NonCopyable { int64_t targetVolatileBytes; // The number of bytes of mutations this TLog should hold in memory before spilling. int64_t overheadBytesInput; int64_t overheadBytesDurable; + int activePeekStreams = 0; WorkerCache tlogCache; FlowLock peekMemoryLimiter; @@ -557,7 +558,6 @@ struct LogData : NonCopyable, public ReferenceCounted { TLogData* tLogData; Promise recoveryComplete, committingQueue; Version unrecoveredBefore, recoveredAt; - int activePeekStreams = 0; struct PeekTrackerData { std::map>> @@ -669,7 +669,7 @@ struct LogData : NonCopyable, public ReferenceCounted { specialCounter(cc, "PeekMemoryReserved", [tLogData]() { return tLogData->peekMemoryLimiter.activePermits(); }); specialCounter(cc, "PeekMemoryRequestsStalled", [tLogData]() { return tLogData->peekMemoryLimiter.waiters(); }); specialCounter(cc, "Generation", [this]() { return this->recoveryCount; }); - specialCounter(cc, "ActivePeekStreams", [this]() { return this->activePeekStreams; }); + specialCounter(cc, "ActivePeekStreams", [tLogData]() { return tLogData->activePeekStreams; }); } ~LogData() { @@ -1919,7 +1919,8 @@ ACTOR Future peekTLog(TLogData* self, // This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { - logData->activePeekStreams ++; + self->activePeekStreams++; + TraceEvent(SevDebug, "TLogPeekStream", logData->logId); state Version begin = req.begin; state bool onlySpilled = false; if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { @@ -1936,7 +1937,9 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref onlySpilled = reply.rep.onlySpilled; wait(delay(0, g_network->getCurrentTask())); } catch (Error& e) { - logData->activePeekStreams --; + self->activePeekStreams--; + TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).detail("Error", e.what()); + if (e.code() == error_code_end_of_stream) { req.reply.sendError(e); return Void(); From 5046ee3b07ec89a81a61403be184d847f45bedf5 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Tue, 20 Jul 2021 17:42:00 +0000 Subject: [PATCH 11/63] add stream peek to logRouter --- fdbrpc/fdbrpc.h | 21 +- fdbserver/LogRouter.actor.cpp | 376 +++++++++++++++++------- fdbserver/LogSystemPeekCursor.actor.cpp | 20 +- fdbserver/QuietDatabase.actor.cpp | 3 +- fdbserver/TLogServer.actor.cpp | 5 +- 5 files changed, 298 insertions(+), 127 deletions(-) diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index 25736055f6..1f40bb47a7 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -492,8 +492,8 @@ public: errors->delPromiseRef(); } - // The endpoints of a ReplyPromiseStream must be initialized at Task::NoDeliverDelay, because with lower priorities a - // delay(0) in FlowTransport deliver can cause out of order delivery. + // The endpoints of a ReplyPromiseStream must be initialized at Task::NoDeliverDelay, because with lower priorities + // a delay(0) in FlowTransport deliver can cause out of order delivery. const Endpoint& getEndpoint() const { return queue->getEndpoint(TaskPriority::NoDeliverDelay); } bool operator==(const ReplyPromiseStream& rhs) const { return queue == rhs.queue; } @@ -710,20 +710,17 @@ public: template ReplyPromiseStream getReplyStream(const X& value) const { + Future disc = makeDependent(IFailureMonitor::failureMonitor()).onDisconnectOrFailure(getEndpoint()); + auto& p = getReplyPromiseStream(value); + Reference peer; if (queue->isRemoteEndpoint()) { - Future disc = - makeDependent(IFailureMonitor::failureMonitor()).onDisconnectOrFailure(getEndpoint()); - auto& p = getReplyPromiseStream(value); - Reference peer = - FlowTransport::transport().sendUnreliable(SerializeSource(value), getEndpoint(), true); - // FIXME: defer sending the message until we know the connection is established - endStreamOnDisconnect(disc, p, getEndpoint(), peer); - return p; + peer = FlowTransport::transport().sendUnreliable(SerializeSource(value), getEndpoint(), true); } else { send(value); - auto& p = getReplyPromiseStream(value); - return p; } + // FIXME: defer sending the message until we know the connection is established + endStreamOnDisconnect(disc, p, getEndpoint(), peer); + return p; } // stream.getReplyUnlessFailedFor( request, double sustainedFailureDuration, double sustainedFailureSlope ) diff --git a/fdbserver/LogRouter.actor.cpp b/fdbserver/LogRouter.actor.cpp index ec0ec6a416..8e9ee308a6 100644 --- a/fdbserver/LogRouter.actor.cpp +++ b/fdbserver/LogRouter.actor.cpp @@ -117,6 +117,7 @@ struct LogRouterData { getMoreBlockedCount; // Increase by 1 if data is not available when LR tries to pull data from satellite tLog. Future logger; Reference eventCacheHolder; + int activePeekStreams = 0; std::vector> tag_data; // we only store data for the remote tag locality @@ -193,6 +194,7 @@ struct LogRouterData { return int64_t(1000 * val); }); specialCounter(cc, "Generation", [this]() { return this->generation; }); + specialCounter(cc, "ActivePeekStreams", [this]() { return this->activePeekStreams; }); logger = traceCounters("LogRouterMetrics", dbgid, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, @@ -404,18 +406,15 @@ std::deque>& get_version_messages(Lo return tagData->version_messages; }; -void peekMessagesFromMemory(LogRouterData* self, - TLogPeekRequest const& req, - BinaryWriter& messages, - Version& endVersion) { +void peekMessagesFromMemory(LogRouterData* self, Tag tag, Version begin, BinaryWriter& messages, Version& endVersion) { ASSERT(!messages.getLength()); - auto& deque = get_version_messages(self, req.tag); + auto& deque = get_version_messages(self, tag); //TraceEvent("TLogPeekMem", self->dbgid).detail("Tag", req.tag1).detail("PDS", self->persistentDataSequence).detail("PDDS", self->persistentDataDurableSequence).detail("Oldest", map1.empty() ? 0 : map1.begin()->key ).detail("OldestMsgCount", map1.empty() ? 0 : map1.begin()->value.size()); auto it = std::lower_bound(deque.begin(), deque.end(), - std::make_pair(req.begin, LengthPrefixedStringRef()), + std::make_pair(begin, LengthPrefixedStringRef()), CompareFirst>()); Version currentVersion = -1; @@ -442,126 +441,296 @@ Version poppedVersion(LogRouterData* self, Tag tag) { return tagData->popped; } -ACTOR Future logRouterPeekMessages(LogRouterData* self, TLogPeekRequest req) { +ACTOR Future peekLogRouter(LogRouterData* self, + Version begin, + Tag tag, + bool returnIfBlocked = false, + bool reqOnlySpilled = false, + Optional> sequence = Optional>()) { state BinaryWriter messages(Unversioned()); - state int sequence = -1; + state int sequenceNum = -1; state UID peekId; - if (req.sequence.present()) { - try { - peekId = req.sequence.get().first; - sequence = req.sequence.get().second; - if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - self->peekTracker.find(peekId) == self->peekTracker.end()) { - throw operation_obsolete(); + if (sequence.present()) { + peekId = sequence.get().first; + sequenceNum = sequence.get().second; + if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + self->peekTracker.find(peekId) == self->peekTracker.end()) { + throw operation_obsolete(); + } + auto& trackerData = self->peekTracker[peekId]; + if (sequenceNum == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { + trackerData.sequence_version[0].send(std::make_pair(begin, reqOnlySpilled)); + } + auto seqBegin = trackerData.sequence_version.begin(); + // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. + while (trackerData.sequence_version.size() && + seqBegin->first <= sequenceNum - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + if (seqBegin->second.canBeSet()) { + seqBegin->second.sendError(operation_obsolete()); } + trackerData.sequence_version.erase(seqBegin); + seqBegin = trackerData.sequence_version.begin(); + } + + if (trackerData.sequence_version.size() && sequenceNum < seqBegin->first) { + throw operation_obsolete(); + } + + trackerData.lastUpdate = now(); + std::pair prevPeekData = wait(trackerData.sequence_version[sequenceNum].getFuture()); + begin = prevPeekData.first; + reqOnlySpilled = prevPeekData.second; + wait(yield()); + } + + //TraceEvent("LogRouterPeek1", self->dbgid).detail("From", req.reply.getEndpoint().getPrimaryAddress()).detail("Ver", self->version.get()).detail("Begin", req.begin); + if (returnIfBlocked && self->version.get() < begin) { + //TraceEvent("LogRouterPeek2", self->dbgid); + if (sequence.present()) { auto& trackerData = self->peekTracker[peekId]; - if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled)); - } - auto seqBegin = trackerData.sequence_version.begin(); - // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. - while (trackerData.sequence_version.size() && - seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { - if (seqBegin->second.canBeSet()) { - seqBegin->second.sendError(operation_obsolete()); - } - trackerData.sequence_version.erase(seqBegin); - seqBegin = trackerData.sequence_version.begin(); + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + if (!sequenceData.isSet()) { + sequenceData.send(std::make_pair(begin, reqOnlySpilled)); } + } + throw end_of_stream(); + } - if (trackerData.sequence_version.size() && sequence < seqBegin->first) { + if (self->version.get() < begin) { + wait(self->version.whenAtLeast(begin)); + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } + + Version poppedVer = poppedVersion(self, tag); + + if (poppedVer > begin || begin < self->startVersion) { + // This should only happen if a packet is sent multiple times and the reply is not needed. + // Since we are using popped differently, do not send a reply. + TraceEvent(SevWarnAlways, "LogRouterPeekPopped", self->dbgid) + .detail("Begin", begin) + .detail("Popped", poppedVer) + .detail("Start", self->startVersion); + if (sequence.present()) { + auto& trackerData = self->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + if (!sequenceData.isSet()) { + sequenceData.send(std::make_pair(begin, reqOnlySpilled)); + } + } + throw success(); // we've already replied in the past + } + + Version endVersion = self->version.get() + 1; + peekMessagesFromMemory(self, tag, begin, messages, endVersion); + + TLogPeekReply reply; + reply.maxKnownVersion = self->version.get(); + reply.minKnownCommittedVersion = self->poppedVersion; + reply.messages = StringRef(reply.arena, messages.toValue()); + reply.popped = self->minPopped.get() >= self->startVersion ? self->minPopped.get() : 0; + reply.end = endVersion; + reply.onlySpilled = false; + + if (sequence.present()) { + auto& trackerData = self->peekTracker[peekId]; + trackerData.lastUpdate = now(); + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + if (!sequenceData.isSet()) + sequenceData.sendError(operation_obsolete()); + throw operation_obsolete(); + } + if (sequenceData.isSet()) { + if (sequenceData.getFuture().get().first != reply.end) { + TEST(true); // tlog peek second attempt ended at a different version throw operation_obsolete(); } + } else { + sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); + } + reply.begin = begin; + } - trackerData.lastUpdate = now(); - std::pair prevPeekData = wait(trackerData.sequence_version[sequence].getFuture()); - req.begin = prevPeekData.first; - req.onlySpilled = prevPeekData.second; - wait(yield()); + //TraceEvent("LogRouterPeek4", self->dbgid); + return reply; +} + +// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover +ACTOR Future logRouterPeekStream(LogRouterData* self, TLogPeekStreamRequest req) { + self->activePeekStreams++; + TraceEvent(SevDebug, "TLogPeekStream", self->dbgid).detail("Token", req.reply.getEndpoint().token); + + state Version begin = req.begin; + state bool onlySpilled = false; + + req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); + loop { + state TLogPeekStreamReply reply; + try { + wait(req.reply.onReady() && + store(reply.rep, peekLogRouter(self, req.begin, req.tag, req.returnIfBlocked, onlySpilled))); + req.reply.send(reply); + begin = reply.rep.end; + onlySpilled = reply.rep.onlySpilled; + wait(delay(0, g_network->getCurrentTask())); } catch (Error& e) { - if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { + self->activePeekStreams--; + TraceEvent(SevDebug, "TLogPeekStreamEnd", self->dbgid).error(e, true); + + if (e.code() == error_code_success) { + continue; + } else if (e.code() == error_code_end_of_stream) { req.reply.sendError(e); return Void(); + } else if (e.code() == error_code_operation_obsolete) { + // reply stream is cancelled on the client + return Void(); } else { throw; } } } +} - //TraceEvent("LogRouterPeek1", self->dbgid).detail("From", req.reply.getEndpoint().getPrimaryAddress()).detail("Ver", self->version.get()).detail("Begin", req.begin); - if (req.returnIfBlocked && self->version.get() < req.begin) { - //TraceEvent("LogRouterPeek2", self->dbgid); - req.reply.sendError(end_of_stream()); - if (req.sequence.present()) { - auto& trackerData = self->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; - if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); - } - } - return Void(); - } - - if (self->version.get() < req.begin) { - wait(self->version.whenAtLeast(req.begin)); - wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); - } - - Version poppedVer = poppedVersion(self, req.tag); - - if (poppedVer > req.begin || req.begin < self->startVersion) { - // This should only happen if a packet is sent multiple times and the reply is not needed. - // Since we are using popped differently, do not send a reply. - TraceEvent(SevWarnAlways, "LogRouterPeekPopped", self->dbgid) - .detail("Begin", req.begin) - .detail("Popped", poppedVer) - .detail("Start", self->startVersion); - req.reply.send(Never()); - if (req.sequence.present()) { - auto& trackerData = self->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; - if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); - } - } - return Void(); - } - - Version endVersion = self->version.get() + 1; - peekMessagesFromMemory(self, req, messages, endVersion); - - TLogPeekReply reply; - reply.maxKnownVersion = self->version.get(); - reply.minKnownCommittedVersion = self->poppedVersion; - reply.messages = messages.toValue(); - reply.popped = self->minPopped.get() >= self->startVersion ? self->minPopped.get() : 0; - reply.end = endVersion; - reply.onlySpilled = false; - - if (req.sequence.present()) { - auto& trackerData = self->peekTracker[peekId]; - trackerData.lastUpdate = now(); - auto& sequenceData = trackerData.sequence_version[sequence + 1]; - if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - req.reply.sendError(operation_obsolete()); - if (!sequenceData.isSet()) - sequenceData.sendError(operation_obsolete()); +ACTOR Future logRouterPeekMessages(LogRouterData* self, TLogPeekRequest req) { + try { + TLogPeekReply reply = + wait(peekLogRouter(self, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); + req.reply.send(reply); + } catch (Error& e) { + if (e.code() == error_code_success) { + req.reply.send(Never()); + return Void(); + } else if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || + e.code() == error_code_end_of_stream) { + req.reply.sendError(e); return Void(); - } - if (sequenceData.isSet()) { - if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // tlog peek second attempt ended at a different version - req.reply.sendError(operation_obsolete()); - return Void(); - } } else { - sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); + throw; } - reply.begin = req.begin; } - req.reply.send(reply); + // state BinaryWriter messages(Unversioned()); + // state int sequence = -1; + // state UID peekId; + // + // if (req.sequence.present()) { + // try { + // peekId = req.sequence.get().first; + // sequence = req.sequence.get().second; + // if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + // self->peekTracker.find(peekId) == self->peekTracker.end()) { + // throw operation_obsolete(); + // } + // auto& trackerData = self->peekTracker[peekId]; + // if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { + // trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled)); + // } + // auto seqBegin = trackerData.sequence_version.begin(); + // // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. + // while (trackerData.sequence_version.size() && + // seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + // if (seqBegin->second.canBeSet()) { + // seqBegin->second.sendError(operation_obsolete()); + // } + // trackerData.sequence_version.erase(seqBegin); + // seqBegin = trackerData.sequence_version.begin(); + // } + // + // if (trackerData.sequence_version.size() && sequence < seqBegin->first) { + // throw operation_obsolete(); + // } + // + // trackerData.lastUpdate = now(); + // std::pair prevPeekData = wait(trackerData.sequence_version[sequence].getFuture()); + // req.begin = prevPeekData.first; + // req.onlySpilled = prevPeekData.second; + // wait(yield()); + // } catch (Error& e) { + // if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { + // req.reply.sendError(e); + // return Void(); + // } else { + // throw; + // } + // } + // } + // + // //TraceEvent("LogRouterPeek1", self->dbgid).detail("From", + // req.reply.getEndpoint().getPrimaryAddress()).detail("Ver", self->version.get()).detail("Begin", req.begin); if + //(req.returnIfBlocked && self->version.get() < req.begin) { + // //TraceEvent("LogRouterPeek2", self->dbgid); + // req.reply.sendError(end_of_stream()); + // if (req.sequence.present()) { + // auto& trackerData = self->peekTracker[peekId]; + // auto& sequenceData = trackerData.sequence_version[sequence + 1]; + // if (!sequenceData.isSet()) { + // sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); + // } + // } + // return Void(); + // } + // + // if (self->version.get() < req.begin) { + // wait(self->version.whenAtLeast(req.begin)); + // wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + // } + // + // Version poppedVer = poppedVersion(self, req.tag); + // + // if (poppedVer > req.begin || req.begin < self->startVersion) { + // // This should only happen if a packet is sent multiple times and the reply is not needed. + // // Since we are using popped differently, do not send a reply. + // TraceEvent(SevWarnAlways, "LogRouterPeekPopped", self->dbgid) + // .detail("Begin", req.begin) + // .detail("Popped", poppedVer) + // .detail("Start", self->startVersion); + // req.reply.send(Never()); + // if (req.sequence.present()) { + // auto& trackerData = self->peekTracker[peekId]; + // auto& sequenceData = trackerData.sequence_version[sequence + 1]; + // if (!sequenceData.isSet()) { + // sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); + // } + // } + // return Void(); + // } + // + // Version endVersion = self->version.get() + 1; + // peekMessagesFromMemory(self, req.tag, req.begin, messages, endVersion); + // + // TLogPeekReply reply; + // reply.maxKnownVersion = self->version.get(); + // reply.minKnownCommittedVersion = self->poppedVersion; + // reply.messages = messages.toValue(); + // reply.popped = self->minPopped.get() >= self->startVersion ? self->minPopped.get() : 0; + // reply.end = endVersion; + // reply.onlySpilled = false; + // + // if (req.sequence.present()) { + // auto& trackerData = self->peekTracker[peekId]; + // trackerData.lastUpdate = now(); + // auto& sequenceData = trackerData.sequence_version[sequence + 1]; + // if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { + // req.reply.sendError(operation_obsolete()); + // if (!sequenceData.isSet()) + // sequenceData.sendError(operation_obsolete()); + // return Void(); + // } + // if (sequenceData.isSet()) { + // if (sequenceData.getFuture().get().first != reply.end) { + // TEST(true); // tlog peek second attempt ended at a different version + // req.reply.sendError(operation_obsolete()); + // return Void(); + // } + // } else { + // sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); + // } + // reply.begin = req.begin; + // } + // + // req.reply.send(reply); //TraceEvent("LogRouterPeek4", self->dbgid); return Void(); } @@ -645,6 +814,9 @@ ACTOR Future logRouterCore(TLogInterface interf, when(TLogPeekRequest req = waitNext(interf.peekMessages.getFuture())) { addActor.send(logRouterPeekMessages(&logRouterData, req)); } + when(TLogPeekStreamRequest req = waitNext(interf.peekStreamMessages.getFuture())) { + addActor.send(logRouterPeekStream(&logRouterData, req)); + } when(TLogPopRequest req = waitNext(interf.popMessages.getFuture())) { // Request from remote tLog to pop data from LR addActor.send(logRouterPop(&logRouterData, req)); diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 7473d7525a..b4e76d3eb8 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -354,6 +354,10 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T .detail("Has", self->hasMessage()) .detail("End", res.end) .detail("Popped", res.popped.present() ? res.popped.get() : 0); + + // NOTE: delay is needed here since TLog need to be scheduled to response if there are TLog and SS + // on the same machine + wait(delay(0)); return Void(); } } @@ -412,16 +416,12 @@ Future ILogSystem::ServerPeekCursor::getMore(TaskPriority taskID) { if (hasMessage() && !parallelGetMore) return Void(); if (!more.isValid() || more.isReady()) { - more = serverPeekStreamGetMore(this, taskID); - // if (usePeekStream && taskID == TaskPriority::TLogPeekReply) { - // more = serverPeekStreamGetMore(this, taskID); - // } - // if (parallelGetMore || onlySpilled || futureResults.size()) { - // more = serverPeekParallelGetMore(this, taskID); - // } - // else { - // more = serverPeekGetMore(this, taskID); - // } + // more = serverPeekStreamGetMore(this, taskID); + if (parallelGetMore || onlySpilled || futureResults.size()) { + more = serverPeekParallelGetMore(this, taskID); + } else { + more = serverPeekGetMore(this, taskID); + } } return more; } diff --git a/fdbserver/QuietDatabase.actor.cpp b/fdbserver/QuietDatabase.actor.cpp index 47b9a9f2f3..0ce415f7f9 100644 --- a/fdbserver/QuietDatabase.actor.cpp +++ b/fdbserver/QuietDatabase.actor.cpp @@ -634,10 +634,11 @@ ACTOR Future waitForQuietDatabase(Database cx, // In a simulated environment, wait 5 seconds so that workers can move to their optimal locations if (g_network->isSimulated()) wait(delay(5.0)); - + printf("------- 1 -------\n"); // The quiet database check (which runs at the end of every test) will always time out due to active data movement. // To get around this, quiet Database will disable the perpetual wiggle in the setup phase. wait(setPerpetualStorageWiggle(cx, false, LockAware::TRUE)); + printf("------- 2 -------\n"); // Require 3 consecutive successful quiet database checks spaced 2 second apart state int numSuccesses = 0; diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index f437f24637..58f708160a 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1920,7 +1920,8 @@ ACTOR Future peekTLog(TLogData* self, // This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { self->activePeekStreams++; - TraceEvent(SevDebug, "TLogPeekStream", logData->logId); + TraceEvent(SevDebug, "TLogPeekStream", logData->logId).detail("Token", req.reply.getEndpoint().token); + state Version begin = req.begin; state bool onlySpilled = false; if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { @@ -1938,7 +1939,7 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref wait(delay(0, g_network->getCurrentTask())); } catch (Error& e) { self->activePeekStreams--; - TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).detail("Error", e.what()); + TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); if (e.code() == error_code_end_of_stream) { req.reply.sendError(e); From 974bb4b3448ad7d31d69bcffb833fa6d2926135a Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Tue, 20 Jul 2021 17:01:37 -0700 Subject: [PATCH 12/63] add stream peek function to oldTLogServer_x_x.actor.cpp and LogRouter --- fdbserver/LogRouter.actor.cpp | 130 +------- fdbserver/OldTLogServer_4_6.actor.cpp | 216 +++++++++++++- fdbserver/OldTLogServer_6_0.actor.cpp | 319 +++++++++++++++++++- fdbserver/OldTLogServer_6_2.actor.cpp | 409 +++++++++++++++++++++++++- flow/error_definitions.h | 1 + 5 files changed, 918 insertions(+), 157 deletions(-) diff --git a/fdbserver/LogRouter.actor.cpp b/fdbserver/LogRouter.actor.cpp index 8e9ee308a6..eb3ea81326 100644 --- a/fdbserver/LogRouter.actor.cpp +++ b/fdbserver/LogRouter.actor.cpp @@ -518,7 +518,7 @@ ACTOR Future peekLogRouter(LogRouterData* self, sequenceData.send(std::make_pair(begin, reqOnlySpilled)); } } - throw success(); // we've already replied in the past + throw no_action_needed(); // we've already replied in the past } Version endVersion = self->version.get() + 1; @@ -578,8 +578,8 @@ ACTOR Future logRouterPeekStream(LogRouterData* self, TLogPeekStreamReques self->activePeekStreams--; TraceEvent(SevDebug, "TLogPeekStreamEnd", self->dbgid).error(e, true); - if (e.code() == error_code_success) { - continue; + if (e.code() == error_code_no_action_needed) { + return Void(); } else if (e.code() == error_code_end_of_stream) { req.reply.sendError(e); return Void(); @@ -599,7 +599,7 @@ ACTOR Future logRouterPeekMessages(LogRouterData* self, TLogPeekRequest re wait(peekLogRouter(self, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); req.reply.send(reply); } catch (Error& e) { - if (e.code() == error_code_success) { + if (e.code() == error_code_no_action_needed) { req.reply.send(Never()); return Void(); } else if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || @@ -610,128 +610,6 @@ ACTOR Future logRouterPeekMessages(LogRouterData* self, TLogPeekRequest re throw; } } - - // state BinaryWriter messages(Unversioned()); - // state int sequence = -1; - // state UID peekId; - // - // if (req.sequence.present()) { - // try { - // peekId = req.sequence.get().first; - // sequence = req.sequence.get().second; - // if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - // self->peekTracker.find(peekId) == self->peekTracker.end()) { - // throw operation_obsolete(); - // } - // auto& trackerData = self->peekTracker[peekId]; - // if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - // trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled)); - // } - // auto seqBegin = trackerData.sequence_version.begin(); - // // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. - // while (trackerData.sequence_version.size() && - // seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { - // if (seqBegin->second.canBeSet()) { - // seqBegin->second.sendError(operation_obsolete()); - // } - // trackerData.sequence_version.erase(seqBegin); - // seqBegin = trackerData.sequence_version.begin(); - // } - // - // if (trackerData.sequence_version.size() && sequence < seqBegin->first) { - // throw operation_obsolete(); - // } - // - // trackerData.lastUpdate = now(); - // std::pair prevPeekData = wait(trackerData.sequence_version[sequence].getFuture()); - // req.begin = prevPeekData.first; - // req.onlySpilled = prevPeekData.second; - // wait(yield()); - // } catch (Error& e) { - // if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { - // req.reply.sendError(e); - // return Void(); - // } else { - // throw; - // } - // } - // } - // - // //TraceEvent("LogRouterPeek1", self->dbgid).detail("From", - // req.reply.getEndpoint().getPrimaryAddress()).detail("Ver", self->version.get()).detail("Begin", req.begin); if - //(req.returnIfBlocked && self->version.get() < req.begin) { - // //TraceEvent("LogRouterPeek2", self->dbgid); - // req.reply.sendError(end_of_stream()); - // if (req.sequence.present()) { - // auto& trackerData = self->peekTracker[peekId]; - // auto& sequenceData = trackerData.sequence_version[sequence + 1]; - // if (!sequenceData.isSet()) { - // sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); - // } - // } - // return Void(); - // } - // - // if (self->version.get() < req.begin) { - // wait(self->version.whenAtLeast(req.begin)); - // wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); - // } - // - // Version poppedVer = poppedVersion(self, req.tag); - // - // if (poppedVer > req.begin || req.begin < self->startVersion) { - // // This should only happen if a packet is sent multiple times and the reply is not needed. - // // Since we are using popped differently, do not send a reply. - // TraceEvent(SevWarnAlways, "LogRouterPeekPopped", self->dbgid) - // .detail("Begin", req.begin) - // .detail("Popped", poppedVer) - // .detail("Start", self->startVersion); - // req.reply.send(Never()); - // if (req.sequence.present()) { - // auto& trackerData = self->peekTracker[peekId]; - // auto& sequenceData = trackerData.sequence_version[sequence + 1]; - // if (!sequenceData.isSet()) { - // sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); - // } - // } - // return Void(); - // } - // - // Version endVersion = self->version.get() + 1; - // peekMessagesFromMemory(self, req.tag, req.begin, messages, endVersion); - // - // TLogPeekReply reply; - // reply.maxKnownVersion = self->version.get(); - // reply.minKnownCommittedVersion = self->poppedVersion; - // reply.messages = messages.toValue(); - // reply.popped = self->minPopped.get() >= self->startVersion ? self->minPopped.get() : 0; - // reply.end = endVersion; - // reply.onlySpilled = false; - // - // if (req.sequence.present()) { - // auto& trackerData = self->peekTracker[peekId]; - // trackerData.lastUpdate = now(); - // auto& sequenceData = trackerData.sequence_version[sequence + 1]; - // if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - // req.reply.sendError(operation_obsolete()); - // if (!sequenceData.isSet()) - // sequenceData.sendError(operation_obsolete()); - // return Void(); - // } - // if (sequenceData.isSet()) { - // if (sequenceData.getFuture().get().first != reply.end) { - // TEST(true); // tlog peek second attempt ended at a different version - // req.reply.sendError(operation_obsolete()); - // return Void(); - // } - // } else { - // sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); - // } - // reply.begin = req.begin; - // } - // - // req.reply.send(reply); - //TraceEvent("LogRouterPeek4", self->dbgid); return Void(); } diff --git a/fdbserver/OldTLogServer_4_6.actor.cpp b/fdbserver/OldTLogServer_4_6.actor.cpp index d8d6755910..13924707bd 100644 --- a/fdbserver/OldTLogServer_4_6.actor.cpp +++ b/fdbserver/OldTLogServer_4_6.actor.cpp @@ -300,6 +300,7 @@ struct TLogData : NonCopyable { int64_t instanceID; int64_t bytesInput; int64_t bytesDurable; + int activePeekStreams = 0; Version prevVersion; @@ -478,6 +479,7 @@ struct LogData : NonCopyable, public ReferenceCounted { }); specialCounter( cc, "QueueDiskBytesTotal", [tLogData]() { return tLogData->rawPersistentQueue->getStorageBytes().total; }); + specialCounter(cc, "ActivePeekStreams", [tLogData]() {return tLogData->activePeekStreams;}); } ~LogData() { @@ -932,14 +934,15 @@ ACTOR Future tLogPop(TLogData* self, TLogPopRequest req, Reference self, - TLogPeekRequest const& req, + Tag tag, + Version reqBegin, BinaryWriter& messages, Version& endVersion) { - OldTag oldTag = convertTag(req.tag); + OldTag oldTag = convertTag(tag); ASSERT(!messages.getLength()); auto& deque = get_version_messages(self, oldTag); - Version begin = std::max(req.begin, self->persistentDataDurableVersion + 1); + Version begin = std::max(reqBegin, self->persistentDataDurableVersion + 1); auto it = std::lower_bound(deque.begin(), deque.end(), std::make_pair(begin, LengthPrefixedStringRef()), @@ -964,14 +967,212 @@ void peekMessagesFromMemory(Reference self, uint32_t subVersion; rd >> messageLength >> subVersion; messageLength += sizeof(uint16_t) + sizeof(Tag); - messages << messageLength << subVersion << uint16_t(1) << req.tag; + messages << messageLength << subVersion << uint16_t(1) << tag; messageLength -= (sizeof(subVersion) + sizeof(uint16_t) + sizeof(Tag)); messages.serializeBytes(rd.readBytes(messageLength), messageLength); } } } +// Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request +ACTOR Future peekTLog(TLogData* self, + Reference logData, + Version begin, + Tag tag, + bool returnIfBlocked = false, + bool reqOnlySpilled = false, + Optional> sequence = Optional>()) { + state BinaryWriter messages(Unversioned()); + state BinaryWriter messages2(Unversioned()); + state int sequenceNum = -1; + state UID peekId; + state double queueStart = now(); + state OldTag oldTag = convertTag(tag); + + if (sequence.present()) { + peekId = sequence.get().first; + sequenceNum = sequence.get().second; + if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + self->peekTracker.find(peekId) == self->peekTracker.end()) { + throw operation_obsolete(); + } + if (sequenceNum > 0) { + auto& trackerData = self->peekTracker[peekId]; + trackerData.lastUpdate = now(); + Version ver = wait(trackerData.sequence_version[sequenceNum].getFuture()); + begin = std::max(ver, begin); + wait(yield()); + } + } + + if (returnIfBlocked && logData->version.get() < begin) { + throw end_of_stream(); + } + + //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + // Wait until we have something to return that the caller doesn't already have + if (logData->version.get() < begin) { + wait(logData->version.whenAtLeast(begin)); + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } + + state Version endVersion = logData->version.get() + 1; + + Version poppedVer = poppedVersion(logData, oldTag); + if (poppedVer > begin) { + TLogPeekReply rep; + rep.maxKnownVersion = logData->version.get(); + rep.minKnownCommittedVersion = 0; + rep.popped = poppedVer; + rep.end = poppedVer; + rep.onlySpilled = false; + + if (sequence.present()) { + auto& trackerData = self->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + trackerData.lastUpdate = now(); + if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + if (!sequenceData.isSet()) + sequenceData.sendError(operation_obsolete()); + throw operation_obsolete(); + } + if (sequenceData.isSet()) { + if (sequenceData.getFuture().get() != rep.end) { + TEST(true); // 0 tlog peek second attempt ended at a different version + throw operation_obsolete(); + } + } else { + sequenceData.send(rep.end); + } + rep.begin = begin; + } + + return rep; + } + + // grab messages from disk + //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + if (begin <= logData->persistentDataDurableVersion) { + // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We + // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if + // an initial attempt to read from disk results in insufficient data and the required data is no longer in + // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the + // result? + + peekMessagesFromMemory(logData, tag, begin, messages2, endVersion); + + RangeResult kvs = wait(self->persistentData->readRange( + KeyRangeRef(persistTagMessagesKey(logData->logId, oldTag, begin), + persistTagMessagesKey(logData->logId, oldTag, logData->persistentDataDurableVersion + 1)), + SERVER_KNOBS->DESIRED_TOTAL_BYTES, + SERVER_KNOBS->DESIRED_TOTAL_BYTES)); + + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); + + for (auto& kv : kvs) { + auto ver = decodeTagMessagesKey(kv.key); + messages << int32_t(-1) << ver; + + BinaryReader rd(kv.value, Unversioned()); + while (!rd.empty()) { + int32_t messageLength; + uint32_t subVersion; + rd >> messageLength >> subVersion; + messageLength += sizeof(uint16_t) + sizeof(Tag); + messages << messageLength << subVersion << uint16_t(1) << tag; + messageLength -= (sizeof(subVersion) + sizeof(uint16_t) + sizeof(Tag)); + messages.serializeBytes(rd.readBytes(messageLength), messageLength); + } + } + + if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) + endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1; + else + messages.serializeBytes(messages2.toValue()); + } else { + peekMessagesFromMemory(logData, tag, begin, messages, endVersion); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); + } + + TLogPeekReply reply; + reply.maxKnownVersion = logData->version.get(); + reply.minKnownCommittedVersion = 0; + reply.onlySpilled = false; + reply.messages = StringRef(reply.arena, messages.toValue()); + reply.end = endVersion; + + //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); + + if (sequence.present()) { + auto& trackerData = self->peekTracker[peekId]; + trackerData.lastUpdate = now(); + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + if (sequenceData.isSet()) { + if (sequenceData.getFuture().get() != reply.end) { + TEST(true); // 0 tlog peek second attempt ended at a different version (2) + throw operation_obsolete(); + } + } else { + sequenceData.send(reply.end); + } + reply.begin = begin; + } + + return reply; +} + +// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover +ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { + self->activePeekStreams++; + TraceEvent(SevDebug, "TLogPeekStream", logData->logId).detail("Token", req.reply.getEndpoint().token); + + state Version begin = req.begin; + state bool onlySpilled = false; + + req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); + loop { + state TLogPeekStreamReply reply; + try { + wait(req.reply.onReady() && + store(reply.rep, peekTLog(self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled))); + req.reply.send(reply); + begin = reply.rep.end; + onlySpilled = reply.rep.onlySpilled; + wait(delay(0, g_network->getCurrentTask())); + } catch (Error& e) { + self->activePeekStreams--; + TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); + + if (e.code() == error_code_end_of_stream) { + req.reply.sendError(e); + return Void(); + } else if (e.code() == error_code_operation_obsolete) { + // reply stream is cancelled on the client + return Void(); + } else { + throw; + } + } + } +} + ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { + /*try { + TLogPeekReply reply = + wait(peekTLog(self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); + req.reply.send(reply); + } catch (Error& e) { + if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || + e.code() == error_code_end_of_stream) { + req.reply.sendError(e); + return Void(); + } else { + throw; + } + } + + return Void();*/ + state BinaryWriter messages(Unversioned()); state BinaryWriter messages2(Unversioned()); state int sequence = -1; @@ -1061,7 +1262,7 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the // result? - peekMessagesFromMemory(logData, req, messages2, endVersion); + peekMessagesFromMemory(logData, req.tag, req.begin, messages2, endVersion); RangeResult kvs = wait(self->persistentData->readRange( KeyRangeRef(persistTagMessagesKey(logData->logId, oldTag, req.begin), @@ -1092,7 +1293,7 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen else messages.serializeBytes(messages2.toValue()); } else { - peekMessagesFromMemory(logData, req, messages, endVersion); + peekMessagesFromMemory(logData, req.tag, req.begin, messages, endVersion); //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); } @@ -1436,6 +1637,7 @@ ACTOR Future restorePersistentState(TLogData* self, LocalityData locality) recruited.initEndpoints(); DUMPTOKEN(recruited.peekMessages); + DUMPTOKEN(recruited.peekStreamMessages); DUMPTOKEN(recruited.popMessages); DUMPTOKEN(recruited.commit); DUMPTOKEN(recruited.lock); @@ -1575,7 +1777,7 @@ ACTOR Future tLog(IKeyValueStore* persistentData, state TLogData self(tlogId, workerID, persistentData, persistentQueue, db); state Future error = actorCollection(self.sharedActors.getFuture()); - TraceEvent("SharedTlog", tlogId); + TraceEvent("SharedTlog", tlogId).detail("Version", "4.6"); try { wait(restorePersistentState(&self, locality)); diff --git a/fdbserver/OldTLogServer_6_0.actor.cpp b/fdbserver/OldTLogServer_6_0.actor.cpp index 24c97f741c..417b3d3dfd 100644 --- a/fdbserver/OldTLogServer_6_0.actor.cpp +++ b/fdbserver/OldTLogServer_6_0.actor.cpp @@ -276,6 +276,7 @@ struct TLogData : NonCopyable { int64_t targetVolatileBytes; // The number of bytes of mutations this TLog should hold in memory before spilling. int64_t overheadBytesInput; int64_t overheadBytesDurable; + int activePeekStreams = 0; WorkerCache tlogCache; @@ -573,6 +574,7 @@ struct LogData : NonCopyable, public ReferenceCounted { }); specialCounter( cc, "QueueDiskBytesTotal", [tLogData]() { return tLogData->rawPersistentQueue->getStorageBytes().total; }); + specialCounter(cc, "ActivePeekStreams", [tLogData]() { return tLogData->activePeekStreams;}); } ~LogData() { @@ -1173,19 +1175,20 @@ std::deque>& getVersionMessages(Refe }; void peekMessagesFromMemory(Reference self, - TLogPeekRequest const& req, + Tag tag, + Version begin, BinaryWriter& messages, Version& endVersion) { - ASSERT(!messages.getLength()); + ASSERT(!messages.getLength()); - auto& deque = getVersionMessages(self, req.tag); - //TraceEvent("TLogPeekMem", self->dbgid).detail("Tag", req.tag1).detail("PDS", self->persistentDataSequence).detail("PDDS", self->persistentDataDurableSequence).detail("Oldest", map1.empty() ? 0 : map1.begin()->key ).detail("OldestMsgCount", map1.empty() ? 0 : map1.begin()->value.size()); + auto& deque = getVersionMessages(self, tag); + //TraceEvent("TLogPeekMem", self->dbgid).detail("Tag", req.tag1).detail("PDS", self->persistentDataSequence).detail("PDDS", self->persistentDataDurableSequence).detail("Oldest", map1.empty() ? 0 : map1.begin()->key ).detail("OldestMsgCount", map1.empty() ? 0 : map1.begin()->value.size()); - Version begin = std::max(req.begin, self->persistentDataDurableVersion + 1); - auto it = std::lower_bound(deque.begin(), - deque.end(), - std::make_pair(begin, LengthPrefixedStringRef()), - CompareFirst>()); + begin = std::max(begin, self->persistentDataDurableVersion + 1); + auto it = std::lower_bound(deque.begin(), + deque.end(), + std::make_pair(begin, LengthPrefixedStringRef()), + CompareFirst>()); Version currentVersion = -1; for (; it != deque.end(); ++it) { @@ -1204,7 +1207,294 @@ void peekMessagesFromMemory(Reference self, } } +// Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request +ACTOR Future peekTLog(TLogData* self, + Reference logData, + Version begin, + Tag tag, + bool returnIfBlocked = false, + bool reqOnlySpilled = false, + Optional> sequence = Optional>()) { + state BinaryWriter messages(Unversioned()); + state BinaryWriter messages2(Unversioned()); + state int sequenceNum = -1; + state UID peekId; + state double queueStart = now(); + + if (tag.locality == tagLocalityTxs && tag.id >= logData->txsTags && logData->txsTags > 0) { + tag.id = tag.id % logData->txsTags; + } + + // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence requests + // STEP: a. mark obsolete sequence requests; b. wait previous sequence requests are handled in order + if (sequence.present()) { + peekId = sequence.get().first; + sequenceNum = sequence.get().second; + if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + logData->peekTracker.find(peekId) == logData->peekTracker.end()) { + throw operation_obsolete(); + } + auto& trackerData = logData->peekTracker[peekId]; + if (sequenceNum == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { + trackerData.tag = tag; + trackerData.sequence_version[0].send(std::make_pair(begin, reqOnlySpilled)); + } + auto seqBegin = trackerData.sequence_version.begin(); + // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. + while (trackerData.sequence_version.size() && + seqBegin->first <= sequenceNum - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + if (seqBegin->second.canBeSet()) { + seqBegin->second.sendError(operation_obsolete()); + } + trackerData.sequence_version.erase(seqBegin); + seqBegin = trackerData.sequence_version.begin(); + } + + if (trackerData.sequence_version.size() && sequenceNum < seqBegin->first) { + throw operation_obsolete(); + } + + Future> fPrevPeekData = trackerData.sequence_version[sequenceNum].getFuture(); + if (fPrevPeekData.isReady()) { + trackerData.unblockedPeeks++; + double t = now() - trackerData.lastUpdate; + if (t > trackerData.idleMax) + trackerData.idleMax = t; + trackerData.idleTime += t; + } + trackerData.lastUpdate = now(); + std::pair prevPeekData = wait(fPrevPeekData); + begin = std::max(prevPeekData.first, begin); + reqOnlySpilled = prevPeekData.second; + wait(yield()); + } + + state double blockStart = now(); + + if (returnIfBlocked && logData->version.get() < begin) { + if (sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + trackerData.lastUpdate = now(); + if (!sequenceData.isSet()) { + sequenceData.send(std::make_pair(begin, reqOnlySpilled)); + } + } + throw end_of_stream(); + } + + //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + // Wait until we have something to return that the caller doesn't already have + if (logData->version.get() < begin) { + wait(logData->version.whenAtLeast(begin)); + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } + + if (logData->locality != tagLocalitySatellite && tag.locality == tagLocalityLogRouter) { + wait(self->concurrentLogRouterReads.take()); + state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); + wait(delay(0.0, TaskPriority::Low)); + } + + if (begin <= logData->persistentDataDurableVersion && tag.locality != tagLocalityTxs && tag != txsTag) { + // Reading spilled data will almost always imply that the storage server is >5s behind the rest + // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up + // slightly faster over keeping the rest of the cluster operating normally. + // txsTag is only ever peeked on recovery, and we would still wish to prioritize requests + // that impact recovery duration. + wait(delay(0, TaskPriority::TLogSpilledPeekReply)); + } + + state double workStart = now(); + + Version poppedVer = poppedVersion(logData, tag); + if (poppedVer > begin) { + // reply with an empty message and let the next reply start from poppedVer + TLogPeekReply rep; + rep.maxKnownVersion = logData->version.get(); + rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; + rep.popped = poppedVer; + rep.end = poppedVer; + rep.onlySpilled = false; + + // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence + // requests. + if (sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + trackerData.lastUpdate = now(); + if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + if (!sequenceData.isSet()) + sequenceData.sendError(operation_obsolete()); + throw operation_obsolete(); + } + if (sequenceData.isSet()) { + if (sequenceData.getFuture().get().first != rep.end) { + TEST(true); // 1 tlog peek second attempt ended at a different version + throw operation_obsolete(); + } + } else { + sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); + } + rep.begin = begin; + } + + return rep; + } + + state Version endVersion = logData->version.get() + 1; + state bool onlySpilled = false; + + // grab messages from disk + //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + if (begin <= logData->persistentDataDurableVersion) { + // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We + // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if + // an initial attempt to read from disk results in insufficient data and the required data is no longer in + // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the + // result? + + if (reqOnlySpilled) { + endVersion = logData->persistentDataDurableVersion + 1; + } else { + peekMessagesFromMemory(logData, tag, begin, messages2, endVersion); + } + + RangeResult kvs = wait(self->persistentData->readRange( + KeyRangeRef(persistTagMessagesKey(logData->logId, tag, begin), + persistTagMessagesKey(logData->logId, tag, logData->persistentDataDurableVersion + 1)), + SERVER_KNOBS->DESIRED_TOTAL_BYTES, + SERVER_KNOBS->DESIRED_TOTAL_BYTES)); + + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().address).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); + + for (auto& kv : kvs) { + auto ver = decodeTagMessagesKey(kv.key); + messages << VERSION_HEADER << ver; + messages.serializeBytes(kv.value); + } + + if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { + endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1; + onlySpilled = true; + } else { + messages.serializeBytes(messages2.toValue()); + } + } else { + peekMessagesFromMemory(logData, tag, begin, messages, endVersion); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().address).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); + } + + TLogPeekReply reply; + reply.maxKnownVersion = logData->version.get(); + reply.minKnownCommittedVersion = logData->minKnownCommittedVersion; + reply.messages = StringRef(reply.arena, messages.toValue()); + reply.end = endVersion; + reply.onlySpilled = onlySpilled; + + //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", reply.getEndpoint().address); + + if (sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + trackerData.lastUpdate = now(); + + double queueT = blockStart - queueStart; + double blockT = workStart - blockStart; + double workT = now() - workStart; + + trackerData.totalPeeks++; + trackerData.replyBytes += reply.messages.size(); + + if (queueT > trackerData.queueMax) + trackerData.queueMax = queueT; + if (blockT > trackerData.blockMax) + trackerData.blockMax = blockT; + if (workT > trackerData.workMax) + trackerData.workMax = workT; + + trackerData.queueTime += queueT; + trackerData.blockTime += blockT; + trackerData.workTime += workT; + + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + if (!sequenceData.isSet()) { + // It would technically be more correct to .send({req.begin, req.onlySpilled}), as the next + // request might still be in the window of active requests, but LogSystemPeekCursor will + // throw away all future responses upon getting an operation_obsolete(), so computing a + // response will probably be a waste of CPU. + sequenceData.sendError(operation_obsolete()); + } + throw operation_obsolete(); + } + if (sequenceData.isSet()) { + trackerData.duplicatePeeks++; + if (sequenceData.getFuture().get().first != reply.end) { + TEST(true); // 1 tlog peek second attempt ended at a different version (2) + throw operation_obsolete(); + } + } else { + sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); + } + reply.begin = begin; + } + + return reply; +} + +// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover +ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { + self->activePeekStreams++; + TraceEvent(SevDebug, "TLogPeekStream", logData->logId).detail("Token", req.reply.getEndpoint().token); + + state Version begin = req.begin; + state bool onlySpilled = false; + if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { + req.tag.id = req.tag.id % logData->txsTags; + } + req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); + loop { + state TLogPeekStreamReply reply; + try { + wait(req.reply.onReady() && + store(reply.rep, peekTLog(self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled))); + req.reply.send(reply); + begin = reply.rep.end; + onlySpilled = reply.rep.onlySpilled; + wait(delay(0, g_network->getCurrentTask())); + } catch (Error& e) { + self->activePeekStreams--; + TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); + + if (e.code() == error_code_end_of_stream) { + req.reply.sendError(e); + return Void(); + } else if (e.code() == error_code_operation_obsolete) { + // reply stream is cancelled on the client + return Void(); + } else { + throw; + } + } + } +} + ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { + /*try { + TLogPeekReply reply = + wait(peekTLog(self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); + req.reply.send(reply); + } catch (Error& e) { + if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || + e.code() == error_code_end_of_stream) { + req.reply.sendError(e); + return Void(); + } else { + throw; + } + } + + return Void();*/ state BinaryWriter messages(Unversioned()); state BinaryWriter messages2(Unversioned()); state int sequence = -1; @@ -1354,7 +1644,7 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen if (req.onlySpilled) { endVersion = logData->persistentDataDurableVersion + 1; } else { - peekMessagesFromMemory(logData, req, messages2, endVersion); + peekMessagesFromMemory(logData, req.tag, req.begin, messages2, endVersion); } RangeResult kvs = wait(self->persistentData->readRange( @@ -1378,7 +1668,7 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen messages.serializeBytes(messages2.toValue()); } } else { - peekMessagesFromMemory(logData, req, messages, endVersion); + peekMessagesFromMemory(logData, req.tag, req.begin, messages, endVersion); //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().address).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); } @@ -1933,6 +2223,9 @@ ACTOR Future serveTLogInterface(TLogData* self, when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) { logData->addActor.send(tLogPeekMessages(self, req, logData)); } + when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { + logData->addActor.send(tLogPeekStream(self, req, logData)); + } when(TLogPopRequest req = waitNext(tli.popMessages.getFuture())) { logData->addActor.send(tLogPop(self, req, logData)); } @@ -2328,6 +2621,7 @@ ACTOR Future restorePersistentState(TLogData* self, recruited.initEndpoints(); DUMPTOKEN(recruited.peekMessages); + DUMPTOKEN(recruited.peekStreamMessages); DUMPTOKEN(recruited.popMessages); DUMPTOKEN(recruited.commit); DUMPTOKEN(recruited.lock); @@ -2538,6 +2832,7 @@ ACTOR Future tLogStart(TLogData* self, InitializeTLogRequest req, Locality recruited.initEndpoints(); DUMPTOKEN(recruited.peekMessages); + DUMPTOKEN(recruited.peekStreamMessages); DUMPTOKEN(recruited.popMessages); DUMPTOKEN(recruited.commit); DUMPTOKEN(recruited.lock); @@ -2730,7 +3025,7 @@ ACTOR Future tLog(IKeyValueStore* persistentData, state TLogData self(tlogId, workerID, persistentData, persistentQueue, db, degraded, folder); state Future error = actorCollection(self.sharedActors.getFuture()); - TraceEvent("SharedTlog", tlogId); + TraceEvent("SharedTlog", tlogId).detail("Version", "6.0"); try { if (restoreFromDisk) { wait(restorePersistentState(&self, locality, oldLog, recovered, tlogRequests)); diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index 68c125858f..94c35775a6 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -339,6 +339,7 @@ struct TLogData : NonCopyable { int64_t targetVolatileBytes; // The number of bytes of mutations this TLog should hold in memory before spilling. int64_t overheadBytesInput; int64_t overheadBytesDurable; + int activePeekStreams; WorkerCache tlogCache; FlowLock peekMemoryLimiter; @@ -662,6 +663,7 @@ struct LogData : NonCopyable, public ReferenceCounted { cc, "QueueDiskBytesTotal", [tLogData]() { return tLogData->rawPersistentQueue->getStorageBytes().total; }); specialCounter(cc, "PeekMemoryReserved", [tLogData]() { return tLogData->peekMemoryLimiter.activePermits(); }); specialCounter(cc, "PeekMemoryRequestsStalled", [tLogData]() { return tLogData->peekMemoryLimiter.waiters(); }); + specialCounter(cc, "ActivePeekStreams", [tLogData]() {return tLogData->activePeekStreams;}); } ~LogData() { @@ -1488,19 +1490,20 @@ ACTOR Future tLogPop(TLogData* self, TLogPopRequest req, Reference self, - TLogPeekRequest const& req, + Tag tag, + Version begin, BinaryWriter& messages, Version& endVersion) { - ASSERT(!messages.getLength()); + ASSERT(!messages.getLength()); - auto& deque = getVersionMessages(self, req.tag); - //TraceEvent("TLogPeekMem", self->dbgid).detail("Tag", req.tag1).detail("PDS", self->persistentDataSequence).detail("PDDS", self->persistentDataDurableSequence).detail("Oldest", map1.empty() ? 0 : map1.begin()->key ).detail("OldestMsgCount", map1.empty() ? 0 : map1.begin()->value.size()); + auto& deque = getVersionMessages(self, tag); + //TraceEvent("TLogPeekMem", self->dbgid).detail("Tag", req.tag1).detail("PDS", self->persistentDataSequence).detail("PDDS", self->persistentDataDurableSequence).detail("Oldest", map1.empty() ? 0 : map1.begin()->key ).detail("OldestMsgCount", map1.empty() ? 0 : map1.begin()->value.size()); - Version begin = std::max(req.begin, self->persistentDataDurableVersion + 1); - auto it = std::lower_bound(deque.begin(), - deque.end(), - std::make_pair(begin, LengthPrefixedStringRef()), - CompareFirst>()); + begin = std::max(begin, self->persistentDataDurableVersion + 1); + auto it = std::lower_bound(deque.begin(), + deque.end(), + std::make_pair(begin, LengthPrefixedStringRef()), + CompareFirst>()); Version currentVersion = -1; for (; it != deque.end(); ++it) { @@ -1541,7 +1544,387 @@ ACTOR Future> parseMessagesForTag(StringRef commitBlob, T return relevantMessages; } +// Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request +ACTOR Future peekTLog(TLogData* self, + Reference logData, + Version begin, + Tag tag, + bool returnIfBlocked = false, + bool reqOnlySpilled = false, + Optional> sequence = Optional>()) { + state BinaryWriter messages(Unversioned()); + state BinaryWriter messages2(Unversioned()); + state int sequenceNum = -1; + state UID peekId; + state double queueStart = now(); + + if (tag.locality == tagLocalityTxs && tag.id >= logData->txsTags && logData->txsTags > 0) { + tag.id = tag.id % logData->txsTags; + } + + // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence requests + // STEP: a. mark obsolete sequence requests; b. wait previous sequence requests are handled in order + if (sequence.present()) { + peekId = sequence.get().first; + sequenceNum = sequence.get().second; + if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + logData->peekTracker.find(peekId) == logData->peekTracker.end()) { + throw operation_obsolete(); + } + auto& trackerData = logData->peekTracker[peekId]; + if (sequenceNum == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { + trackerData.tag = tag; + trackerData.sequence_version[0].send(std::make_pair(begin, reqOnlySpilled)); + } + auto seqBegin = trackerData.sequence_version.begin(); + // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. + while (trackerData.sequence_version.size() && + seqBegin->first <= sequenceNum - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + if (seqBegin->second.canBeSet()) { + seqBegin->second.sendError(operation_obsolete()); + } + trackerData.sequence_version.erase(seqBegin); + seqBegin = trackerData.sequence_version.begin(); + } + + if (trackerData.sequence_version.size() && sequenceNum < seqBegin->first) { + throw operation_obsolete(); + } + + Future> fPrevPeekData = trackerData.sequence_version[sequenceNum].getFuture(); + if (fPrevPeekData.isReady()) { + trackerData.unblockedPeeks++; + double t = now() - trackerData.lastUpdate; + if (t > trackerData.idleMax) + trackerData.idleMax = t; + trackerData.idleTime += t; + } + trackerData.lastUpdate = now(); + std::pair prevPeekData = wait(fPrevPeekData); + begin = std::max(prevPeekData.first, begin); + reqOnlySpilled = prevPeekData.second; + wait(yield()); + } + + state double blockStart = now(); + + if (returnIfBlocked && logData->version.get() < begin) { + if (sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + trackerData.lastUpdate = now(); + if (!sequenceData.isSet()) { + sequenceData.send(std::make_pair(begin, reqOnlySpilled)); + } + } + throw end_of_stream(); + } + + //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + // Wait until we have something to return that the caller doesn't already have + if (logData->version.get() < begin) { + wait(logData->version.whenAtLeast(begin)); + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } + + if (logData->locality != tagLocalitySatellite && tag.locality == tagLocalityLogRouter) { + wait(self->concurrentLogRouterReads.take()); + state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); + wait(delay(0.0, TaskPriority::Low)); + } + + if (begin <= logData->persistentDataDurableVersion && tag.locality != tagLocalityTxs && tag != txsTag) { + // Reading spilled data will almost always imply that the storage server is >5s behind the rest + // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up + // slightly faster over keeping the rest of the cluster operating normally. + // txsTag is only ever peeked on recovery, and we would still wish to prioritize requests + // that impact recovery duration. + wait(delay(0, TaskPriority::TLogSpilledPeekReply)); + } + + state double workStart = now(); + + Version poppedVer = poppedVersion(logData, tag); + if (poppedVer > begin) { + // reply with an empty message and let the next reply start from poppedVer + TLogPeekReply rep; + rep.maxKnownVersion = logData->version.get(); + rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; + rep.popped = poppedVer; + rep.end = poppedVer; + rep.onlySpilled = false; + + // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence + // requests. + if (sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + trackerData.lastUpdate = now(); + if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + if (!sequenceData.isSet()) + sequenceData.sendError(operation_obsolete()); + throw operation_obsolete(); + } + if (sequenceData.isSet()) { + if (sequenceData.getFuture().get().first != rep.end) { + TEST(true); // 1 tlog peek second attempt ended at a different version + throw operation_obsolete(); + } + } else { + sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); + } + rep.begin = begin; + } + + return rep; + } + + state Version endVersion = logData->version.get() + 1; + state bool onlySpilled = false; + + // grab messages from disk + //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + if (begin <= logData->persistentDataDurableVersion) { + // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We + // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if + // an initial attempt to read from disk results in insufficient data and the required data is no longer in + // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the + // result? + + if (reqOnlySpilled) { + endVersion = logData->persistentDataDurableVersion + 1; + } else { + peekMessagesFromMemory(logData, tag, begin, messages2, endVersion); + } + + if (tag.locality == tagLocalityTxs || tag == txsTag) { + RangeResult kvs = wait(self->persistentData->readRange( + KeyRangeRef(persistTagMessagesKey(logData->logId, tag, begin), + persistTagMessagesKey(logData->logId, tag, logData->persistentDataDurableVersion + 1)), + SERVER_KNOBS->DESIRED_TOTAL_BYTES, + SERVER_KNOBS->DESIRED_TOTAL_BYTES)); + + for (auto& kv : kvs) { + auto ver = decodeTagMessagesKey(kv.key); + messages << VERSION_HEADER << ver; + messages.serializeBytes(kv.value); + } + + if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { + endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1; + onlySpilled = true; + } else { + messages.serializeBytes(messages2.toValue()); + } + } else { + // FIXME: Limit to approximately DESIRED_TOTATL_BYTES somehow. + RangeResult kvrefs = wait(self->persistentData->readRange( + KeyRangeRef( + persistTagMessageRefsKey(logData->logId, tag, begin), + persistTagMessageRefsKey(logData->logId, tag, logData->persistentDataDurableVersion + 1)), + SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1)); + + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); + + state std::vector> commitLocations; + state bool earlyEnd = false; + uint32_t mutationBytes = 0; + state uint64_t commitBytes = 0; + state Version firstVersion = std::numeric_limits::max(); + for (int i = 0; i < kvrefs.size() && i < SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK; i++) { + auto& kv = kvrefs[i]; + VectorRef spilledData; + BinaryReader r(kv.value, AssumeVersion(logData->protocolVersion)); + r >> spilledData; + for (const SpilledData& sd : spilledData) { + if (mutationBytes >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { + earlyEnd = true; + break; + } + if (sd.version >= begin) { + firstVersion = std::min(firstVersion, sd.version); + const IDiskQueue::location end = sd.start.lo + sd.length; + commitLocations.emplace_back(sd.start, end); + // This isn't perfect, because we aren't accounting for page boundaries, but should be + // close enough. + commitBytes += sd.length; + mutationBytes += sd.mutationBytes; + } + } + if (earlyEnd) + break; + } + earlyEnd = earlyEnd || (kvrefs.size() >= SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1); + wait(self->peekMemoryLimiter.take(TaskPriority::TLogSpilledPeekReply, commitBytes)); + state FlowLock::Releaser memoryReservation(self->peekMemoryLimiter, commitBytes); + state std::vector>> messageReads; + messageReads.reserve(commitLocations.size()); + for (const auto& pair : commitLocations) { + messageReads.push_back(self->rawPersistentQueue->read(pair.first, pair.second, CheckHashes::TRUE)); + } + commitLocations.clear(); + wait(waitForAll(messageReads)); + + state Version lastRefMessageVersion = 0; + state int index = 0; + loop { + if (index >= messageReads.size()) + break; + Standalone queueEntryData = messageReads[index].get(); + uint8_t valid; + const uint32_t length = *(uint32_t*)queueEntryData.begin(); + queueEntryData = queueEntryData.substr(4, queueEntryData.size() - 4); + BinaryReader rd(queueEntryData, IncludeVersion()); + state TLogQueueEntry entry; + rd >> entry >> valid; + ASSERT(valid == 0x01); + ASSERT(length + sizeof(valid) == queueEntryData.size()); + + messages << VERSION_HEADER << entry.version; + + std::vector rawMessages = + wait(parseMessagesForTag(entry.messages, tag, logData->logRouterTags)); + for (const StringRef& msg : rawMessages) { + messages.serializeBytes(msg); + } + + lastRefMessageVersion = entry.version; + index++; + } + + messageReads.clear(); + memoryReservation.release(); + + if (earlyEnd) { + endVersion = lastRefMessageVersion + 1; + onlySpilled = true; + } else { + messages.serializeBytes(messages2.toValue()); + } + } + } else { + if (reqOnlySpilled) { + endVersion = logData->persistentDataDurableVersion + 1; + } else { + peekMessagesFromMemory(logData, tag, begin, messages, endVersion); + } + + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); + } + + TLogPeekReply reply; + reply.maxKnownVersion = logData->version.get(); + reply.minKnownCommittedVersion = logData->minKnownCommittedVersion; + reply.messages = StringRef(reply.arena, messages.toValue()); + reply.end = endVersion; + reply.onlySpilled = onlySpilled; + + //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("Tag", req.tag.toString()). + // detail("BeginVer", req.begin).detail("EndVer", reply.end). + // detail("MsgBytes", reply.messages.expectedSize()). + // detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); + + if (sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + trackerData.lastUpdate = now(); + + double queueT = blockStart - queueStart; + double blockT = workStart - blockStart; + double workT = now() - workStart; + + trackerData.totalPeeks++; + trackerData.replyBytes += reply.messages.size(); + + if (queueT > trackerData.queueMax) + trackerData.queueMax = queueT; + if (blockT > trackerData.blockMax) + trackerData.blockMax = blockT; + if (workT > trackerData.workMax) + trackerData.workMax = workT; + + trackerData.queueTime += queueT; + trackerData.blockTime += blockT; + trackerData.workTime += workT; + + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + if (!sequenceData.isSet()) { + // It would technically be more correct to .send({req.begin, req.onlySpilled}), as the next + // request might still be in the window of active requests, but LogSystemPeekCursor will + // throw away all future responses upon getting an operation_obsolete(), so computing a + // response will probably be a waste of CPU. + sequenceData.sendError(operation_obsolete()); + } + throw operation_obsolete(); + } + if (sequenceData.isSet()) { + trackerData.duplicatePeeks++; + if (sequenceData.getFuture().get().first != reply.end) { + TEST(true); // 1 tlog peek second attempt ended at a different version (2) + throw operation_obsolete(); + } + } else { + sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); + } + reply.begin = begin; + } + + return reply; +} + +// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover +ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { + self->activePeekStreams++; + TraceEvent(SevDebug, "TLogPeekStream", logData->logId).detail("Token", req.reply.getEndpoint().token); + + state Version begin = req.begin; + state bool onlySpilled = false; + if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { + req.tag.id = req.tag.id % logData->txsTags; + } + req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); + loop { + state TLogPeekStreamReply reply; + try { + wait(req.reply.onReady() && + store(reply.rep, peekTLog(self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled))); + req.reply.send(reply); + begin = reply.rep.end; + onlySpilled = reply.rep.onlySpilled; + wait(delay(0, g_network->getCurrentTask())); + } catch (Error& e) { + self->activePeekStreams--; + TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); + + if (e.code() == error_code_end_of_stream) { + req.reply.sendError(e); + return Void(); + } else if (e.code() == error_code_operation_obsolete) { + // reply stream is cancelled on the client + return Void(); + } else { + throw; + } + } + } +} + ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { + /*try { + TLogPeekReply reply = + wait(peekTLog(self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); + req.reply.send(reply); + } catch (Error& e) { + if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || + e.code() == error_code_end_of_stream) { + req.reply.sendError(e); + return Void(); + } else { + throw; + } + } + + return Void();*/ state BinaryWriter messages(Unversioned()); state BinaryWriter messages2(Unversioned()); state int sequence = -1; @@ -1691,7 +2074,7 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen if (req.onlySpilled) { endVersion = logData->persistentDataDurableVersion + 1; } else { - peekMessagesFromMemory(logData, req, messages2, endVersion); + peekMessagesFromMemory(logData, req.tag, req.begin, messages2, endVersion); } if (req.tag.locality == tagLocalityTxs || req.tag == txsTag) { @@ -1803,7 +2186,7 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen if (req.onlySpilled) { endVersion = logData->persistentDataDurableVersion + 1; } else { - peekMessagesFromMemory(logData, req, messages, endVersion); + peekMessagesFromMemory(logData, req.tag, req.begin, messages, endVersion); } //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); @@ -2789,6 +3172,7 @@ ACTOR Future restorePersistentState(TLogData* self, recruited.initEndpoints(); DUMPTOKEN(recruited.peekMessages); + DUMPTOKEN(recruited.peekStreamMessages); DUMPTOKEN(recruited.popMessages); DUMPTOKEN(recruited.commit); DUMPTOKEN(recruited.lock); @@ -3020,6 +3404,7 @@ ACTOR Future tLogStart(TLogData* self, InitializeTLogRequest req, Locality recruited.initEndpoints(); DUMPTOKEN(recruited.peekMessages); + DUMPTOKEN(recruited.peekStreamMessages); DUMPTOKEN(recruited.popMessages); DUMPTOKEN(recruited.commit); DUMPTOKEN(recruited.lock); @@ -3219,7 +3604,7 @@ ACTOR Future tLog(IKeyValueStore* persistentData, state TLogData self(tlogId, workerID, persistentData, persistentQueue, db, degraded, folder); state Future error = actorCollection(self.sharedActors.getFuture()); - TraceEvent("SharedTlog", tlogId); + TraceEvent("SharedTlog", tlogId).detail("Version", "6.2"); try { if (restoreFromDisk) { wait(restorePersistentState(&self, locality, oldLog, recovered, tlogRequests)); diff --git a/flow/error_definitions.h b/flow/error_definitions.h index 8ffb54f290..3af43b2d88 100755 --- a/flow/error_definitions.h +++ b/flow/error_definitions.h @@ -33,6 +33,7 @@ // clang-format off ERROR( success, 0, "Success" ) ERROR( end_of_stream, 1, "End of stream" ) +ERROR( no_action_needed, 2, "No action needed" ) ERROR( operation_failed, 1000, "Operation failed") ERROR( wrong_shard_server, 1001, "Shard is not available from this server") ERROR( operation_obsolete, 1002, "Operation result no longer necessary") From 68b08a32247763075524e0be67859961afe9bd35 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Wed, 21 Jul 2021 13:23:16 -0700 Subject: [PATCH 13/63] add TLog streaming peek to OldTLogServer_x_x --- fdbserver/OldTLogServer_4_6.actor.cpp | 344 ++++-------- fdbserver/OldTLogServer_6_0.actor.cpp | 544 ++++++------------ fdbserver/OldTLogServer_6_2.actor.cpp | 769 ++++++++------------------ 3 files changed, 478 insertions(+), 1179 deletions(-) diff --git a/fdbserver/OldTLogServer_4_6.actor.cpp b/fdbserver/OldTLogServer_4_6.actor.cpp index 13924707bd..f84a4a47c8 100644 --- a/fdbserver/OldTLogServer_4_6.actor.cpp +++ b/fdbserver/OldTLogServer_4_6.actor.cpp @@ -479,7 +479,7 @@ struct LogData : NonCopyable, public ReferenceCounted { }); specialCounter( cc, "QueueDiskBytesTotal", [tLogData]() { return tLogData->rawPersistentQueue->getStorageBytes().total; }); - specialCounter(cc, "ActivePeekStreams", [tLogData]() {return tLogData->activePeekStreams;}); + specialCounter(cc, "ActivePeekStreams", [tLogData]() { return tLogData->activePeekStreams; }); } ~LogData() { @@ -982,244 +982,44 @@ ACTOR Future peekTLog(TLogData* self, bool returnIfBlocked = false, bool reqOnlySpilled = false, Optional> sequence = Optional>()) { - state BinaryWriter messages(Unversioned()); - state BinaryWriter messages2(Unversioned()); - state int sequenceNum = -1; - state UID peekId; - state double queueStart = now(); - state OldTag oldTag = convertTag(tag); - - if (sequence.present()) { - peekId = sequence.get().first; - sequenceNum = sequence.get().second; - if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - self->peekTracker.find(peekId) == self->peekTracker.end()) { - throw operation_obsolete(); - } - if (sequenceNum > 0) { - auto& trackerData = self->peekTracker[peekId]; - trackerData.lastUpdate = now(); - Version ver = wait(trackerData.sequence_version[sequenceNum].getFuture()); - begin = std::max(ver, begin); - wait(yield()); - } - } - - if (returnIfBlocked && logData->version.get() < begin) { - throw end_of_stream(); - } - - //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - // Wait until we have something to return that the caller doesn't already have - if (logData->version.get() < begin) { - wait(logData->version.whenAtLeast(begin)); - wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); - } - - state Version endVersion = logData->version.get() + 1; - - Version poppedVer = poppedVersion(logData, oldTag); - if (poppedVer > begin) { - TLogPeekReply rep; - rep.maxKnownVersion = logData->version.get(); - rep.minKnownCommittedVersion = 0; - rep.popped = poppedVer; - rep.end = poppedVer; - rep.onlySpilled = false; - - if (sequence.present()) { - auto& trackerData = self->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - trackerData.lastUpdate = now(); - if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { - if (!sequenceData.isSet()) - sequenceData.sendError(operation_obsolete()); - throw operation_obsolete(); - } - if (sequenceData.isSet()) { - if (sequenceData.getFuture().get() != rep.end) { - TEST(true); // 0 tlog peek second attempt ended at a different version - throw operation_obsolete(); - } - } else { - sequenceData.send(rep.end); - } - rep.begin = begin; - } - - return rep; - } - - // grab messages from disk - //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - if (begin <= logData->persistentDataDurableVersion) { - // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We - // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if - // an initial attempt to read from disk results in insufficient data and the required data is no longer in - // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the - // result? - - peekMessagesFromMemory(logData, tag, begin, messages2, endVersion); - - RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, oldTag, begin), - persistTagMessagesKey(logData->logId, oldTag, logData->persistentDataDurableVersion + 1)), - SERVER_KNOBS->DESIRED_TOTAL_BYTES, - SERVER_KNOBS->DESIRED_TOTAL_BYTES)); - - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); - - for (auto& kv : kvs) { - auto ver = decodeTagMessagesKey(kv.key); - messages << int32_t(-1) << ver; - - BinaryReader rd(kv.value, Unversioned()); - while (!rd.empty()) { - int32_t messageLength; - uint32_t subVersion; - rd >> messageLength >> subVersion; - messageLength += sizeof(uint16_t) + sizeof(Tag); - messages << messageLength << subVersion << uint16_t(1) << tag; - messageLength -= (sizeof(subVersion) + sizeof(uint16_t) + sizeof(Tag)); - messages.serializeBytes(rd.readBytes(messageLength), messageLength); - } - } - - if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) - endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1; - else - messages.serializeBytes(messages2.toValue()); - } else { - peekMessagesFromMemory(logData, tag, begin, messages, endVersion); - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); - } - - TLogPeekReply reply; - reply.maxKnownVersion = logData->version.get(); - reply.minKnownCommittedVersion = 0; - reply.onlySpilled = false; - reply.messages = StringRef(reply.arena, messages.toValue()); - reply.end = endVersion; - - //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); - - if (sequence.present()) { - auto& trackerData = self->peekTracker[peekId]; - trackerData.lastUpdate = now(); - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - if (sequenceData.isSet()) { - if (sequenceData.getFuture().get() != reply.end) { - TEST(true); // 0 tlog peek second attempt ended at a different version (2) - throw operation_obsolete(); - } - } else { - sequenceData.send(reply.end); - } - reply.begin = begin; - } - - return reply; -} - -// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover -ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { - self->activePeekStreams++; - TraceEvent(SevDebug, "TLogPeekStream", logData->logId).detail("Token", req.reply.getEndpoint().token); - - state Version begin = req.begin; - state bool onlySpilled = false; - - req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); - loop { - state TLogPeekStreamReply reply; - try { - wait(req.reply.onReady() && - store(reply.rep, peekTLog(self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled))); - req.reply.send(reply); - begin = reply.rep.end; - onlySpilled = reply.rep.onlySpilled; - wait(delay(0, g_network->getCurrentTask())); - } catch (Error& e) { - self->activePeekStreams--; - TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); - - if (e.code() == error_code_end_of_stream) { - req.reply.sendError(e); - return Void(); - } else if (e.code() == error_code_operation_obsolete) { - // reply stream is cancelled on the client - return Void(); - } else { - throw; - } - } - } -} - -ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { - /*try { - TLogPeekReply reply = - wait(peekTLog(self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); - req.reply.send(reply); - } catch (Error& e) { - if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || - e.code() == error_code_end_of_stream) { - req.reply.sendError(e); - return Void(); - } else { - throw; - } - } - - return Void();*/ - state BinaryWriter messages(Unversioned()); state BinaryWriter messages2(Unversioned()); - state int sequence = -1; + state int sequenceNum = -1; state UID peekId; - state OldTag oldTag = convertTag(req.tag); + state double queueStart = now(); + state OldTag oldTag = convertTag(tag); - if (req.sequence.present()) { - try { - peekId = req.sequence.get().first; - sequence = req.sequence.get().second; - if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - self->peekTracker.find(peekId) == self->peekTracker.end()) { - throw operation_obsolete(); - } - if (sequence > 0) { - auto& trackerData = self->peekTracker[peekId]; - trackerData.lastUpdate = now(); - Version ver = wait(trackerData.sequence_version[sequence].getFuture()); - req.begin = std::max(ver, req.begin); - wait(yield()); - } - } catch (Error& e) { - if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { - req.reply.sendError(e); - return Void(); - } else { - throw; - } + if (sequence.present()) { + peekId = sequence.get().first; + sequenceNum = sequence.get().second; + if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + self->peekTracker.find(peekId) == self->peekTracker.end()) { + throw operation_obsolete(); + } + if (sequenceNum > 0) { + auto& trackerData = self->peekTracker[peekId]; + trackerData.lastUpdate = now(); + Version ver = wait(trackerData.sequence_version[sequenceNum].getFuture()); + begin = std::max(ver, begin); + wait(yield()); } } - if (req.returnIfBlocked && logData->version.get() < req.begin) { - req.reply.sendError(end_of_stream()); - return Void(); + if (returnIfBlocked && logData->version.get() < begin) { + throw end_of_stream(); } //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); // Wait until we have something to return that the caller doesn't already have - if (logData->version.get() < req.begin) { - wait(logData->version.whenAtLeast(req.begin)); + if (logData->version.get() < begin) { + wait(logData->version.whenAtLeast(begin)); wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); } state Version endVersion = logData->version.get() + 1; Version poppedVer = poppedVersion(logData, oldTag); - if (poppedVer > req.begin) { + if (poppedVer > begin) { TLogPeekReply rep; rep.maxKnownVersion = logData->version.get(); rep.minKnownCommittedVersion = 0; @@ -1227,50 +1027,47 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen rep.end = poppedVer; rep.onlySpilled = false; - if (req.sequence.present()) { + if (sequence.present()) { auto& trackerData = self->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; trackerData.lastUpdate = now(); - if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - req.reply.sendError(operation_obsolete()); + if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { if (!sequenceData.isSet()) sequenceData.sendError(operation_obsolete()); - return Void(); + throw operation_obsolete(); } if (sequenceData.isSet()) { if (sequenceData.getFuture().get() != rep.end) { - TEST(true); // tlog peek second attempt ended at a different version - req.reply.sendError(operation_obsolete()); - return Void(); + TEST(true); // 0 tlog peek second attempt ended at a different version + throw operation_obsolete(); } } else { sequenceData.send(rep.end); } - rep.begin = req.begin; + rep.begin = begin; } - req.reply.send(rep); - return Void(); + return rep; } // grab messages from disk //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - if (req.begin <= logData->persistentDataDurableVersion) { + if (begin <= logData->persistentDataDurableVersion) { // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if // an initial attempt to read from disk results in insufficient data and the required data is no longer in // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the // result? - peekMessagesFromMemory(logData, req.tag, req.begin, messages2, endVersion); + peekMessagesFromMemory(logData, tag, begin, messages2, endVersion); RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, oldTag, req.begin), + KeyRangeRef(persistTagMessagesKey(logData->logId, oldTag, begin), persistTagMessagesKey(logData->logId, oldTag, logData->persistentDataDurableVersion + 1)), SERVER_KNOBS->DESIRED_TOTAL_BYTES, SERVER_KNOBS->DESIRED_TOTAL_BYTES)); - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); for (auto& kv : kvs) { auto ver = decodeTagMessagesKey(kv.key); @@ -1282,7 +1079,7 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen uint32_t subVersion; rd >> messageLength >> subVersion; messageLength += sizeof(uint16_t) + sizeof(Tag); - messages << messageLength << subVersion << uint16_t(1) << req.tag; + messages << messageLength << subVersion << uint16_t(1) << tag; messageLength -= (sizeof(subVersion) + sizeof(uint16_t) + sizeof(Tag)); messages.serializeBytes(rd.readBytes(messageLength), messageLength); } @@ -1293,36 +1090,87 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen else messages.serializeBytes(messages2.toValue()); } else { - peekMessagesFromMemory(logData, req.tag, req.begin, messages, endVersion); - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); + peekMessagesFromMemory(logData, tag, begin, messages, endVersion); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); } TLogPeekReply reply; reply.maxKnownVersion = logData->version.get(); reply.minKnownCommittedVersion = 0; reply.onlySpilled = false; - reply.messages = messages.toValue(); + reply.messages = StringRef(reply.arena, messages.toValue()); reply.end = endVersion; //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); - if (req.sequence.present()) { + if (sequence.present()) { auto& trackerData = self->peekTracker[peekId]; trackerData.lastUpdate = now(); - auto& sequenceData = trackerData.sequence_version[sequence + 1]; + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; if (sequenceData.isSet()) { if (sequenceData.getFuture().get() != reply.end) { - TEST(true); // tlog peek second attempt ended at a different version (2) - req.reply.sendError(operation_obsolete()); - return Void(); + TEST(true); // 0 tlog peek second attempt ended at a different version (2) + throw operation_obsolete(); } } else { sequenceData.send(reply.end); } - reply.begin = req.begin; + reply.begin = begin; + } + + return reply; +} + +// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover +ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { + self->activePeekStreams++; + TraceEvent(SevDebug, "TLogPeekStream", logData->logId).detail("Token", req.reply.getEndpoint().token); + + state Version begin = req.begin; + state bool onlySpilled = false; + + req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); + loop { + state TLogPeekStreamReply reply; + try { + wait(req.reply.onReady() && + store(reply.rep, peekTLog(self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled))); + req.reply.send(reply); + begin = reply.rep.end; + onlySpilled = reply.rep.onlySpilled; + wait(delay(0, g_network->getCurrentTask())); + } catch (Error& e) { + self->activePeekStreams--; + TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); + + if (e.code() == error_code_end_of_stream) { + req.reply.sendError(e); + return Void(); + } else if (e.code() == error_code_operation_obsolete) { + // reply stream is cancelled on the client + return Void(); + } else { + throw; + } + } + } +} + +ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { + try { + TLogPeekReply reply = + wait(peekTLog(self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); + req.reply.send(reply); + } catch (Error& e) { + if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || + e.code() == error_code_end_of_stream) { + req.reply.sendError(e); + return Void(); + } else { + throw; + } } - req.reply.send(reply); return Void(); } diff --git a/fdbserver/OldTLogServer_6_0.actor.cpp b/fdbserver/OldTLogServer_6_0.actor.cpp index 58e0e5563d..b76a865bac 100644 --- a/fdbserver/OldTLogServer_6_0.actor.cpp +++ b/fdbserver/OldTLogServer_6_0.actor.cpp @@ -574,7 +574,7 @@ struct LogData : NonCopyable, public ReferenceCounted { }); specialCounter( cc, "QueueDiskBytesTotal", [tLogData]() { return tLogData->rawPersistentQueue->getStorageBytes().total; }); - specialCounter(cc, "ActivePeekStreams", [tLogData]() { return tLogData->activePeekStreams;}); + specialCounter(cc, "ActivePeekStreams", [tLogData]() { return tLogData->activePeekStreams; }); } ~LogData() { @@ -1179,16 +1179,16 @@ void peekMessagesFromMemory(Reference self, Version begin, BinaryWriter& messages, Version& endVersion) { - ASSERT(!messages.getLength()); + ASSERT(!messages.getLength()); - auto& deque = getVersionMessages(self, tag); - //TraceEvent("TLogPeekMem", self->dbgid).detail("Tag", req.tag1).detail("PDS", self->persistentDataSequence).detail("PDDS", self->persistentDataDurableSequence).detail("Oldest", map1.empty() ? 0 : map1.begin()->key ).detail("OldestMsgCount", map1.empty() ? 0 : map1.begin()->value.size()); + auto& deque = getVersionMessages(self, tag); + //TraceEvent("TLogPeekMem", self->dbgid).detail("Tag", req.tag1).detail("PDS", self->persistentDataSequence).detail("PDDS", self->persistentDataDurableSequence).detail("Oldest", map1.empty() ? 0 : map1.begin()->key ).detail("OldestMsgCount", map1.empty() ? 0 : map1.begin()->value.size()); - begin = std::max(begin, self->persistentDataDurableVersion + 1); - auto it = std::lower_bound(deque.begin(), - deque.end(), - std::make_pair(begin, LengthPrefixedStringRef()), - CompareFirst>()); + begin = std::max(begin, self->persistentDataDurableVersion + 1); + auto it = std::lower_bound(deque.begin(), + deque.end(), + std::make_pair(begin, LengthPrefixedStringRef()), + CompareFirst>()); Version currentVersion = -1; for (; it != deque.end(); ++it) { @@ -1215,375 +1215,88 @@ ACTOR Future peekTLog(TLogData* self, bool returnIfBlocked = false, bool reqOnlySpilled = false, Optional> sequence = Optional>()) { - state BinaryWriter messages(Unversioned()); - state BinaryWriter messages2(Unversioned()); - state int sequenceNum = -1; - state UID peekId; - state double queueStart = now(); - - if (tag.locality == tagLocalityTxs && tag.id >= logData->txsTags && logData->txsTags > 0) { - tag.id = tag.id % logData->txsTags; - } - - // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence requests - // STEP: a. mark obsolete sequence requests; b. wait previous sequence requests are handled in order - if (sequence.present()) { - peekId = sequence.get().first; - sequenceNum = sequence.get().second; - if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - logData->peekTracker.find(peekId) == logData->peekTracker.end()) { - throw operation_obsolete(); - } - auto& trackerData = logData->peekTracker[peekId]; - if (sequenceNum == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - trackerData.tag = tag; - trackerData.sequence_version[0].send(std::make_pair(begin, reqOnlySpilled)); - } - auto seqBegin = trackerData.sequence_version.begin(); - // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. - while (trackerData.sequence_version.size() && - seqBegin->first <= sequenceNum - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { - if (seqBegin->second.canBeSet()) { - seqBegin->second.sendError(operation_obsolete()); - } - trackerData.sequence_version.erase(seqBegin); - seqBegin = trackerData.sequence_version.begin(); - } - - if (trackerData.sequence_version.size() && sequenceNum < seqBegin->first) { - throw operation_obsolete(); - } - - Future> fPrevPeekData = trackerData.sequence_version[sequenceNum].getFuture(); - if (fPrevPeekData.isReady()) { - trackerData.unblockedPeeks++; - double t = now() - trackerData.lastUpdate; - if (t > trackerData.idleMax) - trackerData.idleMax = t; - trackerData.idleTime += t; - } - trackerData.lastUpdate = now(); - std::pair prevPeekData = wait(fPrevPeekData); - begin = std::max(prevPeekData.first, begin); - reqOnlySpilled = prevPeekData.second; - wait(yield()); - } - - state double blockStart = now(); - - if (returnIfBlocked && logData->version.get() < begin) { - if (sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - trackerData.lastUpdate = now(); - if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(begin, reqOnlySpilled)); - } - } - throw end_of_stream(); - } - - //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - // Wait until we have something to return that the caller doesn't already have - if (logData->version.get() < begin) { - wait(logData->version.whenAtLeast(begin)); - wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); - } - - if (logData->locality != tagLocalitySatellite && tag.locality == tagLocalityLogRouter) { - wait(self->concurrentLogRouterReads.take()); - state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); - wait(delay(0.0, TaskPriority::Low)); - } - - if (begin <= logData->persistentDataDurableVersion && tag.locality != tagLocalityTxs && tag != txsTag) { - // Reading spilled data will almost always imply that the storage server is >5s behind the rest - // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up - // slightly faster over keeping the rest of the cluster operating normally. - // txsTag is only ever peeked on recovery, and we would still wish to prioritize requests - // that impact recovery duration. - wait(delay(0, TaskPriority::TLogSpilledPeekReply)); - } - - state double workStart = now(); - - Version poppedVer = poppedVersion(logData, tag); - if (poppedVer > begin) { - // reply with an empty message and let the next reply start from poppedVer - TLogPeekReply rep; - rep.maxKnownVersion = logData->version.get(); - rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; - rep.popped = poppedVer; - rep.end = poppedVer; - rep.onlySpilled = false; - - // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence - // requests. - if (sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - trackerData.lastUpdate = now(); - if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { - if (!sequenceData.isSet()) - sequenceData.sendError(operation_obsolete()); - throw operation_obsolete(); - } - if (sequenceData.isSet()) { - if (sequenceData.getFuture().get().first != rep.end) { - TEST(true); // 1 tlog peek second attempt ended at a different version - throw operation_obsolete(); - } - } else { - sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); - } - rep.begin = begin; - } - - return rep; - } - - state Version endVersion = logData->version.get() + 1; - state bool onlySpilled = false; - - // grab messages from disk - //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - if (begin <= logData->persistentDataDurableVersion) { - // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We - // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if - // an initial attempt to read from disk results in insufficient data and the required data is no longer in - // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the - // result? - - if (reqOnlySpilled) { - endVersion = logData->persistentDataDurableVersion + 1; - } else { - peekMessagesFromMemory(logData, tag, begin, messages2, endVersion); - } - - RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, tag, begin), - persistTagMessagesKey(logData->logId, tag, logData->persistentDataDurableVersion + 1)), - SERVER_KNOBS->DESIRED_TOTAL_BYTES, - SERVER_KNOBS->DESIRED_TOTAL_BYTES)); - - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().address).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); - - for (auto& kv : kvs) { - auto ver = decodeTagMessagesKey(kv.key); - messages << VERSION_HEADER << ver; - messages.serializeBytes(kv.value); - } - - if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { - endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1; - onlySpilled = true; - } else { - messages.serializeBytes(messages2.toValue()); - } - } else { - peekMessagesFromMemory(logData, tag, begin, messages, endVersion); - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().address).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); - } - - TLogPeekReply reply; - reply.maxKnownVersion = logData->version.get(); - reply.minKnownCommittedVersion = logData->minKnownCommittedVersion; - reply.messages = StringRef(reply.arena, messages.toValue()); - reply.end = endVersion; - reply.onlySpilled = onlySpilled; - - //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", reply.getEndpoint().address); - - if (sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - trackerData.lastUpdate = now(); - - double queueT = blockStart - queueStart; - double blockT = workStart - blockStart; - double workT = now() - workStart; - - trackerData.totalPeeks++; - trackerData.replyBytes += reply.messages.size(); - - if (queueT > trackerData.queueMax) - trackerData.queueMax = queueT; - if (blockT > trackerData.blockMax) - trackerData.blockMax = blockT; - if (workT > trackerData.workMax) - trackerData.workMax = workT; - - trackerData.queueTime += queueT; - trackerData.blockTime += blockT; - trackerData.workTime += workT; - - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { - if (!sequenceData.isSet()) { - // It would technically be more correct to .send({req.begin, req.onlySpilled}), as the next - // request might still be in the window of active requests, but LogSystemPeekCursor will - // throw away all future responses upon getting an operation_obsolete(), so computing a - // response will probably be a waste of CPU. - sequenceData.sendError(operation_obsolete()); - } - throw operation_obsolete(); - } - if (sequenceData.isSet()) { - trackerData.duplicatePeeks++; - if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // 1 tlog peek second attempt ended at a different version (2) - throw operation_obsolete(); - } - } else { - sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); - } - reply.begin = begin; - } - - return reply; -} - -// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover -ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { - self->activePeekStreams++; - TraceEvent(SevDebug, "TLogPeekStream", logData->logId).detail("Token", req.reply.getEndpoint().token); - - state Version begin = req.begin; - state bool onlySpilled = false; - if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { - req.tag.id = req.tag.id % logData->txsTags; - } - req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); - loop { - state TLogPeekStreamReply reply; - try { - wait(req.reply.onReady() && - store(reply.rep, peekTLog(self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled))); - req.reply.send(reply); - begin = reply.rep.end; - onlySpilled = reply.rep.onlySpilled; - wait(delay(0, g_network->getCurrentTask())); - } catch (Error& e) { - self->activePeekStreams--; - TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); - - if (e.code() == error_code_end_of_stream) { - req.reply.sendError(e); - return Void(); - } else if (e.code() == error_code_operation_obsolete) { - // reply stream is cancelled on the client - return Void(); - } else { - throw; - } - } - } -} - -ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { - /*try { - TLogPeekReply reply = - wait(peekTLog(self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); - req.reply.send(reply); - } catch (Error& e) { - if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || - e.code() == error_code_end_of_stream) { - req.reply.sendError(e); - return Void(); - } else { - throw; - } - } - - return Void();*/ state BinaryWriter messages(Unversioned()); state BinaryWriter messages2(Unversioned()); - state int sequence = -1; + state int sequenceNum = -1; state UID peekId; state double queueStart = now(); - if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { - req.tag.id = req.tag.id % logData->txsTags; + if (tag.locality == tagLocalityTxs && tag.id >= logData->txsTags && logData->txsTags > 0) { + tag.id = tag.id % logData->txsTags; } - if (req.sequence.present()) { - try { - peekId = req.sequence.get().first; - sequence = req.sequence.get().second; - if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - logData->peekTracker.find(peekId) == logData->peekTracker.end()) { - throw operation_obsolete(); - } - auto& trackerData = logData->peekTracker[peekId]; - if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - trackerData.tag = req.tag; - trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled)); - } - auto seqBegin = trackerData.sequence_version.begin(); - while (trackerData.sequence_version.size() && - seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { - if (seqBegin->second.canBeSet()) { - seqBegin->second.sendError(operation_obsolete()); - } - trackerData.sequence_version.erase(seqBegin); - seqBegin = trackerData.sequence_version.begin(); - } - - if (trackerData.sequence_version.size() && sequence < seqBegin->first) { - throw operation_obsolete(); - } - - Future> fPrevPeekData = trackerData.sequence_version[sequence].getFuture(); - if (fPrevPeekData.isReady()) { - trackerData.unblockedPeeks++; - double t = now() - trackerData.lastUpdate; - if (t > trackerData.idleMax) - trackerData.idleMax = t; - trackerData.idleTime += t; - } - trackerData.lastUpdate = now(); - std::pair prevPeekData = wait(fPrevPeekData); - - req.begin = std::max(prevPeekData.first, req.begin); - req.onlySpilled = prevPeekData.second; - wait(yield()); - } catch (Error& e) { - if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { - req.reply.sendError(e); - return Void(); - } else { - throw; - } + // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence requests + // STEP: a. mark obsolete sequence requests; b. wait previous sequence requests are handled in order + if (sequence.present()) { + peekId = sequence.get().first; + sequenceNum = sequence.get().second; + if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + logData->peekTracker.find(peekId) == logData->peekTracker.end()) { + throw operation_obsolete(); } + auto& trackerData = logData->peekTracker[peekId]; + if (sequenceNum == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { + trackerData.tag = tag; + trackerData.sequence_version[0].send(std::make_pair(begin, reqOnlySpilled)); + } + auto seqBegin = trackerData.sequence_version.begin(); + // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. + while (trackerData.sequence_version.size() && + seqBegin->first <= sequenceNum - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + if (seqBegin->second.canBeSet()) { + seqBegin->second.sendError(operation_obsolete()); + } + trackerData.sequence_version.erase(seqBegin); + seqBegin = trackerData.sequence_version.begin(); + } + + if (trackerData.sequence_version.size() && sequenceNum < seqBegin->first) { + throw operation_obsolete(); + } + + Future> fPrevPeekData = trackerData.sequence_version[sequenceNum].getFuture(); + if (fPrevPeekData.isReady()) { + trackerData.unblockedPeeks++; + double t = now() - trackerData.lastUpdate; + if (t > trackerData.idleMax) + trackerData.idleMax = t; + trackerData.idleTime += t; + } + trackerData.lastUpdate = now(); + std::pair prevPeekData = wait(fPrevPeekData); + begin = std::max(prevPeekData.first, begin); + reqOnlySpilled = prevPeekData.second; + wait(yield()); } state double blockStart = now(); - if (req.returnIfBlocked && logData->version.get() < req.begin) { - req.reply.sendError(end_of_stream()); - if (req.sequence.present()) { + if (returnIfBlocked && logData->version.get() < begin) { + if (sequence.present()) { auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + trackerData.lastUpdate = now(); if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); + sequenceData.send(std::make_pair(begin, reqOnlySpilled)); } } - return Void(); + throw end_of_stream(); } //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); // Wait until we have something to return that the caller doesn't already have - if (logData->version.get() < req.begin) { - wait(logData->version.whenAtLeast(req.begin)); + if (logData->version.get() < begin) { + wait(logData->version.whenAtLeast(begin)); wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); } - if (logData->locality != tagLocalitySatellite && req.tag.locality == tagLocalityLogRouter) { + if (logData->locality != tagLocalitySatellite && tag.locality == tagLocalityLogRouter) { wait(self->concurrentLogRouterReads.take()); state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); wait(delay(0.0, TaskPriority::Low)); } - if (req.begin <= logData->persistentDataDurableVersion && req.tag.locality != tagLocalityTxs && req.tag != txsTag) { + if (begin <= logData->persistentDataDurableVersion && tag.locality != tagLocalityTxs && tag != txsTag) { // Reading spilled data will almost always imply that the storage server is >5s behind the rest // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up // slightly faster over keeping the rest of the cluster operating normally. @@ -1594,8 +1307,9 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen state double workStart = now(); - Version poppedVer = poppedVersion(logData, req.tag); - if (poppedVer > req.begin) { + Version poppedVer = poppedVersion(logData, tag); + if (poppedVer > begin) { + // reply with an empty message and let the next reply start from poppedVer TLogPeekReply rep; rep.maxKnownVersion = logData->version.get(); rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; @@ -1603,30 +1317,29 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen rep.end = poppedVer; rep.onlySpilled = false; - if (req.sequence.present()) { + // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence + // requests. + if (sequence.present()) { auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; trackerData.lastUpdate = now(); - if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - req.reply.sendError(operation_obsolete()); + if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { if (!sequenceData.isSet()) sequenceData.sendError(operation_obsolete()); - return Void(); + throw operation_obsolete(); } if (sequenceData.isSet()) { if (sequenceData.getFuture().get().first != rep.end) { - TEST(true); // tlog peek second attempt ended at a different version - req.reply.sendError(operation_obsolete()); - return Void(); + TEST(true); // 1 tlog peek second attempt ended at a different version + throw operation_obsolete(); } } else { sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); } - rep.begin = req.begin; + rep.begin = begin; } - req.reply.send(rep); - return Void(); + return rep; } state Version endVersion = logData->version.get() + 1; @@ -1634,26 +1347,26 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen // grab messages from disk //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - if (req.begin <= logData->persistentDataDurableVersion) { + if (begin <= logData->persistentDataDurableVersion) { // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if // an initial attempt to read from disk results in insufficient data and the required data is no longer in // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the // result? - if (req.onlySpilled) { + if (reqOnlySpilled) { endVersion = logData->persistentDataDurableVersion + 1; } else { - peekMessagesFromMemory(logData, req.tag, req.begin, messages2, endVersion); + peekMessagesFromMemory(logData, tag, begin, messages2, endVersion); } RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, req.tag, req.begin), - persistTagMessagesKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)), + KeyRangeRef(persistTagMessagesKey(logData->logId, tag, begin), + persistTagMessagesKey(logData->logId, tag, logData->persistentDataDurableVersion + 1)), SERVER_KNOBS->DESIRED_TOTAL_BYTES, SERVER_KNOBS->DESIRED_TOTAL_BYTES)); - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().address).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().address).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); for (auto& kv : kvs) { auto ver = decodeTagMessagesKey(kv.key); @@ -1668,20 +1381,20 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen messages.serializeBytes(messages2.toValue()); } } else { - peekMessagesFromMemory(logData, req.tag, req.begin, messages, endVersion); - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().address).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); + peekMessagesFromMemory(logData, tag, begin, messages, endVersion); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().address).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); } TLogPeekReply reply; reply.maxKnownVersion = logData->version.get(); reply.minKnownCommittedVersion = logData->minKnownCommittedVersion; - reply.messages = messages.toValue(); + reply.messages = StringRef(reply.arena, messages.toValue()); reply.end = endVersion; reply.onlySpilled = onlySpilled; - //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().address); + //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", reply.getEndpoint().address); - if (req.sequence.present()) { + if (sequence.present()) { auto& trackerData = logData->peekTracker[peekId]; trackerData.lastUpdate = now(); @@ -1703,27 +1416,84 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen trackerData.blockTime += blockT; trackerData.workTime += workT; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; - if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - req.reply.sendError(operation_obsolete()); - if (!sequenceData.isSet()) + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + if (!sequenceData.isSet()) { + // It would technically be more correct to .send({req.begin, req.onlySpilled}), as the next + // request might still be in the window of active requests, but LogSystemPeekCursor will + // throw away all future responses upon getting an operation_obsolete(), so computing a + // response will probably be a waste of CPU. sequenceData.sendError(operation_obsolete()); - return Void(); + } + throw operation_obsolete(); } if (sequenceData.isSet()) { trackerData.duplicatePeeks++; if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // tlog peek second attempt ended at a different version (2) - req.reply.sendError(operation_obsolete()); - return Void(); + TEST(true); // 1 tlog peek second attempt ended at a different version (2) + throw operation_obsolete(); } } else { sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); } - reply.begin = req.begin; + reply.begin = begin; + } + + return reply; +} + +// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover +ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { + self->activePeekStreams++; + TraceEvent(SevDebug, "TLogPeekStream", logData->logId).detail("Token", req.reply.getEndpoint().token); + + state Version begin = req.begin; + state bool onlySpilled = false; + if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { + req.tag.id = req.tag.id % logData->txsTags; + } + req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); + loop { + state TLogPeekStreamReply reply; + try { + wait(req.reply.onReady() && + store(reply.rep, peekTLog(self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled))); + req.reply.send(reply); + begin = reply.rep.end; + onlySpilled = reply.rep.onlySpilled; + wait(delay(0, g_network->getCurrentTask())); + } catch (Error& e) { + self->activePeekStreams--; + TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); + + if (e.code() == error_code_end_of_stream) { + req.reply.sendError(e); + return Void(); + } else if (e.code() == error_code_operation_obsolete) { + // reply stream is cancelled on the client + return Void(); + } else { + throw; + } + } + } +} + +ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { + try { + TLogPeekReply reply = + wait(peekTLog(self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); + req.reply.send(reply); + } catch (Error& e) { + if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || + e.code() == error_code_end_of_stream) { + req.reply.sendError(e); + return Void(); + } else { + throw; + } } - req.reply.send(reply); return Void(); } @@ -2223,9 +1993,9 @@ ACTOR Future serveTLogInterface(TLogData* self, when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) { logData->addActor.send(tLogPeekMessages(self, req, logData)); } - when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { - logData->addActor.send(tLogPeekStream(self, req, logData)); - } + when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { + logData->addActor.send(tLogPeekStream(self, req, logData)); + } when(TLogPopRequest req = waitNext(tli.popMessages.getFuture())) { logData->addActor.send(tLogPop(self, req, logData)); } diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index 72a5773a72..7f6bea5f57 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -663,7 +663,7 @@ struct LogData : NonCopyable, public ReferenceCounted { cc, "QueueDiskBytesTotal", [tLogData]() { return tLogData->rawPersistentQueue->getStorageBytes().total; }); specialCounter(cc, "PeekMemoryReserved", [tLogData]() { return tLogData->peekMemoryLimiter.activePermits(); }); specialCounter(cc, "PeekMemoryRequestsStalled", [tLogData]() { return tLogData->peekMemoryLimiter.waiters(); }); - specialCounter(cc, "ActivePeekStreams", [tLogData]() {return tLogData->activePeekStreams;}); + specialCounter(cc, "ActivePeekStreams", [tLogData]() { return tLogData->activePeekStreams; }); } ~LogData() { @@ -1443,17 +1443,19 @@ ACTOR Future tLogPopCore(TLogData* self, Tag inputTag, Version to, Referen } uint64_t PoppedVersionLag = logData->persistentDataDurableVersion - logData->queuePoppedVersion; - if ( SERVER_KNOBS->ENABLE_DETAILED_TLOG_POP_TRACE && - (logData->queuePoppedVersion > 0) && //avoid generating massive events at beginning - (tagData->unpoppedRecovered || PoppedVersionLag >= SERVER_KNOBS->TLOG_POPPED_VER_LAG_THRESHOLD_FOR_TLOGPOP_TRACE)) { //when recovery or long lag + if (SERVER_KNOBS->ENABLE_DETAILED_TLOG_POP_TRACE && + (logData->queuePoppedVersion > 0) && // avoid generating massive events at beginning + (tagData->unpoppedRecovered || + PoppedVersionLag >= + SERVER_KNOBS->TLOG_POPPED_VER_LAG_THRESHOLD_FOR_TLOGPOP_TRACE)) { // when recovery or long lag TraceEvent("TLogPopDetails", logData->logId) - .detail("Tag", tagData->tag.toString()) - .detail("UpTo", upTo) - .detail("PoppedVersionLag", PoppedVersionLag) - .detail("MinPoppedTag", logData->minPoppedTag.toString()) - .detail("QueuePoppedVersion", logData->queuePoppedVersion) - .detail("UnpoppedRecovered", tagData->unpoppedRecovered ? "True" : "False") - .detail("NothingPersistent", tagData->nothingPersistent ? "True" : "False"); + .detail("Tag", tagData->tag.toString()) + .detail("UpTo", upTo) + .detail("PoppedVersionLag", PoppedVersionLag) + .detail("MinPoppedTag", logData->minPoppedTag.toString()) + .detail("QueuePoppedVersion", logData->queuePoppedVersion) + .detail("UnpoppedRecovered", tagData->unpoppedRecovered ? "True" : "False") + .detail("NothingPersistent", tagData->nothingPersistent ? "True" : "False"); } if (upTo > logData->persistentDataDurableVersion) wait(tagData->eraseMessagesBefore(upTo, self, logData, TaskPriority::TLogPop)); @@ -1494,16 +1496,16 @@ void peekMessagesFromMemory(Reference self, Version begin, BinaryWriter& messages, Version& endVersion) { - ASSERT(!messages.getLength()); + ASSERT(!messages.getLength()); - auto& deque = getVersionMessages(self, tag); - //TraceEvent("TLogPeekMem", self->dbgid).detail("Tag", req.tag1).detail("PDS", self->persistentDataSequence).detail("PDDS", self->persistentDataDurableSequence).detail("Oldest", map1.empty() ? 0 : map1.begin()->key ).detail("OldestMsgCount", map1.empty() ? 0 : map1.begin()->value.size()); + auto& deque = getVersionMessages(self, tag); + //TraceEvent("TLogPeekMem", self->dbgid).detail("Tag", req.tag1).detail("PDS", self->persistentDataSequence).detail("PDDS", self->persistentDataDurableSequence).detail("Oldest", map1.empty() ? 0 : map1.begin()->key ).detail("OldestMsgCount", map1.empty() ? 0 : map1.begin()->value.size()); - begin = std::max(begin, self->persistentDataDurableVersion + 1); - auto it = std::lower_bound(deque.begin(), - deque.end(), - std::make_pair(begin, LengthPrefixedStringRef()), - CompareFirst>()); + begin = std::max(begin, self->persistentDataDurableVersion + 1); + auto it = std::lower_bound(deque.begin(), + deque.end(), + std::make_pair(begin, LengthPrefixedStringRef()), + CompareFirst>()); Version currentVersion = -1; for (; it != deque.end(); ++it) { @@ -1552,139 +1554,139 @@ ACTOR Future peekTLog(TLogData* self, bool returnIfBlocked = false, bool reqOnlySpilled = false, Optional> sequence = Optional>()) { - state BinaryWriter messages(Unversioned()); - state BinaryWriter messages2(Unversioned()); - state int sequenceNum = -1; - state UID peekId; - state double queueStart = now(); + state BinaryWriter messages(Unversioned()); + state BinaryWriter messages2(Unversioned()); + state int sequenceNum = -1; + state UID peekId; + state double queueStart = now(); - if (tag.locality == tagLocalityTxs && tag.id >= logData->txsTags && logData->txsTags > 0) { - tag.id = tag.id % logData->txsTags; - } + if (tag.locality == tagLocalityTxs && tag.id >= logData->txsTags && logData->txsTags > 0) { + tag.id = tag.id % logData->txsTags; + } - // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence requests - // STEP: a. mark obsolete sequence requests; b. wait previous sequence requests are handled in order - if (sequence.present()) { - peekId = sequence.get().first; - sequenceNum = sequence.get().second; - if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - logData->peekTracker.find(peekId) == logData->peekTracker.end()) { - throw operation_obsolete(); - } - auto& trackerData = logData->peekTracker[peekId]; - if (sequenceNum == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - trackerData.tag = tag; - trackerData.sequence_version[0].send(std::make_pair(begin, reqOnlySpilled)); - } - auto seqBegin = trackerData.sequence_version.begin(); - // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. - while (trackerData.sequence_version.size() && - seqBegin->first <= sequenceNum - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { - if (seqBegin->second.canBeSet()) { - seqBegin->second.sendError(operation_obsolete()); - } - trackerData.sequence_version.erase(seqBegin); - seqBegin = trackerData.sequence_version.begin(); - } + // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence requests + // STEP: a. mark obsolete sequence requests; b. wait previous sequence requests are handled in order + if (sequence.present()) { + peekId = sequence.get().first; + sequenceNum = sequence.get().second; + if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + logData->peekTracker.find(peekId) == logData->peekTracker.end()) { + throw operation_obsolete(); + } + auto& trackerData = logData->peekTracker[peekId]; + if (sequenceNum == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { + trackerData.tag = tag; + trackerData.sequence_version[0].send(std::make_pair(begin, reqOnlySpilled)); + } + auto seqBegin = trackerData.sequence_version.begin(); + // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. + while (trackerData.sequence_version.size() && + seqBegin->first <= sequenceNum - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + if (seqBegin->second.canBeSet()) { + seqBegin->second.sendError(operation_obsolete()); + } + trackerData.sequence_version.erase(seqBegin); + seqBegin = trackerData.sequence_version.begin(); + } - if (trackerData.sequence_version.size() && sequenceNum < seqBegin->first) { - throw operation_obsolete(); - } + if (trackerData.sequence_version.size() && sequenceNum < seqBegin->first) { + throw operation_obsolete(); + } - Future> fPrevPeekData = trackerData.sequence_version[sequenceNum].getFuture(); - if (fPrevPeekData.isReady()) { - trackerData.unblockedPeeks++; - double t = now() - trackerData.lastUpdate; - if (t > trackerData.idleMax) - trackerData.idleMax = t; - trackerData.idleTime += t; - } - trackerData.lastUpdate = now(); - std::pair prevPeekData = wait(fPrevPeekData); - begin = std::max(prevPeekData.first, begin); - reqOnlySpilled = prevPeekData.second; - wait(yield()); - } + Future> fPrevPeekData = trackerData.sequence_version[sequenceNum].getFuture(); + if (fPrevPeekData.isReady()) { + trackerData.unblockedPeeks++; + double t = now() - trackerData.lastUpdate; + if (t > trackerData.idleMax) + trackerData.idleMax = t; + trackerData.idleTime += t; + } + trackerData.lastUpdate = now(); + std::pair prevPeekData = wait(fPrevPeekData); + begin = std::max(prevPeekData.first, begin); + reqOnlySpilled = prevPeekData.second; + wait(yield()); + } - state double blockStart = now(); + state double blockStart = now(); - if (returnIfBlocked && logData->version.get() < begin) { - if (sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - trackerData.lastUpdate = now(); - if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(begin, reqOnlySpilled)); - } - } - throw end_of_stream(); - } + if (returnIfBlocked && logData->version.get() < begin) { + if (sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + trackerData.lastUpdate = now(); + if (!sequenceData.isSet()) { + sequenceData.send(std::make_pair(begin, reqOnlySpilled)); + } + } + throw end_of_stream(); + } - //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - // Wait until we have something to return that the caller doesn't already have - if (logData->version.get() < begin) { - wait(logData->version.whenAtLeast(begin)); - wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); - } + //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + // Wait until we have something to return that the caller doesn't already have + if (logData->version.get() < begin) { + wait(logData->version.whenAtLeast(begin)); + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } - if (logData->locality != tagLocalitySatellite && tag.locality == tagLocalityLogRouter) { - wait(self->concurrentLogRouterReads.take()); - state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); - wait(delay(0.0, TaskPriority::Low)); - } + if (logData->locality != tagLocalitySatellite && tag.locality == tagLocalityLogRouter) { + wait(self->concurrentLogRouterReads.take()); + state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); + wait(delay(0.0, TaskPriority::Low)); + } - if (begin <= logData->persistentDataDurableVersion && tag.locality != tagLocalityTxs && tag != txsTag) { - // Reading spilled data will almost always imply that the storage server is >5s behind the rest - // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up - // slightly faster over keeping the rest of the cluster operating normally. - // txsTag is only ever peeked on recovery, and we would still wish to prioritize requests - // that impact recovery duration. - wait(delay(0, TaskPriority::TLogSpilledPeekReply)); - } + if (begin <= logData->persistentDataDurableVersion && tag.locality != tagLocalityTxs && tag != txsTag) { + // Reading spilled data will almost always imply that the storage server is >5s behind the rest + // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up + // slightly faster over keeping the rest of the cluster operating normally. + // txsTag is only ever peeked on recovery, and we would still wish to prioritize requests + // that impact recovery duration. + wait(delay(0, TaskPriority::TLogSpilledPeekReply)); + } - state double workStart = now(); + state double workStart = now(); - Version poppedVer = poppedVersion(logData, tag); - if (poppedVer > begin) { - // reply with an empty message and let the next reply start from poppedVer - TLogPeekReply rep; - rep.maxKnownVersion = logData->version.get(); - rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; - rep.popped = poppedVer; - rep.end = poppedVer; - rep.onlySpilled = false; + Version poppedVer = poppedVersion(logData, tag); + if (poppedVer > begin) { + // reply with an empty message and let the next reply start from poppedVer + TLogPeekReply rep; + rep.maxKnownVersion = logData->version.get(); + rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; + rep.popped = poppedVer; + rep.end = poppedVer; + rep.onlySpilled = false; - // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence - // requests. - if (sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - trackerData.lastUpdate = now(); - if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { - if (!sequenceData.isSet()) - sequenceData.sendError(operation_obsolete()); - throw operation_obsolete(); - } - if (sequenceData.isSet()) { - if (sequenceData.getFuture().get().first != rep.end) { - TEST(true); // 1 tlog peek second attempt ended at a different version - throw operation_obsolete(); - } - } else { - sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); - } - rep.begin = begin; - } + // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence + // requests. + if (sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + trackerData.lastUpdate = now(); + if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + if (!sequenceData.isSet()) + sequenceData.sendError(operation_obsolete()); + throw operation_obsolete(); + } + if (sequenceData.isSet()) { + if (sequenceData.getFuture().get().first != rep.end) { + TEST(true); // 1 tlog peek second attempt ended at a different version + throw operation_obsolete(); + } + } else { + sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); + } + rep.begin = begin; + } - return rep; - } + return rep; + } - state Version endVersion = logData->version.get() + 1; - state bool onlySpilled = false; + state Version endVersion = logData->version.get() + 1; + state bool onlySpilled = false; - // grab messages from disk - //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - if (begin <= logData->persistentDataDurableVersion) { + // grab messages from disk + //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + if (begin <= logData->persistentDataDurableVersion) { // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if // an initial attempt to read from disk results in insufficient data and the required data is no longer in @@ -1697,390 +1699,10 @@ ACTOR Future peekTLog(TLogData* self, peekMessagesFromMemory(logData, tag, begin, messages2, endVersion); } - if (tag.locality == tagLocalityTxs || tag == txsTag) { - RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, tag, begin), - persistTagMessagesKey(logData->logId, tag, logData->persistentDataDurableVersion + 1)), - SERVER_KNOBS->DESIRED_TOTAL_BYTES, - SERVER_KNOBS->DESIRED_TOTAL_BYTES)); - - for (auto& kv : kvs) { - auto ver = decodeTagMessagesKey(kv.key); - messages << VERSION_HEADER << ver; - messages.serializeBytes(kv.value); - } - - if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { - endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1; - onlySpilled = true; - } else { - messages.serializeBytes(messages2.toValue()); - } - } else { - // FIXME: Limit to approximately DESIRED_TOTATL_BYTES somehow. - RangeResult kvrefs = wait(self->persistentData->readRange( - KeyRangeRef( - persistTagMessageRefsKey(logData->logId, tag, begin), - persistTagMessageRefsKey(logData->logId, tag, logData->persistentDataDurableVersion + 1)), - SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1)); - - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); - - state std::vector> commitLocations; - state bool earlyEnd = false; - uint32_t mutationBytes = 0; - state uint64_t commitBytes = 0; - state Version firstVersion = std::numeric_limits::max(); - for (int i = 0; i < kvrefs.size() && i < SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK; i++) { - auto& kv = kvrefs[i]; - VectorRef spilledData; - BinaryReader r(kv.value, AssumeVersion(logData->protocolVersion)); - r >> spilledData; - for (const SpilledData& sd : spilledData) { - if (mutationBytes >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { - earlyEnd = true; - break; - } - if (sd.version >= begin) { - firstVersion = std::min(firstVersion, sd.version); - const IDiskQueue::location end = sd.start.lo + sd.length; - commitLocations.emplace_back(sd.start, end); - // This isn't perfect, because we aren't accounting for page boundaries, but should be - // close enough. - commitBytes += sd.length; - mutationBytes += sd.mutationBytes; - } - } - if (earlyEnd) - break; - } - earlyEnd = earlyEnd || (kvrefs.size() >= SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1); - wait(self->peekMemoryLimiter.take(TaskPriority::TLogSpilledPeekReply, commitBytes)); - state FlowLock::Releaser memoryReservation(self->peekMemoryLimiter, commitBytes); - state std::vector>> messageReads; - messageReads.reserve(commitLocations.size()); - for (const auto& pair : commitLocations) { - messageReads.push_back(self->rawPersistentQueue->read(pair.first, pair.second, CheckHashes::TRUE)); - } - commitLocations.clear(); - wait(waitForAll(messageReads)); - - state Version lastRefMessageVersion = 0; - state int index = 0; - loop { - if (index >= messageReads.size()) - break; - Standalone queueEntryData = messageReads[index].get(); - uint8_t valid; - const uint32_t length = *(uint32_t*)queueEntryData.begin(); - queueEntryData = queueEntryData.substr(4, queueEntryData.size() - 4); - BinaryReader rd(queueEntryData, IncludeVersion()); - state TLogQueueEntry entry; - rd >> entry >> valid; - ASSERT(valid == 0x01); - ASSERT(length + sizeof(valid) == queueEntryData.size()); - - messages << VERSION_HEADER << entry.version; - - std::vector rawMessages = - wait(parseMessagesForTag(entry.messages, tag, logData->logRouterTags)); - for (const StringRef& msg : rawMessages) { - messages.serializeBytes(msg); - } - - lastRefMessageVersion = entry.version; - index++; - } - - messageReads.clear(); - memoryReservation.release(); - - if (earlyEnd) { - endVersion = lastRefMessageVersion + 1; - onlySpilled = true; - } else { - messages.serializeBytes(messages2.toValue()); - } - } - } else { - if (reqOnlySpilled) { - endVersion = logData->persistentDataDurableVersion + 1; - } else { - peekMessagesFromMemory(logData, tag, begin, messages, endVersion); - } - - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); - } - - TLogPeekReply reply; - reply.maxKnownVersion = logData->version.get(); - reply.minKnownCommittedVersion = logData->minKnownCommittedVersion; - reply.messages = StringRef(reply.arena, messages.toValue()); - reply.end = endVersion; - reply.onlySpilled = onlySpilled; - - //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("Tag", req.tag.toString()). - // detail("BeginVer", req.begin).detail("EndVer", reply.end). - // detail("MsgBytes", reply.messages.expectedSize()). - // detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); - - if (sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - trackerData.lastUpdate = now(); - - double queueT = blockStart - queueStart; - double blockT = workStart - blockStart; - double workT = now() - workStart; - - trackerData.totalPeeks++; - trackerData.replyBytes += reply.messages.size(); - - if (queueT > trackerData.queueMax) - trackerData.queueMax = queueT; - if (blockT > trackerData.blockMax) - trackerData.blockMax = blockT; - if (workT > trackerData.workMax) - trackerData.workMax = workT; - - trackerData.queueTime += queueT; - trackerData.blockTime += blockT; - trackerData.workTime += workT; - - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { - if (!sequenceData.isSet()) { - // It would technically be more correct to .send({req.begin, req.onlySpilled}), as the next - // request might still be in the window of active requests, but LogSystemPeekCursor will - // throw away all future responses upon getting an operation_obsolete(), so computing a - // response will probably be a waste of CPU. - sequenceData.sendError(operation_obsolete()); - } - throw operation_obsolete(); - } - if (sequenceData.isSet()) { - trackerData.duplicatePeeks++; - if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // 1 tlog peek second attempt ended at a different version (2) - throw operation_obsolete(); - } - } else { - sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); - } - reply.begin = begin; - } - - return reply; -} - -// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover -ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { - self->activePeekStreams++; - TraceEvent(SevDebug, "TLogPeekStream", logData->logId).detail("Token", req.reply.getEndpoint().token); - - state Version begin = req.begin; - state bool onlySpilled = false; - if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { - req.tag.id = req.tag.id % logData->txsTags; - } - req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); - loop { - state TLogPeekStreamReply reply; - try { - wait(req.reply.onReady() && - store(reply.rep, peekTLog(self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled))); - req.reply.send(reply); - begin = reply.rep.end; - onlySpilled = reply.rep.onlySpilled; - wait(delay(0, g_network->getCurrentTask())); - } catch (Error& e) { - self->activePeekStreams--; - TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); - - if (e.code() == error_code_end_of_stream) { - req.reply.sendError(e); - return Void(); - } else if (e.code() == error_code_operation_obsolete) { - // reply stream is cancelled on the client - return Void(); - } else { - throw; - } - } - } -} - -ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { - /*try { - TLogPeekReply reply = - wait(peekTLog(self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); - req.reply.send(reply); - } catch (Error& e) { - if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || - e.code() == error_code_end_of_stream) { - req.reply.sendError(e); - return Void(); - } else { - throw; - } - } - - return Void();*/ - state BinaryWriter messages(Unversioned()); - state BinaryWriter messages2(Unversioned()); - state int sequence = -1; - state UID peekId; - state double queueStart = now(); - - if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { - req.tag.id = req.tag.id % logData->txsTags; - } - - if (req.sequence.present()) { - try { - peekId = req.sequence.get().first; - sequence = req.sequence.get().second; - if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - logData->peekTracker.find(peekId) == logData->peekTracker.end()) { - throw operation_obsolete(); - } - auto& trackerData = logData->peekTracker[peekId]; - if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - trackerData.tag = req.tag; - trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled)); - } - auto seqBegin = trackerData.sequence_version.begin(); - // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. - while (trackerData.sequence_version.size() && - seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { - if (seqBegin->second.canBeSet()) { - seqBegin->second.sendError(operation_obsolete()); - } - trackerData.sequence_version.erase(seqBegin); - seqBegin = trackerData.sequence_version.begin(); - } - - if (trackerData.sequence_version.size() && sequence < seqBegin->first) { - throw operation_obsolete(); - } - - Future> fPrevPeekData = trackerData.sequence_version[sequence].getFuture(); - if (fPrevPeekData.isReady()) { - trackerData.unblockedPeeks++; - double t = now() - trackerData.lastUpdate; - if (t > trackerData.idleMax) - trackerData.idleMax = t; - trackerData.idleTime += t; - } - trackerData.lastUpdate = now(); - std::pair prevPeekData = wait(fPrevPeekData); - req.begin = std::max(prevPeekData.first, req.begin); - req.onlySpilled = prevPeekData.second; - wait(yield()); - } catch (Error& e) { - if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { - req.reply.sendError(e); - return Void(); - } else { - throw; - } - } - } - - state double blockStart = now(); - - if (req.returnIfBlocked && logData->version.get() < req.begin) { - req.reply.sendError(end_of_stream()); - if (req.sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; - if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); - } - } - return Void(); - } - - //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - // Wait until we have something to return that the caller doesn't already have - if (logData->version.get() < req.begin) { - wait(logData->version.whenAtLeast(req.begin)); - wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); - } - - if (req.tag.locality == tagLocalityLogRouter) { - wait(self->concurrentLogRouterReads.take()); - state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); - wait(delay(0.0, TaskPriority::Low)); - } - - if (req.begin <= logData->persistentDataDurableVersion && req.tag.locality != tagLocalityTxs && req.tag != txsTag) { - // Reading spilled data will almost always imply that the storage server is >5s behind the rest - // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up - // slightly faster over keeping the rest of the cluster operating normally. - // txsTag is only ever peeked on recovery, and we would still wish to prioritize requests - // that impact recovery duration. - wait(delay(0, TaskPriority::TLogSpilledPeekReply)); - } - - state double workStart = now(); - - Version poppedVer = poppedVersion(logData, req.tag); - if (poppedVer > req.begin) { - TLogPeekReply rep; - rep.maxKnownVersion = logData->version.get(); - rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; - rep.popped = poppedVer; - rep.end = poppedVer; - rep.onlySpilled = false; - - if (req.sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; - trackerData.lastUpdate = now(); - if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - req.reply.sendError(operation_obsolete()); - if (!sequenceData.isSet()) - sequenceData.sendError(operation_obsolete()); - return Void(); - } - if (sequenceData.isSet()) { - if (sequenceData.getFuture().get().first != rep.end) { - TEST(true); // tlog peek second attempt ended at a different version - req.reply.sendError(operation_obsolete()); - return Void(); - } - } else { - sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); - } - rep.begin = req.begin; - } - - req.reply.send(rep); - return Void(); - } - - state Version endVersion = logData->version.get() + 1; - state bool onlySpilled = false; - - // grab messages from disk - //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - if (req.begin <= logData->persistentDataDurableVersion) { - // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We - // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if - // an initial attempt to read from disk results in insufficient data and the required data is no longer in - // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the - // result? - - if (req.onlySpilled) { - endVersion = logData->persistentDataDurableVersion + 1; - } else { - peekMessagesFromMemory(logData, req.tag, req.begin, messages2, endVersion); - } - - if (req.tag.locality == tagLocalityTxs || req.tag == txsTag) { + if (tag.locality == tagLocalityTxs || tag == txsTag) { RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, req.tag, req.begin), - persistTagMessagesKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)), + KeyRangeRef(persistTagMessagesKey(logData->logId, tag, begin), + persistTagMessagesKey(logData->logId, tag, logData->persistentDataDurableVersion + 1)), SERVER_KNOBS->DESIRED_TOTAL_BYTES, SERVER_KNOBS->DESIRED_TOTAL_BYTES)); @@ -2099,12 +1721,11 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen } else { // FIXME: Limit to approximately DESIRED_TOTATL_BYTES somehow. RangeResult kvrefs = wait(self->persistentData->readRange( - KeyRangeRef( - persistTagMessageRefsKey(logData->logId, req.tag, req.begin), - persistTagMessageRefsKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)), + KeyRangeRef(persistTagMessageRefsKey(logData->logId, tag, begin), + persistTagMessageRefsKey(logData->logId, tag, logData->persistentDataDurableVersion + 1)), SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1)); - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); state std::vector> commitLocations; state bool earlyEnd = false; @@ -2121,7 +1742,7 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen earlyEnd = true; break; } - if (sd.version >= req.begin) { + if (sd.version >= begin) { firstVersion = std::min(firstVersion, sd.version); const IDiskQueue::location end = sd.start.lo + sd.length; commitLocations.emplace_back(sd.start, end); @@ -2163,7 +1784,7 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen messages << VERSION_HEADER << entry.version; std::vector rawMessages = - wait(parseMessagesForTag(entry.messages, req.tag, logData->logRouterTags)); + wait(parseMessagesForTag(entry.messages, tag, logData->logRouterTags)); for (const StringRef& msg : rawMessages) { messages.serializeBytes(msg); } @@ -2183,10 +1804,10 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen } } } else { - if (req.onlySpilled) { + if (reqOnlySpilled) { endVersion = logData->persistentDataDurableVersion + 1; } else { - peekMessagesFromMemory(logData, req.tag, req.begin, messages, endVersion); + peekMessagesFromMemory(logData, tag, begin, messages, endVersion); } //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); @@ -2195,13 +1816,16 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen TLogPeekReply reply; reply.maxKnownVersion = logData->version.get(); reply.minKnownCommittedVersion = logData->minKnownCommittedVersion; - reply.messages = messages.toValue(); + reply.messages = StringRef(reply.arena, messages.toValue()); reply.end = endVersion; reply.onlySpilled = onlySpilled; - //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); + //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("Tag", req.tag.toString()). + // detail("BeginVer", req.begin).detail("EndVer", reply.end). + // detail("MsgBytes", reply.messages.expectedSize()). + // detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); - if (req.sequence.present()) { + if (sequence.present()) { auto& trackerData = logData->peekTracker[peekId]; trackerData.lastUpdate = now(); @@ -2223,27 +1847,84 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen trackerData.blockTime += blockT; trackerData.workTime += workT; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; - if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - req.reply.sendError(operation_obsolete()); - if (!sequenceData.isSet()) + auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + if (!sequenceData.isSet()) { + // It would technically be more correct to .send({req.begin, req.onlySpilled}), as the next + // request might still be in the window of active requests, but LogSystemPeekCursor will + // throw away all future responses upon getting an operation_obsolete(), so computing a + // response will probably be a waste of CPU. sequenceData.sendError(operation_obsolete()); - return Void(); + } + throw operation_obsolete(); } if (sequenceData.isSet()) { trackerData.duplicatePeeks++; if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // tlog peek second attempt ended at a different version (2) - req.reply.sendError(operation_obsolete()); - return Void(); + TEST(true); // 1 tlog peek second attempt ended at a different version (2) + throw operation_obsolete(); } } else { sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); } - reply.begin = req.begin; + reply.begin = begin; + } + + return reply; +} + +// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover +ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { + self->activePeekStreams++; + TraceEvent(SevDebug, "TLogPeekStream", logData->logId).detail("Token", req.reply.getEndpoint().token); + + state Version begin = req.begin; + state bool onlySpilled = false; + if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { + req.tag.id = req.tag.id % logData->txsTags; + } + req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); + loop { + state TLogPeekStreamReply reply; + try { + wait(req.reply.onReady() && + store(reply.rep, peekTLog(self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled))); + req.reply.send(reply); + begin = reply.rep.end; + onlySpilled = reply.rep.onlySpilled; + wait(delay(0, g_network->getCurrentTask())); + } catch (Error& e) { + self->activePeekStreams--; + TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); + + if (e.code() == error_code_end_of_stream) { + req.reply.sendError(e); + return Void(); + } else if (e.code() == error_code_operation_obsolete) { + // reply stream is cancelled on the client + return Void(); + } else { + throw; + } + } + } +} + +ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { + try { + TLogPeekReply reply = + wait(peekTLog(self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); + req.reply.send(reply); + } catch (Error& e) { + if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || + e.code() == error_code_end_of_stream) { + req.reply.sendError(e); + return Void(); + } else { + throw; + } } - req.reply.send(reply); return Void(); } @@ -3211,9 +2892,9 @@ ACTOR Future restorePersistentState(TLogData* self, logsByVersion.emplace_back(ver, id1); TraceEvent("TLogPersistentStateRestore", self->dbgid) - .detail("LogId", logData->logId) - .detail("Ver", ver) - .detail("RecoveryCount", logData->recoveryCount); + .detail("LogId", logData->logId) + .detail("Ver", ver) + .detail("RecoveryCount", logData->recoveryCount); // Restore popped keys. Pop operations that took place after the last (committed) updatePersistentDataVersion // might be lost, but that is fine because we will get the corresponding data back, too. tagKeys = prefixRange(rawId.withPrefix(persistTagPoppedKeys.begin)); From cd32478b52a393b5e2d80e7b2222d12d1527cab6 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Thu, 22 Jul 2021 15:45:59 -0700 Subject: [PATCH 14/63] memory error(Simple config) --- fdbrpc/fdbrpc.h | 62 +++++++++++-------------- fdbserver/LogRouter.actor.cpp | 11 ++--- fdbserver/LogSystemPeekCursor.actor.cpp | 33 ++++++------- fdbserver/OldTLogServer_4_6.actor.cpp | 5 +- fdbserver/OldTLogServer_6_0.actor.cpp | 5 +- fdbserver/OldTLogServer_6_2.actor.cpp | 5 +- fdbserver/QuietDatabase.actor.cpp | 2 + fdbserver/SimulatedCluster.actor.cpp | 2 +- fdbserver/TLogServer.actor.cpp | 5 +- 9 files changed, 55 insertions(+), 75 deletions(-) diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index 1f40bb47a7..5a56fc120c 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -277,9 +277,9 @@ struct AcknowledgementReceiver final : FlowReceiver, FastAllocated::operator new; using FastAllocated::operator delete; - uint64_t bytesSent; - uint64_t bytesAcknowledged; - uint64_t bytesLimit; + int64_t bytesSent; + int64_t bytesAcknowledged; + int64_t bytesLimit; Promise ready; Future failures; @@ -300,7 +300,7 @@ struct AcknowledgementReceiver final : FlowReceiver, FastAllocated hold = ready; hold.sendError(message.getError()); } else { - ASSERT(message.get().bytes > bytesAcknowledged); + ASSERT(message.get().bytes > bytesAcknowledged || (message.get().bytes < 0 && bytesAcknowledged > 0)); bytesAcknowledged = message.get().bytes; if (ready.isValid() && bytesSent - bytesAcknowledged < bytesLimit) { Promise hold = ready; @@ -336,6 +336,8 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, void destroy() override { delete this; } void receive(ArenaObjectReader& reader) override { this->addPromiseRef(); + TraceEvent(SevDebug, "NetNotifiedQueueWithAcknowledgementsReceive") + .detail("PromiseRef", this->getPromiseReferenceCount()); ErrorOr> message; reader.deserialize(message); @@ -358,25 +360,19 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, // send an ack immediately if (acknowledgements.getRawEndpoint().isValid()) { acknowledgements.bytesAcknowledged += message.get().asUnderlyingType().expectedSize(); - // int64_t overflow: we need to reset this stream - if (acknowledgements.bytesAcknowledged > std::numeric_limits::max()) { - FlowTransport::transport().sendUnreliable( - SerializeSource>(operation_obsolete()), - acknowledgements.getEndpoint(TaskPriority::ReadSocket), - false); - } else { - FlowTransport::transport().sendUnreliable( - SerializeSource>( - AcknowledgementReply(acknowledgements.bytesAcknowledged)), - acknowledgements.getEndpoint(TaskPriority::ReadSocket), - false); - } + FlowTransport::transport().sendUnreliable( + SerializeSource>( + AcknowledgementReply(acknowledgements.bytesAcknowledged)), + acknowledgements.getEndpoint(TaskPriority::NoDeliverDelay), + false); } } this->send(std::move(message.get().asUnderlyingType())); } this->delPromiseRef(); + TraceEvent(SevDebug, "NetNotifiedQueueWithAcknowledgementsReceiveEnd") + .detail("PromiseRef", this->getPromiseReferenceCount()); } T pop() override { @@ -384,17 +380,10 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, // A reply that has been queued up is being consumed, so send an ack to the server if (acknowledgements.getRawEndpoint().isValid()) { acknowledgements.bytesAcknowledged += res.expectedSize(); - if (acknowledgements.bytesAcknowledged > std::numeric_limits::max()) { - FlowTransport::transport().sendUnreliable( - SerializeSource>(operation_obsolete()), - acknowledgements.getEndpoint(TaskPriority::ReadSocket), - false); - } else { - FlowTransport::transport().sendUnreliable(SerializeSource>( - AcknowledgementReply(acknowledgements.bytesAcknowledged)), - acknowledgements.getEndpoint(TaskPriority::ReadSocket), - false); - } + FlowTransport::transport().sendUnreliable(SerializeSource>( + AcknowledgementReply(acknowledgements.bytesAcknowledged)), + acknowledgements.getEndpoint(TaskPriority::NoDeliverDelay), + false); } return res; } @@ -408,7 +397,8 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, false); } if (isRemoteEndpoint() && !sentError && !acknowledgements.failures.isReady()) { - // The ReplyPromiseStream was cancelled before sending an error, so the storage server must have died + // Notify the client ReplyPromiseStream was cancelled before sending an error, so the storage server must + // have died FlowTransport::transport().sendUnreliable(SerializeSource>>(broken_promise()), getEndpoint(TaskPriority::NoDeliverDelay), false); @@ -431,6 +421,7 @@ public: void send(U&& value) const { if (queue->isRemoteEndpoint()) { if (!queue->acknowledgements.getRawEndpoint().isValid()) { + // register acknowledge receiver on sender and tell the receiver where to send acknowledge messages value.acknowledgeToken = queue->acknowledgements.getEndpoint(TaskPriority::NoDeliverDelay).token; } queue->acknowledgements.bytesSent += value.expectedSize(); @@ -710,16 +701,17 @@ public: template ReplyPromiseStream getReplyStream(const X& value) const { - Future disc = makeDependent(IFailureMonitor::failureMonitor()).onDisconnectOrFailure(getEndpoint()); - auto& p = getReplyPromiseStream(value); - Reference peer; + auto p = getReplyPromiseStream(value); if (queue->isRemoteEndpoint()) { - peer = FlowTransport::transport().sendUnreliable(SerializeSource(value), getEndpoint(), true); + Future disc = + makeDependent(IFailureMonitor::failureMonitor()).onDisconnectOrFailure(getEndpoint()); + Reference peer = + FlowTransport::transport().sendUnreliable(SerializeSource(value), getEndpoint(), true); + // FIXME: defer sending the message until we know the connection is established + endStreamOnDisconnect(disc, p, getEndpoint(), peer); } else { send(value); } - // FIXME: defer sending the message until we know the connection is established - endStreamOnDisconnect(disc, p, getEndpoint(), peer); return p; } diff --git a/fdbserver/LogRouter.actor.cpp b/fdbserver/LogRouter.actor.cpp index eb3ea81326..f91a138b9e 100644 --- a/fdbserver/LogRouter.actor.cpp +++ b/fdbserver/LogRouter.actor.cpp @@ -559,7 +559,7 @@ ACTOR Future peekLogRouter(LogRouterData* self, // This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover ACTOR Future logRouterPeekStream(LogRouterData* self, TLogPeekStreamRequest req) { self->activePeekStreams++; - TraceEvent(SevDebug, "TLogPeekStream", self->dbgid).detail("Token", req.reply.getEndpoint().token); + TraceEvent(SevDebug, "LogRouterPeekStream", self->dbgid).detail("Token", req.reply.getEndpoint().token); state Version begin = req.begin; state bool onlySpilled = false; @@ -576,16 +576,13 @@ ACTOR Future logRouterPeekStream(LogRouterData* self, TLogPeekStreamReques wait(delay(0, g_network->getCurrentTask())); } catch (Error& e) { self->activePeekStreams--; - TraceEvent(SevDebug, "TLogPeekStreamEnd", self->dbgid).error(e, true); + TraceEvent(SevDebug, "LogRouterPeekStreamEnd", self->dbgid).error(e, true); if (e.code() == error_code_no_action_needed) { - return Void(); - } else if (e.code() == error_code_end_of_stream) { + req.reply.sendError(end_of_stream()); + } else if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { req.reply.sendError(e); return Void(); - } else if (e.code() == error_code_operation_obsolete) { - // reply stream is cancelled on the client - return Void(); } else { throw; } diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index b4e76d3eb8..4f3d7a6d0b 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -38,7 +38,8 @@ ACTOR Future tryEstablishPeekStream(ILogSystem::ServerPeekCursor* self) { self->peekReplyStream = self->interf->get().interf().peekStreamMessages.getReplyStream(TLogPeekStreamRequest( self->messageVersion.version, self->tag, self->returnIfBlocked, std::numeric_limits::max())); TraceEvent(SevDebug, "SPC_StreamCreated", self->randomID) - .detail("PeerAddress", self->interf->get().interf().peekStreamMessages.getEndpoint().getPrimaryAddress()); + .detail("PeerAddress", self->interf->get().interf().peekStreamMessages.getEndpoint().getPrimaryAddress()) + .detail("PeerToken", self->interf->get().interf().peekStreamMessages.getEndpoint().token); return Void(); } @@ -350,10 +351,10 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T fPeekReply) : Never())) { updateCursorWithReply(self, res); - TraceEvent("SPC_GetMoreB", self->randomID) - .detail("Has", self->hasMessage()) - .detail("End", res.end) - .detail("Popped", res.popped.present() ? res.popped.get() : 0); + // TraceEvent("SPC_GetMoreB", self->randomID) + // .detail("Has", self->hasMessage()) + // .detail("End", res.end) + // .detail("Popped", res.popped.present() ? res.popped.get() : 0); // NOTE: delay is needed here since TLog need to be scheduled to response if there are TLog and SS // on the same machine @@ -363,7 +364,7 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T } } catch (Error& e) { TraceEvent(SevDebug, "SPC_GetMoreB_Error", self->randomID).detail("Error", e.what()); - if (e.code() == error_code_connection_failed) { + if (e.code() == error_code_connection_failed || e.code() == error_code_operation_obsolete) { self->peekReplyStream.reset(); } else if (e.code() == error_code_end_of_stream) { self->end.reset(self->messageVersion.version); @@ -408,20 +409,20 @@ ACTOR Future serverPeekGetMore(ILogSystem::ServerPeekCursor* self, TaskPri } Future ILogSystem::ServerPeekCursor::getMore(TaskPriority taskID) { - TraceEvent("SPC_GetMore", randomID) - .detail("HasMessage", hasMessage()) - .detail("More", !more.isValid() || more.isReady()) - .detail("MessageVersion", messageVersion.toString()) - .detail("End", end.toString()); + // TraceEvent("SPC_GetMore", randomID) + // .detail("HasMessage", hasMessage()) + // .detail("More", !more.isValid() || more.isReady()) + // .detail("MessageVersion", messageVersion.toString()) + // .detail("End", end.toString()); if (hasMessage() && !parallelGetMore) return Void(); if (!more.isValid() || more.isReady()) { - // more = serverPeekStreamGetMore(this, taskID); - if (parallelGetMore || onlySpilled || futureResults.size()) { - more = serverPeekParallelGetMore(this, taskID); + more = serverPeekStreamGetMore(this, taskID); + /*if (parallelGetMore || onlySpilled || futureResults.size()) { + more = serverPeekParallelGetMore(this, taskID); } else { - more = serverPeekGetMore(this, taskID); - } + more = serverPeekGetMore(this, taskID); + }*/ } return more; } diff --git a/fdbserver/OldTLogServer_4_6.actor.cpp b/fdbserver/OldTLogServer_4_6.actor.cpp index f84a4a47c8..1ea777e550 100644 --- a/fdbserver/OldTLogServer_4_6.actor.cpp +++ b/fdbserver/OldTLogServer_4_6.actor.cpp @@ -1143,12 +1143,9 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref self->activePeekStreams--; TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); - if (e.code() == error_code_end_of_stream) { + if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { req.reply.sendError(e); return Void(); - } else if (e.code() == error_code_operation_obsolete) { - // reply stream is cancelled on the client - return Void(); } else { throw; } diff --git a/fdbserver/OldTLogServer_6_0.actor.cpp b/fdbserver/OldTLogServer_6_0.actor.cpp index b76a865bac..abc7c37517 100644 --- a/fdbserver/OldTLogServer_6_0.actor.cpp +++ b/fdbserver/OldTLogServer_6_0.actor.cpp @@ -1466,12 +1466,9 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref self->activePeekStreams--; TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); - if (e.code() == error_code_end_of_stream) { + if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { req.reply.sendError(e); return Void(); - } else if (e.code() == error_code_operation_obsolete) { - // reply stream is cancelled on the client - return Void(); } else { throw; } diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index 7f6bea5f57..c536aab0d4 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -1897,12 +1897,9 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref self->activePeekStreams--; TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); - if (e.code() == error_code_end_of_stream) { + if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { req.reply.sendError(e); return Void(); - } else if (e.code() == error_code_operation_obsolete) { - // reply stream is cancelled on the client - return Void(); } else { throw; } diff --git a/fdbserver/QuietDatabase.actor.cpp b/fdbserver/QuietDatabase.actor.cpp index 13ef1bff8c..d633352088 100644 --- a/fdbserver/QuietDatabase.actor.cpp +++ b/fdbserver/QuietDatabase.actor.cpp @@ -636,7 +636,9 @@ ACTOR Future waitForQuietDatabase(Database cx, wait(delay(5.0)); // The quiet database check (which runs at the end of every test) will always time out due to active data movement. // To get around this, quiet Database will disable the perpetual wiggle in the setup phase. + printf("------- 1 -------\n"); wait(setPerpetualStorageWiggle(cx, false, LockAware::True)); + printf("------- 2 -------\n"); // Require 3 consecutive successful quiet database checks spaced 2 second apart state int numSuccesses = 0; diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index 5f656f13f1..bcab98b9fc 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -253,7 +253,7 @@ public: // Refer to FDBTypes.h::TLogVersion. Defaults to the maximum supported version. int maxTLogVersion = TLogVersion::MAX_SUPPORTED; // Set true to simplify simulation configs for easier debugging - bool simpleConfig = false; + bool simpleConfig = true; Optional generateFearless, buggify; Optional datacenters, desiredTLogCount, commitProxyCount, grvProxyCount, resolverCount, storageEngineType, stderrSeverity, machineCount, processesPerMachine, coordinators; diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 814c6ba317..3e50dc9132 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1941,12 +1941,9 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref self->activePeekStreams--; TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); - if (e.code() == error_code_end_of_stream) { + if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { req.reply.sendError(e); return Void(); - } else if (e.code() == error_code_operation_obsolete) { - // reply stream is cancelled on the client - return Void(); } else { throw; } From 09214b11603d04eb929ecb78645d72c55dd3c8f4 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Thu, 22 Jul 2021 16:14:55 -0700 Subject: [PATCH 15/63] add stream actor to serveTLogInterf --- fdbserver/OldTLogServer_4_6.actor.cpp | 3 +++ fdbserver/OldTLogServer_6_2.actor.cpp | 3 +++ 2 files changed, 6 insertions(+) diff --git a/fdbserver/OldTLogServer_4_6.actor.cpp b/fdbserver/OldTLogServer_4_6.actor.cpp index 1ea777e550..7c3c067b92 100644 --- a/fdbserver/OldTLogServer_4_6.actor.cpp +++ b/fdbserver/OldTLogServer_4_6.actor.cpp @@ -1337,6 +1337,9 @@ ACTOR Future serveTLogInterface(TLogData* self, when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) { logData->addActor.send(tLogPeekMessages(self, req, logData)); } + when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { + logData->addActor.send(tLogPeekStream(self, req, logData)); + } when(TLogPopRequest req = waitNext(tli.popMessages.getFuture())) { logData->addActor.send(tLogPop(self, req, logData)); } diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index c536aab0d4..3d9749e0d5 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -2437,6 +2437,9 @@ ACTOR Future serveTLogInterface(TLogData* self, when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) { logData->addActor.send(tLogPeekMessages(self, req, logData)); } + when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { + logData->addActor.send(tLogPeekStream(self, req, logData)); + } when(TLogPopRequest req = waitNext(tli.popMessages.getFuture())) { logData->addActor.send(tLogPop(self, req, logData)); } From 15871923312cf85f16f70c728ecfcf1391b92080 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Mon, 26 Jul 2021 09:36:23 -0700 Subject: [PATCH 16/63] temperary change to fix local out-of-order delivery --- fdbrpc/FlowTransport.actor.cpp | 16 ++++++++++------ flow/Knobs.cpp | 2 +- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/fdbrpc/FlowTransport.actor.cpp b/fdbrpc/FlowTransport.actor.cpp index d44483da12..069f2b266b 100644 --- a/fdbrpc/FlowTransport.actor.cpp +++ b/fdbrpc/FlowTransport.actor.cpp @@ -158,7 +158,10 @@ const Endpoint& EndpointMap::insert(NetworkAddressList localAddresses, NetworkMessageReceiver* EndpointMap::get(Endpoint::Token const& token) { uint32_t index = token.second(); if (index < wellKnownEndpointCount && data[index].receiver == nullptr) { - TraceEvent(SevWarnAlways, "WellKnownEndpointNotAdded").detail("Token", token).detail("Index", index).backtrace(); + TraceEvent(SevWarnAlways, "WellKnownEndpointNotAdded") + .detail("Token", token) + .detail("Index", index) + .backtrace(); } if (index < data.size() && data[index].token().first() == token.first() && ((data[index].token().second() & 0xffffffff00000000LL) | index) == token.second()) @@ -799,8 +802,9 @@ Peer::Peer(TransportData* transport, NetworkAddress const& destination) reconnectionDelay(FLOW_KNOBS->INITIAL_RECONNECTION_TIME), compatible(true), outstandingReplies(0), incompatibleProtocolVersionNewer(false), peerReferences(-1), bytesReceived(0), lastDataPacketSentTime(now()), pingLatencies(destination.isPublic() ? FLOW_KNOBS->PING_SAMPLE_AMOUNT : 1), lastLoggedBytesReceived(0), - bytesSent(0), lastLoggedBytesSent(0), timeoutCount(0), lastLoggedTime(0.0), connectOutgoingCount(0), connectIncomingCount(0), - connectFailedCount(0), connectLatencies(destination.isPublic() ? FLOW_KNOBS->NETWORK_CONNECT_SAMPLE_AMOUNT : 1), + bytesSent(0), lastLoggedBytesSent(0), timeoutCount(0), lastLoggedTime(0.0), connectOutgoingCount(0), + connectIncomingCount(0), connectFailedCount(0), + connectLatencies(destination.isPublic() ? FLOW_KNOBS->NETWORK_CONNECT_SAMPLE_AMOUNT : 1), protocolVersion(Reference>>(new AsyncVar>())) { IFailureMonitor::failureMonitor().setStatus(destination, FailureStatus(false)); } @@ -921,9 +925,9 @@ ACTOR static void deliver(TransportData* self, bool inReadSocket) { // We want to run the task at the right priority. If the priority is higher than the current priority (which is // ReadSocket) we can just upgrade. Otherwise we'll context switch so that we don't block other tasks that might run - // with a higher priority. ReplyPromiseStream needs to guarentee that messages are recieved in the order they were - // sent, so even in the case of local delivery those messages need to skip this delay. - if (priority < TaskPriority::ReadSocket || (priority != TaskPriority::NoDeliverDelay && !inReadSocket)) { + // with a higher priority. + // NOTE: don't skip delay(0) when it's local deliver since it could cause out of order object deconstruction. + if (priority < TaskPriority::ReadSocket || !inReadSocket) { wait(delay(0, priority)); } else { g_network->setCurrentTask(priority); diff --git a/flow/Knobs.cpp b/flow/Knobs.cpp index 7a8f1e24f7..25c6e32b80 100644 --- a/flow/Knobs.cpp +++ b/flow/Knobs.cpp @@ -184,7 +184,7 @@ void FlowKnobs::initialize(Randomize randomize, IsSimulated isSimulated) { init( FAST_NETWORK_LATENCY, 800e-6 ); init( SLOW_NETWORK_LATENCY, 100e-3 ); init( MAX_CLOGGING_LATENCY, 0 ); if( randomize && BUGGIFY ) MAX_CLOGGING_LATENCY = 0.1 * deterministicRandom()->random01(); - init( MAX_BUGGIFIED_DELAY, 0 ); if( randomize && BUGGIFY ) MAX_BUGGIFIED_DELAY = 0.2 * deterministicRandom()->random01(); + init( MAX_BUGGIFIED_DELAY, 0 ); // if( randomize && BUGGIFY ) MAX_BUGGIFIED_DELAY = 0.2 * deterministicRandom()->random01(); init( SIM_CONNECT_ERROR_MODE, deterministicRandom()->randomInt(0,3) ); //Tracefiles From c6b0de1264fb16d2701945856aca760e9ee31ba1 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Mon, 26 Jul 2021 09:36:53 -0700 Subject: [PATCH 17/63] problem: OOM --- fdbrpc/fdbrpc.h | 14 +++++++------- fdbserver/LogRouter.actor.cpp | 3 ++- fdbserver/LogSystemPeekCursor.actor.cpp | 8 ++++---- fdbserver/OldTLogServer_4_6.actor.cpp | 2 ++ fdbserver/OldTLogServer_6_0.actor.cpp | 3 ++- fdbserver/OldTLogServer_6_2.actor.cpp | 2 ++ fdbserver/TLogServer.actor.cpp | 3 ++- 7 files changed, 21 insertions(+), 14 deletions(-) diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index 5a56fc120c..2a5cdf7ee4 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -328,16 +328,16 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, NetNotifiedQueueWithAcknowledgements(int futures, int promises, const Endpoint& remoteEndpoint) : NotifiedQueue(futures, promises), FlowReceiver(remoteEndpoint, true) { // A ReplyPromiseStream will be terminated on the server side if the network connection with the client breaks - acknowledgements.failures = tagError( - makeDependent(IFailureMonitor::failureMonitor()).onDisconnect(remoteEndpoint.getPrimaryAddress()), - operation_obsolete()); + acknowledgements.failures = + tagError(makeDependent(IFailureMonitor::failureMonitor()).onDisconnectOrFailure(remoteEndpoint), + operation_obsolete()); } void destroy() override { delete this; } void receive(ArenaObjectReader& reader) override { this->addPromiseRef(); - TraceEvent(SevDebug, "NetNotifiedQueueWithAcknowledgementsReceive") - .detail("PromiseRef", this->getPromiseReferenceCount()); + // TraceEvent(SevDebug, "NetNotifiedQueueWithAcknowledgementsReceive") + // .detail("PromiseRef", this->getPromiseReferenceCount()); ErrorOr> message; reader.deserialize(message); @@ -371,8 +371,8 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, this->send(std::move(message.get().asUnderlyingType())); } this->delPromiseRef(); - TraceEvent(SevDebug, "NetNotifiedQueueWithAcknowledgementsReceiveEnd") - .detail("PromiseRef", this->getPromiseReferenceCount()); + // TraceEvent(SevDebug, "NetNotifiedQueueWithAcknowledgementsReceiveEnd") + // .detail("PromiseRef", this->getPromiseReferenceCount()); } T pop() override { diff --git a/fdbserver/LogRouter.actor.cpp b/fdbserver/LogRouter.actor.cpp index 8527a7a01e..f0c634dfae 100644 --- a/fdbserver/LogRouter.actor.cpp +++ b/fdbserver/LogRouter.actor.cpp @@ -559,7 +559,6 @@ ACTOR Future peekLogRouter(LogRouterData* self, // This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover ACTOR Future logRouterPeekStream(LogRouterData* self, TLogPeekStreamRequest req) { self->activePeekStreams++; - TraceEvent(SevDebug, "LogRouterPeekStream", self->dbgid).detail("Token", req.reply.getEndpoint().token); state Version begin = req.begin; state bool onlySpilled = false; @@ -690,6 +689,8 @@ ACTOR Future logRouterCore(TLogInterface interf, addActor.send(logRouterPeekMessages(&logRouterData, req)); } when(TLogPeekStreamRequest req = waitNext(interf.peekStreamMessages.getFuture())) { + TraceEvent(SevDebug, "LogRouterPeekStream", logRouterData.dbgid) + .detail("Token", interf.peekStreamMessages.getEndpoint().token); addActor.send(logRouterPeekStream(&logRouterData, req)); } when(TLogPopRequest req = waitNext(interf.popMessages.getFuture())) { diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 4f3d7a6d0b..3668b52d08 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -351,10 +351,10 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T fPeekReply) : Never())) { updateCursorWithReply(self, res); - // TraceEvent("SPC_GetMoreB", self->randomID) - // .detail("Has", self->hasMessage()) - // .detail("End", res.end) - // .detail("Popped", res.popped.present() ? res.popped.get() : 0); + TraceEvent("SPC_GetMoreB", self->randomID) + .detail("Has", self->hasMessage()) + .detail("End", res.end) + .detail("Popped", res.popped.present() ? res.popped.get() : 0); // NOTE: delay is needed here since TLog need to be scheduled to response if there are TLog and SS // on the same machine diff --git a/fdbserver/OldTLogServer_4_6.actor.cpp b/fdbserver/OldTLogServer_4_6.actor.cpp index 2b8c8b2cc5..97d9b8efa2 100644 --- a/fdbserver/OldTLogServer_4_6.actor.cpp +++ b/fdbserver/OldTLogServer_4_6.actor.cpp @@ -1338,6 +1338,8 @@ ACTOR Future serveTLogInterface(TLogData* self, logData->addActor.send(tLogPeekMessages(self, req, logData)); } when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { + TraceEvent(SevDebug, "TLogPeekStream", logData->logId) + .detail("Token", tli.peekStreamMessages.getEndpoint().token); logData->addActor.send(tLogPeekStream(self, req, logData)); } when(TLogPopRequest req = waitNext(tli.popMessages.getFuture())) { diff --git a/fdbserver/OldTLogServer_6_0.actor.cpp b/fdbserver/OldTLogServer_6_0.actor.cpp index 3de98dda8e..c9ec74354d 100644 --- a/fdbserver/OldTLogServer_6_0.actor.cpp +++ b/fdbserver/OldTLogServer_6_0.actor.cpp @@ -1445,7 +1445,6 @@ ACTOR Future peekTLog(TLogData* self, // This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { self->activePeekStreams++; - TraceEvent(SevDebug, "TLogPeekStream", logData->logId).detail("Token", req.reply.getEndpoint().token); state Version begin = req.begin; state bool onlySpilled = false; @@ -1991,6 +1990,8 @@ ACTOR Future serveTLogInterface(TLogData* self, logData->addActor.send(tLogPeekMessages(self, req, logData)); } when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { + TraceEvent(SevDebug, "TLogPeekStream", logData->logId) + .detail("Token", tli.peekStreamMessages.getEndpoint().token); logData->addActor.send(tLogPeekStream(self, req, logData)); } when(TLogPopRequest req = waitNext(tli.popMessages.getFuture())) { diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index 64b3ce4008..d4fcc595d0 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -2438,6 +2438,8 @@ ACTOR Future serveTLogInterface(TLogData* self, logData->addActor.send(tLogPeekMessages(self, req, logData)); } when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { + TraceEvent(SevDebug, "TLogPeekStream", logData->logId) + .detail("Token", tli.peekStreamMessages.getEndpoint().token); logData->addActor.send(tLogPeekStream(self, req, logData)); } when(TLogPopRequest req = waitNext(tli.popMessages.getFuture())) { diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 799c8a52da..665a5cfc53 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1920,7 +1920,6 @@ ACTOR Future peekTLog(TLogData* self, // This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { self->activePeekStreams++; - TraceEvent(SevDebug, "TLogPeekStream", logData->logId).detail("Token", req.reply.getEndpoint().token); state Version begin = req.begin; state bool onlySpilled = false; @@ -2465,6 +2464,8 @@ ACTOR Future serveTLogInterface(TLogData* self, } } when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { + TraceEvent(SevDebug, "TLogPeekStream", logData->logId) + .detail("Token", tli.peekStreamMessages.getEndpoint().token); logData->addActor.send(tLogPeekStream(self, req, logData)); } when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) { From 12d4f5c26184b42df960ef3d5ab005c44dbf2341 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Wed, 28 Jul 2021 14:11:25 -0700 Subject: [PATCH 18/63] disable streaming peek for localities < 0 --- fdbclient/FDBTypes.h | 4 ++-- fdbserver/LogRouter.actor.cpp | 30 ++++++++++++++++--------- fdbserver/LogSystemPeekCursor.actor.cpp | 14 +++++++----- fdbserver/OldTLogServer_4_6.actor.cpp | 6 ++++- fdbserver/OldTLogServer_6_0.actor.cpp | 6 ++++- fdbserver/OldTLogServer_6_2.actor.cpp | 6 ++++- fdbserver/QuietDatabase.actor.cpp | 4 ++-- fdbserver/SimulatedCluster.actor.cpp | 2 +- fdbserver/TLogServer.actor.cpp | 6 ++++- 9 files changed, 53 insertions(+), 25 deletions(-) diff --git a/fdbclient/FDBTypes.h b/fdbclient/FDBTypes.h index 25e31d1134..1aa1c31273 100644 --- a/fdbclient/FDBTypes.h +++ b/fdbclient/FDBTypes.h @@ -41,8 +41,8 @@ typedef UID SpanID; enum { tagLocalitySpecial = -1, // tag with this locality means it is invalidTag (id=0), txsTag (id=1), or cacheTag (id=2) tagLocalityLogRouter = -2, - tagLocalityRemoteLog = -3, // tag created by log router for remote tLogs - tagLocalityUpgraded = -4, + tagLocalityRemoteLog = -3, // tag created by log router for remote (aka. not in Primary DC) tLogs + tagLocalityUpgraded = -4, // tlogs with old log format tagLocalitySatellite = -5, tagLocalityLogRouterMapped = -6, // The pseudo tag used by log routers to pop the real LogRouter tag (i.e., -2) tagLocalityTxs = -7, diff --git a/fdbserver/LogRouter.actor.cpp b/fdbserver/LogRouter.actor.cpp index f0c634dfae..945fca63be 100644 --- a/fdbserver/LogRouter.actor.cpp +++ b/fdbserver/LogRouter.actor.cpp @@ -446,6 +446,7 @@ ACTOR Future peekLogRouter(LogRouterData* self, Tag tag, bool returnIfBlocked = false, bool reqOnlySpilled = false, + bool streamReply = false, Optional> sequence = Optional>()) { state BinaryWriter messages(Unversioned()); state int sequenceNum = -1; @@ -518,7 +519,12 @@ ACTOR Future peekLogRouter(LogRouterData* self, sequenceData.send(std::make_pair(begin, reqOnlySpilled)); } } - throw no_action_needed(); // we've already replied in the past + if (streamReply) { + // for streaming reply, we skip the popped part + begin = std::min(poppedVer, self->startVersion); + } else { + throw no_action_needed(); // we've already replied in the past + } } Version endVersion = self->version.get() + 1; @@ -568,18 +574,20 @@ ACTOR Future logRouterPeekStream(LogRouterData* self, TLogPeekStreamReques state TLogPeekStreamReply reply; try { wait(req.reply.onReady() && - store(reply.rep, peekLogRouter(self, req.begin, req.tag, req.returnIfBlocked, onlySpilled))); + store(reply.rep, peekLogRouter(self, req.begin, req.tag, req.returnIfBlocked, onlySpilled, true))); req.reply.send(reply); begin = reply.rep.end; onlySpilled = reply.rep.onlySpilled; - wait(delay(0, g_network->getCurrentTask())); + if (reply.rep.end > self->version.get()) { + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } else { + wait(delay(0, g_network->getCurrentTask())); + } } catch (Error& e) { self->activePeekStreams--; - TraceEvent(SevDebug, "LogRouterPeekStreamEnd", self->dbgid).error(e, true); - if (e.code() == error_code_no_action_needed) { - req.reply.sendError(end_of_stream()); - } else if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { + TraceEvent(SevDebug, "LogRouterPeekStreamEnd", self->dbgid).error(e, true); + if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { req.reply.sendError(e); return Void(); } else { @@ -592,7 +600,7 @@ ACTOR Future logRouterPeekStream(LogRouterData* self, TLogPeekStreamReques ACTOR Future logRouterPeekMessages(LogRouterData* self, TLogPeekRequest req) { try { TLogPeekReply reply = - wait(peekLogRouter(self, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); + wait(peekLogRouter(self, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, false, req.sequence)); req.reply.send(reply); } catch (Error& e) { if (e.code() == error_code_no_action_needed) { @@ -689,9 +697,11 @@ ACTOR Future logRouterCore(TLogInterface interf, addActor.send(logRouterPeekMessages(&logRouterData, req)); } when(TLogPeekStreamRequest req = waitNext(interf.peekStreamMessages.getFuture())) { - TraceEvent(SevDebug, "LogRouterPeekStream", logRouterData.dbgid) + // addActor.send(logRouterPeekStream(&logRouterData, req)); + // FIXME: temporarily disable streaming peek from LogRouter + TraceEvent(SevError, "LogRouterPeekStream", logRouterData.dbgid) .detail("Token", interf.peekStreamMessages.getEndpoint().token); - addActor.send(logRouterPeekStream(&logRouterData, req)); + req.reply.sendError(operation_failed()); } when(TLogPopRequest req = waitNext(interf.popMessages.getFuture())) { // Request from remote tLog to pop data from LR diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 3668b52d08..b097d88c8f 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -358,7 +358,7 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T // NOTE: delay is needed here since TLog need to be scheduled to response if there are TLog and SS // on the same machine - wait(delay(0)); + wait(delay(0, taskID)); return Void(); } } @@ -417,12 +417,14 @@ Future ILogSystem::ServerPeekCursor::getMore(TaskPriority taskID) { if (hasMessage() && !parallelGetMore) return Void(); if (!more.isValid() || more.isReady()) { - more = serverPeekStreamGetMore(this, taskID); - /*if (parallelGetMore || onlySpilled || futureResults.size()) { - more = serverPeekParallelGetMore(this, taskID); + // TODO: remove locality check when log router support streaming peek + if (usePeekStream && tag.locality >= 0) { + more = serverPeekStreamGetMore(this, taskID); + } else if (parallelGetMore || onlySpilled || futureResults.size()) { + more = serverPeekParallelGetMore(this, taskID); } else { - more = serverPeekGetMore(this, taskID); - }*/ + more = serverPeekGetMore(this, taskID); + } } return more; } diff --git a/fdbserver/OldTLogServer_4_6.actor.cpp b/fdbserver/OldTLogServer_4_6.actor.cpp index 97d9b8efa2..e5645223c8 100644 --- a/fdbserver/OldTLogServer_4_6.actor.cpp +++ b/fdbserver/OldTLogServer_4_6.actor.cpp @@ -1138,7 +1138,11 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref req.reply.send(reply); begin = reply.rep.end; onlySpilled = reply.rep.onlySpilled; - wait(delay(0, g_network->getCurrentTask())); + if (reply.rep.end > logData->version.get()) { + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } else { + wait(delay(0, g_network->getCurrentTask())); + } } catch (Error& e) { self->activePeekStreams--; TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); diff --git a/fdbserver/OldTLogServer_6_0.actor.cpp b/fdbserver/OldTLogServer_6_0.actor.cpp index c9ec74354d..bdd2a64bc3 100644 --- a/fdbserver/OldTLogServer_6_0.actor.cpp +++ b/fdbserver/OldTLogServer_6_0.actor.cpp @@ -1460,7 +1460,11 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref req.reply.send(reply); begin = reply.rep.end; onlySpilled = reply.rep.onlySpilled; - wait(delay(0, g_network->getCurrentTask())); + if (reply.rep.end > logData->version.get()) { + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } else { + wait(delay(0, g_network->getCurrentTask())); + } } catch (Error& e) { self->activePeekStreams--; TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index d4fcc595d0..16077d648f 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -1892,7 +1892,11 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref req.reply.send(reply); begin = reply.rep.end; onlySpilled = reply.rep.onlySpilled; - wait(delay(0, g_network->getCurrentTask())); + if (reply.rep.end > logData->version.get()) { + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } else { + wait(delay(0, g_network->getCurrentTask())); + } } catch (Error& e) { self->activePeekStreams--; TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); diff --git a/fdbserver/QuietDatabase.actor.cpp b/fdbserver/QuietDatabase.actor.cpp index f5e5443ca0..305543f8c9 100644 --- a/fdbserver/QuietDatabase.actor.cpp +++ b/fdbserver/QuietDatabase.actor.cpp @@ -639,9 +639,9 @@ ACTOR Future waitForQuietDatabase(Database cx, wait(delay(5.0)); // The quiet database check (which runs at the end of every test) will always time out due to active data movement. // To get around this, quiet Database will disable the perpetual wiggle in the setup phase. - printf("------- 1 -------\n"); + printf("Set perpetual_storage_wiggle=0 ...\n"); wait(setPerpetualStorageWiggle(cx, false, LockAware::True)); - printf("------- 2 -------\n"); + printf("Set perpetual_storage_wiggle=0 Done.\n"); // Require 3 consecutive successful quiet database checks spaced 2 second apart state int numSuccesses = 0; diff --git a/fdbserver/SimulatedCluster.actor.cpp b/fdbserver/SimulatedCluster.actor.cpp index bcab98b9fc..5f656f13f1 100644 --- a/fdbserver/SimulatedCluster.actor.cpp +++ b/fdbserver/SimulatedCluster.actor.cpp @@ -253,7 +253,7 @@ public: // Refer to FDBTypes.h::TLogVersion. Defaults to the maximum supported version. int maxTLogVersion = TLogVersion::MAX_SUPPORTED; // Set true to simplify simulation configs for easier debugging - bool simpleConfig = true; + bool simpleConfig = false; Optional generateFearless, buggify; Optional datacenters, desiredTLogCount, commitProxyCount, grvProxyCount, resolverCount, storageEngineType, stderrSeverity, machineCount, processesPerMachine, coordinators; diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 665a5cfc53..87f7770b5c 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1935,7 +1935,11 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref req.reply.send(reply); begin = reply.rep.end; onlySpilled = reply.rep.onlySpilled; - wait(delay(0, g_network->getCurrentTask())); + if (reply.rep.end > logData->version.get()) { + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } else { + wait(delay(0, g_network->getCurrentTask())); + } } catch (Error& e) { self->activePeekStreams--; TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); From 1c4bce17aab8f4f57615a25108bbc59175a942aa Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Fri, 30 Jul 2021 19:08:22 -0700 Subject: [PATCH 19/63] revert code refactor --- fdbserver/LogRouter.actor.cpp | 266 ++++++-------- fdbserver/LogSystemPeekCursor.actor.cpp | 2 +- fdbserver/OldTLogServer_4_6.actor.cpp | 159 +++++++- fdbserver/OldTLogServer_6_0.actor.cpp | 239 +++++++++++- fdbserver/OldTLogServer_6_2.actor.cpp | 450 +++++++++++++++++++---- fdbserver/TLogServer.actor.cpp | 467 ++++++++++++++++++++---- 6 files changed, 1279 insertions(+), 304 deletions(-) diff --git a/fdbserver/LogRouter.actor.cpp b/fdbserver/LogRouter.actor.cpp index 945fca63be..b0e39dbb9e 100644 --- a/fdbserver/LogRouter.actor.cpp +++ b/fdbserver/LogRouter.actor.cpp @@ -441,153 +441,50 @@ Version poppedVersion(LogRouterData* self, Tag tag) { return tagData->popped; } -ACTOR Future peekLogRouter(LogRouterData* self, - Version begin, - Tag tag, - bool returnIfBlocked = false, - bool reqOnlySpilled = false, - bool streamReply = false, - Optional> sequence = Optional>()) { - state BinaryWriter messages(Unversioned()); - state int sequenceNum = -1; - state UID peekId; - - if (sequence.present()) { - peekId = sequence.get().first; - sequenceNum = sequence.get().second; - if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - self->peekTracker.find(peekId) == self->peekTracker.end()) { - throw operation_obsolete(); - } - auto& trackerData = self->peekTracker[peekId]; - if (sequenceNum == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - trackerData.sequence_version[0].send(std::make_pair(begin, reqOnlySpilled)); - } - auto seqBegin = trackerData.sequence_version.begin(); - // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. - while (trackerData.sequence_version.size() && - seqBegin->first <= sequenceNum - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { - if (seqBegin->second.canBeSet()) { - seqBegin->second.sendError(operation_obsolete()); - } - trackerData.sequence_version.erase(seqBegin); - seqBegin = trackerData.sequence_version.begin(); - } - - if (trackerData.sequence_version.size() && sequenceNum < seqBegin->first) { - throw operation_obsolete(); - } - - trackerData.lastUpdate = now(); - std::pair prevPeekData = wait(trackerData.sequence_version[sequenceNum].getFuture()); - begin = prevPeekData.first; - reqOnlySpilled = prevPeekData.second; - wait(yield()); - } - - //TraceEvent("LogRouterPeek1", self->dbgid).detail("From", req.reply.getEndpoint().getPrimaryAddress()).detail("Ver", self->version.get()).detail("Begin", req.begin); - if (returnIfBlocked && self->version.get() < begin) { - //TraceEvent("LogRouterPeek2", self->dbgid); - if (sequence.present()) { - auto& trackerData = self->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(begin, reqOnlySpilled)); - } - } - throw end_of_stream(); - } - - if (self->version.get() < begin) { - wait(self->version.whenAtLeast(begin)); - wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); - } - - Version poppedVer = poppedVersion(self, tag); - - if (poppedVer > begin || begin < self->startVersion) { - // This should only happen if a packet is sent multiple times and the reply is not needed. - // Since we are using popped differently, do not send a reply. - TraceEvent(SevWarnAlways, "LogRouterPeekPopped", self->dbgid) - .detail("Begin", begin) - .detail("Popped", poppedVer) - .detail("Start", self->startVersion); - if (sequence.present()) { - auto& trackerData = self->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(begin, reqOnlySpilled)); - } - } - if (streamReply) { - // for streaming reply, we skip the popped part - begin = std::min(poppedVer, self->startVersion); - } else { - throw no_action_needed(); // we've already replied in the past - } - } - - Version endVersion = self->version.get() + 1; - peekMessagesFromMemory(self, tag, begin, messages, endVersion); - - TLogPeekReply reply; - reply.maxKnownVersion = self->version.get(); - reply.minKnownCommittedVersion = self->poppedVersion; - reply.messages = StringRef(reply.arena, messages.toValue()); - reply.popped = self->minPopped.get() >= self->startVersion ? self->minPopped.get() : 0; - reply.end = endVersion; - reply.onlySpilled = false; - - if (sequence.present()) { - auto& trackerData = self->peekTracker[peekId]; - trackerData.lastUpdate = now(); - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { - if (!sequenceData.isSet()) - sequenceData.sendError(operation_obsolete()); - throw operation_obsolete(); - } - if (sequenceData.isSet()) { - if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // tlog peek second attempt ended at a different version - throw operation_obsolete(); - } - } else { - sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); - } - reply.begin = begin; - } - - //TraceEvent("LogRouterPeek4", self->dbgid); - return reply; -} - // This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover ACTOR Future logRouterPeekStream(LogRouterData* self, TLogPeekStreamRequest req) { - self->activePeekStreams++; + return Void(); +} - state Version begin = req.begin; - state bool onlySpilled = false; +ACTOR Future logRouterPeekMessages(LogRouterData* self, TLogPeekRequest req) { + state BinaryWriter messages(Unversioned()); + state int sequence = -1; + state UID peekId; - req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); - loop { - state TLogPeekStreamReply reply; + if (req.sequence.present()) { try { - wait(req.reply.onReady() && - store(reply.rep, peekLogRouter(self, req.begin, req.tag, req.returnIfBlocked, onlySpilled, true))); - req.reply.send(reply); - begin = reply.rep.end; - onlySpilled = reply.rep.onlySpilled; - if (reply.rep.end > self->version.get()) { - wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); - } else { - wait(delay(0, g_network->getCurrentTask())); + peekId = req.sequence.get().first; + sequence = req.sequence.get().second; + if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + self->peekTracker.find(peekId) == self->peekTracker.end()) { + throw operation_obsolete(); + } + auto& trackerData = self->peekTracker[peekId]; + if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { + trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled)); + } + auto seqBegin = trackerData.sequence_version.begin(); + // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. + while (trackerData.sequence_version.size() && + seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + if (seqBegin->second.canBeSet()) { + seqBegin->second.sendError(operation_obsolete()); + } + trackerData.sequence_version.erase(seqBegin); + seqBegin = trackerData.sequence_version.begin(); } - } catch (Error& e) { - self->activePeekStreams--; - TraceEvent(SevDebug, "LogRouterPeekStreamEnd", self->dbgid).error(e, true); - if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { + if (trackerData.sequence_version.size() && sequence < seqBegin->first) { + throw operation_obsolete(); + } + + trackerData.lastUpdate = now(); + std::pair prevPeekData = wait(trackerData.sequence_version[sequence].getFuture()); + req.begin = prevPeekData.first; + req.onlySpilled = prevPeekData.second; + wait(yield()); + } catch (Error& e) { + if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { req.reply.sendError(e); return Void(); } else { @@ -595,28 +492,85 @@ ACTOR Future logRouterPeekStream(LogRouterData* self, TLogPeekStreamReques } } } -} -ACTOR Future logRouterPeekMessages(LogRouterData* self, TLogPeekRequest req) { - try { - TLogPeekReply reply = - wait(peekLogRouter(self, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, false, req.sequence)); - req.reply.send(reply); - } catch (Error& e) { - if (e.code() == error_code_no_action_needed) { - req.reply.send(Never()); - return Void(); - } else if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || - e.code() == error_code_end_of_stream) { - req.reply.sendError(e); - return Void(); - } else { - throw; + //TraceEvent("LogRouterPeek1", self->dbgid).detail("From", req.reply.getEndpoint().getPrimaryAddress()).detail("Ver", self->version.get()).detail("Begin", req.begin); + if (req.returnIfBlocked && self->version.get() < req.begin) { + //TraceEvent("LogRouterPeek2", self->dbgid); + req.reply.sendError(end_of_stream()); + if (req.sequence.present()) { + auto& trackerData = self->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequence + 1]; + if (!sequenceData.isSet()) { + sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); + } } + return Void(); } + + if (self->version.get() < req.begin) { + wait(self->version.whenAtLeast(req.begin)); + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } + + Version poppedVer = poppedVersion(self, req.tag); + + if (poppedVer > req.begin || req.begin < self->startVersion) { + // This should only happen if a packet is sent multiple times and the reply is not needed. + // Since we are using popped differently, do not send a reply. + TraceEvent(SevWarnAlways, "LogRouterPeekPopped", self->dbgid) + .detail("Begin", req.begin) + .detail("Popped", poppedVer) + .detail("Start", self->startVersion); + req.reply.send(Never()); + if (req.sequence.present()) { + auto& trackerData = self->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequence + 1]; + if (!sequenceData.isSet()) { + sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); + } + } + return Void(); + } + + Version endVersion = self->version.get() + 1; + peekMessagesFromMemory(self, req.tag, req.begin, messages, endVersion); + + TLogPeekReply reply; + reply.maxKnownVersion = self->version.get(); + reply.minKnownCommittedVersion = self->poppedVersion; + reply.messages = messages.toValue(); + reply.popped = self->minPopped.get() >= self->startVersion ? self->minPopped.get() : 0; + reply.end = endVersion; + reply.onlySpilled = false; + + if (req.sequence.present()) { + auto& trackerData = self->peekTracker[peekId]; + trackerData.lastUpdate = now(); + auto& sequenceData = trackerData.sequence_version[sequence + 1]; + if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { + req.reply.sendError(operation_obsolete()); + if (!sequenceData.isSet()) + sequenceData.sendError(operation_obsolete()); + return Void(); + } + if (sequenceData.isSet()) { + if (sequenceData.getFuture().get().first != reply.end) { + TEST(true); // tlog peek second attempt ended at a different version + req.reply.sendError(operation_obsolete()); + return Void(); + } + } else { + sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); + } + reply.begin = req.begin; + } + + req.reply.send(reply); + //TraceEvent("LogRouterPeek4", self->dbgid); return Void(); } + ACTOR Future cleanupPeekTrackers(LogRouterData* self) { loop { double minTimeUntilExpiration = SERVER_KNOBS->PEEK_TRACKER_EXPIRATION_TIME; diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index b097d88c8f..c210f89f29 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -418,7 +418,7 @@ Future ILogSystem::ServerPeekCursor::getMore(TaskPriority taskID) { return Void(); if (!more.isValid() || more.isReady()) { // TODO: remove locality check when log router support streaming peek - if (usePeekStream && tag.locality >= 0) { + if (false && usePeekStream && tag.locality >= 0) { more = serverPeekStreamGetMore(this, taskID); } else if (parallelGetMore || onlySpilled || futureResults.size()) { more = serverPeekParallelGetMore(this, taskID); diff --git a/fdbserver/OldTLogServer_4_6.actor.cpp b/fdbserver/OldTLogServer_4_6.actor.cpp index 9c4cea2eac..d5fafab6c7 100644 --- a/fdbserver/OldTLogServer_4_6.actor.cpp +++ b/fdbserver/OldTLogServer_4_6.actor.cpp @@ -1157,21 +1157,158 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref } } + ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { - try { - TLogPeekReply reply = - wait(peekTLog(self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); - req.reply.send(reply); - } catch (Error& e) { - if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || - e.code() == error_code_end_of_stream) { - req.reply.sendError(e); - return Void(); - } else { - throw; + state BinaryWriter messages(Unversioned()); + state BinaryWriter messages2(Unversioned()); + state int sequence = -1; + state UID peekId; + state OldTag oldTag = convertTag(req.tag); + + if (req.sequence.present()) { + try { + peekId = req.sequence.get().first; + sequence = req.sequence.get().second; + if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + self->peekTracker.find(peekId) == self->peekTracker.end()) { + throw operation_obsolete(); + } + if (sequence > 0) { + auto& trackerData = self->peekTracker[peekId]; + trackerData.lastUpdate = now(); + Version ver = wait(trackerData.sequence_version[sequence].getFuture()); + req.begin = std::max(ver, req.begin); + wait(yield()); + } + } catch (Error& e) { + if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { + req.reply.sendError(e); + return Void(); + } else { + throw; + } } } + if (req.returnIfBlocked && logData->version.get() < req.begin) { + req.reply.sendError(end_of_stream()); + return Void(); + } + + //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + // Wait until we have something to return that the caller doesn't already have + if (logData->version.get() < req.begin) { + wait(logData->version.whenAtLeast(req.begin)); + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } + + state Version endVersion = logData->version.get() + 1; + + Version poppedVer = poppedVersion(logData, oldTag); + if (poppedVer > req.begin) { + TLogPeekReply rep; + rep.maxKnownVersion = logData->version.get(); + rep.minKnownCommittedVersion = 0; + rep.popped = poppedVer; + rep.end = poppedVer; + rep.onlySpilled = false; + + if (req.sequence.present()) { + auto& trackerData = self->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequence + 1]; + trackerData.lastUpdate = now(); + if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { + req.reply.sendError(operation_obsolete()); + if (!sequenceData.isSet()) + sequenceData.sendError(operation_obsolete()); + return Void(); + } + if (sequenceData.isSet()) { + if (sequenceData.getFuture().get() != rep.end) { + TEST(true); // tlog peek second attempt ended at a different version + req.reply.sendError(operation_obsolete()); + return Void(); + } + } else { + sequenceData.send(rep.end); + } + rep.begin = req.begin; + } + + req.reply.send(rep); + return Void(); + } + + // grab messages from disk + //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + if (req.begin <= logData->persistentDataDurableVersion) { + // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We + // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if + // an initial attempt to read from disk results in insufficient data and the required data is no longer in + // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the + // result? + + peekMessagesFromMemory(logData, req.tag, req.begin, messages2, endVersion); + + RangeResult kvs = wait(self->persistentData->readRange( + KeyRangeRef(persistTagMessagesKey(logData->logId, oldTag, req.begin), + persistTagMessagesKey(logData->logId, oldTag, logData->persistentDataDurableVersion + 1)), + SERVER_KNOBS->DESIRED_TOTAL_BYTES, + SERVER_KNOBS->DESIRED_TOTAL_BYTES)); + + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); + + for (auto& kv : kvs) { + auto ver = decodeTagMessagesKey(kv.key); + messages << int32_t(-1) << ver; + + BinaryReader rd(kv.value, Unversioned()); + while (!rd.empty()) { + int32_t messageLength; + uint32_t subVersion; + rd >> messageLength >> subVersion; + messageLength += sizeof(uint16_t) + sizeof(Tag); + messages << messageLength << subVersion << uint16_t(1) << req.tag; + messageLength -= (sizeof(subVersion) + sizeof(uint16_t) + sizeof(Tag)); + messages.serializeBytes(rd.readBytes(messageLength), messageLength); + } + } + + if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) + endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1; + else + messages.serializeBytes(messages2.toValue()); + } else { + peekMessagesFromMemory(logData, req.tag, req.begin, messages, endVersion); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); + } + + TLogPeekReply reply; + reply.maxKnownVersion = logData->version.get(); + reply.minKnownCommittedVersion = 0; + reply.onlySpilled = false; + reply.messages = messages.toValue(); + reply.end = endVersion; + + //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); + + if (req.sequence.present()) { + auto& trackerData = self->peekTracker[peekId]; + trackerData.lastUpdate = now(); + auto& sequenceData = trackerData.sequence_version[sequence + 1]; + if (sequenceData.isSet()) { + if (sequenceData.getFuture().get() != reply.end) { + TEST(true); // tlog peek second attempt ended at a different version (2) + req.reply.sendError(operation_obsolete()); + return Void(); + } + } else { + sequenceData.send(reply.end); + } + reply.begin = req.begin; + } + + req.reply.send(reply); return Void(); } diff --git a/fdbserver/OldTLogServer_6_0.actor.cpp b/fdbserver/OldTLogServer_6_0.actor.cpp index 04142babf7..b6964bf9c9 100644 --- a/fdbserver/OldTLogServer_6_0.actor.cpp +++ b/fdbserver/OldTLogServer_6_0.actor.cpp @@ -1479,24 +1479,241 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref } } + ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { - try { - TLogPeekReply reply = - wait(peekTLog(self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); - req.reply.send(reply); - } catch (Error& e) { - if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || - e.code() == error_code_end_of_stream) { - req.reply.sendError(e); - return Void(); - } else { - throw; + state BinaryWriter messages(Unversioned()); + state BinaryWriter messages2(Unversioned()); + state int sequence = -1; + state UID peekId; + state double queueStart = now(); + + if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { + req.tag.id = req.tag.id % logData->txsTags; + } + + if (req.sequence.present()) { + try { + peekId = req.sequence.get().first; + sequence = req.sequence.get().second; + if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + logData->peekTracker.find(peekId) == logData->peekTracker.end()) { + throw operation_obsolete(); + } + auto& trackerData = logData->peekTracker[peekId]; + if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { + trackerData.tag = req.tag; + trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled)); + } + auto seqBegin = trackerData.sequence_version.begin(); + while (trackerData.sequence_version.size() && + seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + if (seqBegin->second.canBeSet()) { + seqBegin->second.sendError(operation_obsolete()); + } + trackerData.sequence_version.erase(seqBegin); + seqBegin = trackerData.sequence_version.begin(); + } + + if (trackerData.sequence_version.size() && sequence < seqBegin->first) { + throw operation_obsolete(); + } + + Future> fPrevPeekData = trackerData.sequence_version[sequence].getFuture(); + if (fPrevPeekData.isReady()) { + trackerData.unblockedPeeks++; + double t = now() - trackerData.lastUpdate; + if (t > trackerData.idleMax) + trackerData.idleMax = t; + trackerData.idleTime += t; + } + trackerData.lastUpdate = now(); + std::pair prevPeekData = wait(fPrevPeekData); + + req.begin = std::max(prevPeekData.first, req.begin); + req.onlySpilled = prevPeekData.second; + wait(yield()); + } catch (Error& e) { + if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { + req.reply.sendError(e); + return Void(); + } else { + throw; + } } } + state double blockStart = now(); + + if (req.returnIfBlocked && logData->version.get() < req.begin) { + req.reply.sendError(end_of_stream()); + if (req.sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequence + 1]; + if (!sequenceData.isSet()) { + sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); + } + } + return Void(); + } + + //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + // Wait until we have something to return that the caller doesn't already have + if (logData->version.get() < req.begin) { + wait(logData->version.whenAtLeast(req.begin)); + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } + + if (logData->locality != tagLocalitySatellite && req.tag.locality == tagLocalityLogRouter) { + wait(self->concurrentLogRouterReads.take()); + state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); + wait(delay(0.0, TaskPriority::Low)); + } + + if (req.begin <= logData->persistentDataDurableVersion && req.tag.locality != tagLocalityTxs && req.tag != txsTag) { + // Reading spilled data will almost always imply that the storage server is >5s behind the rest + // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up + // slightly faster over keeping the rest of the cluster operating normally. + // txsTag is only ever peeked on recovery, and we would still wish to prioritize requests + // that impact recovery duration. + wait(delay(0, TaskPriority::TLogSpilledPeekReply)); + } + + state double workStart = now(); + + Version poppedVer = poppedVersion(logData, req.tag); + if (poppedVer > req.begin) { + TLogPeekReply rep; + rep.maxKnownVersion = logData->version.get(); + rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; + rep.popped = poppedVer; + rep.end = poppedVer; + rep.onlySpilled = false; + + if (req.sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequence + 1]; + trackerData.lastUpdate = now(); + if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { + req.reply.sendError(operation_obsolete()); + if (!sequenceData.isSet()) + sequenceData.sendError(operation_obsolete()); + return Void(); + } + if (sequenceData.isSet()) { + if (sequenceData.getFuture().get().first != rep.end) { + TEST(true); // tlog peek second attempt ended at a different version + req.reply.sendError(operation_obsolete()); + return Void(); + } + } else { + sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); + } + rep.begin = req.begin; + } + + req.reply.send(rep); + return Void(); + } + + state Version endVersion = logData->version.get() + 1; + state bool onlySpilled = false; + + // grab messages from disk + //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + if (req.begin <= logData->persistentDataDurableVersion) { + // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We + // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if + // an initial attempt to read from disk results in insufficient data and the required data is no longer in + // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the + // result? + + if (req.onlySpilled) { + endVersion = logData->persistentDataDurableVersion + 1; + } else { + peekMessagesFromMemory(logData, req.tag, req.begin, messages2, endVersion); + } + + RangeResult kvs = wait(self->persistentData->readRange( + KeyRangeRef(persistTagMessagesKey(logData->logId, req.tag, req.begin), + persistTagMessagesKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)), + SERVER_KNOBS->DESIRED_TOTAL_BYTES, + SERVER_KNOBS->DESIRED_TOTAL_BYTES)); + + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().address).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); + + for (auto& kv : kvs) { + auto ver = decodeTagMessagesKey(kv.key); + messages << VERSION_HEADER << ver; + messages.serializeBytes(kv.value); + } + + if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { + endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1; + onlySpilled = true; + } else { + messages.serializeBytes(messages2.toValue()); + } + } else { + peekMessagesFromMemory(logData, req.tag, req.begin, messages, endVersion); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().address).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); + } + + TLogPeekReply reply; + reply.maxKnownVersion = logData->version.get(); + reply.minKnownCommittedVersion = logData->minKnownCommittedVersion; + reply.messages = messages.toValue(); + reply.end = endVersion; + reply.onlySpilled = onlySpilled; + + //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().address); + + if (req.sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + trackerData.lastUpdate = now(); + + double queueT = blockStart - queueStart; + double blockT = workStart - blockStart; + double workT = now() - workStart; + + trackerData.totalPeeks++; + trackerData.replyBytes += reply.messages.size(); + + if (queueT > trackerData.queueMax) + trackerData.queueMax = queueT; + if (blockT > trackerData.blockMax) + trackerData.blockMax = blockT; + if (workT > trackerData.workMax) + trackerData.workMax = workT; + + trackerData.queueTime += queueT; + trackerData.blockTime += blockT; + trackerData.workTime += workT; + + auto& sequenceData = trackerData.sequence_version[sequence + 1]; + if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { + req.reply.sendError(operation_obsolete()); + if (!sequenceData.isSet()) + sequenceData.sendError(operation_obsolete()); + return Void(); + } + if (sequenceData.isSet()) { + trackerData.duplicatePeeks++; + if (sequenceData.getFuture().get().first != reply.end) { + TEST(true); // tlog peek second attempt ended at a different version (2) + req.reply.sendError(operation_obsolete()); + return Void(); + } + } else { + sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); + } + reply.begin = req.begin; + } + + req.reply.send(reply); return Void(); } + ACTOR Future doQueueCommit(TLogData* self, Reference logData, std::vector> missingFinalCommit) { diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index 917f772503..88029f9eb1 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -1547,7 +1547,8 @@ ACTOR Future> parseMessagesForTag(StringRef commitBlob, T } // Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request -ACTOR Future peekTLog(TLogData* self, +ACTOR template +Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, Reference logData, Version begin, Tag tag, @@ -1567,50 +1568,60 @@ ACTOR Future peekTLog(TLogData* self, // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence requests // STEP: a. mark obsolete sequence requests; b. wait previous sequence requests are handled in order if (sequence.present()) { - peekId = sequence.get().first; - sequenceNum = sequence.get().second; - if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - logData->peekTracker.find(peekId) == logData->peekTracker.end()) { - throw operation_obsolete(); - } - auto& trackerData = logData->peekTracker[peekId]; - if (sequenceNum == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - trackerData.tag = tag; - trackerData.sequence_version[0].send(std::make_pair(begin, reqOnlySpilled)); - } - auto seqBegin = trackerData.sequence_version.begin(); - // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. - while (trackerData.sequence_version.size() && - seqBegin->first <= sequenceNum - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { - if (seqBegin->second.canBeSet()) { - seqBegin->second.sendError(operation_obsolete()); + try{ + peekId = sequence.get().first; + sequenceNum = sequence.get().second; + if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + logData->peekTracker.find(peekId) == logData->peekTracker.end()) { + throw operation_obsolete(); + } + auto& trackerData = logData->peekTracker[peekId]; + if (sequenceNum == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { + trackerData.tag = tag; + trackerData.sequence_version[0].send(std::make_pair(begin, reqOnlySpilled)); + } + auto seqBegin = trackerData.sequence_version.begin(); + // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. + while (trackerData.sequence_version.size() && + seqBegin->first <= sequenceNum - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + if (seqBegin->second.canBeSet()) { + seqBegin->second.sendError(operation_obsolete()); + } + trackerData.sequence_version.erase(seqBegin); + seqBegin = trackerData.sequence_version.begin(); + } + + if (trackerData.sequence_version.size() && sequenceNum < seqBegin->first) { + throw operation_obsolete(); + } + + Future> fPrevPeekData = trackerData.sequence_version[sequenceNum].getFuture(); + if (fPrevPeekData.isReady()) { + trackerData.unblockedPeeks++; + double t = now() - trackerData.lastUpdate; + if (t > trackerData.idleMax) + trackerData.idleMax = t; + trackerData.idleTime += t; + } + trackerData.lastUpdate = now(); + std::pair prevPeekData = wait(fPrevPeekData); + begin = std::max(prevPeekData.first, begin); + reqOnlySpilled = prevPeekData.second; + wait(yield()); + } catch (Error& e) { + if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { + replyPromise.sendError(e); + return Void(); + } else { + throw; } - trackerData.sequence_version.erase(seqBegin); - seqBegin = trackerData.sequence_version.begin(); } - - if (trackerData.sequence_version.size() && sequenceNum < seqBegin->first) { - throw operation_obsolete(); - } - - Future> fPrevPeekData = trackerData.sequence_version[sequenceNum].getFuture(); - if (fPrevPeekData.isReady()) { - trackerData.unblockedPeeks++; - double t = now() - trackerData.lastUpdate; - if (t > trackerData.idleMax) - trackerData.idleMax = t; - trackerData.idleTime += t; - } - trackerData.lastUpdate = now(); - std::pair prevPeekData = wait(fPrevPeekData); - begin = std::max(prevPeekData.first, begin); - reqOnlySpilled = prevPeekData.second; - wait(yield()); } state double blockStart = now(); if (returnIfBlocked && logData->version.get() < begin) { + replyPromise.sendError(end_of_stream()); if (sequence.present()) { auto& trackerData = logData->peekTracker[peekId]; auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; @@ -1619,7 +1630,7 @@ ACTOR Future peekTLog(TLogData* self, sequenceData.send(std::make_pair(begin, reqOnlySpilled)); } } - throw end_of_stream(); + return Void(); } //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); @@ -1663,14 +1674,16 @@ ACTOR Future peekTLog(TLogData* self, auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; trackerData.lastUpdate = now(); if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + replyPromise.sendError(operation_obsolete()); if (!sequenceData.isSet()) sequenceData.sendError(operation_obsolete()); - throw operation_obsolete(); + return Void(); } if (sequenceData.isSet()) { if (sequenceData.getFuture().get().first != rep.end) { TEST(true); // 1 tlog peek second attempt ended at a different version - throw operation_obsolete(); + replyPromise.sendError(operation_obsolete()); + return Void(); } } else { sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); @@ -1678,7 +1691,8 @@ ACTOR Future peekTLog(TLogData* self, rep.begin = begin; } - return rep; + replyPromise.send(rep); + return Void(); } state Version endVersion = logData->version.get() + 1; @@ -1849,6 +1863,7 @@ ACTOR Future peekTLog(TLogData* self, auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + replyPromise.sendError(operation_obsolete()); if (!sequenceData.isSet()) { // It would technically be more correct to .send({req.begin, req.onlySpilled}), as the next // request might still be in the window of active requests, but LogSystemPeekCursor will @@ -1856,13 +1871,14 @@ ACTOR Future peekTLog(TLogData* self, // response will probably be a waste of CPU. sequenceData.sendError(operation_obsolete()); } - throw operation_obsolete(); + return Void(); } if (sequenceData.isSet()) { trackerData.duplicatePeeks++; if (sequenceData.getFuture().get().first != reply.end) { TEST(true); // 1 tlog peek second attempt ended at a different version (2) - throw operation_obsolete(); + replyPromise.sendError(operation_obsolete()); + return Void(); } } else { sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); @@ -1870,25 +1886,29 @@ ACTOR Future peekTLog(TLogData* self, reply.begin = begin; } - return reply; + replyPromise.send(reply); + return Void(); } // This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { self->activePeekStreams++; - TraceEvent(SevDebug, "TLogPeekStream", logData->logId).detail("Token", req.reply.getEndpoint().token); state Version begin = req.begin; state bool onlySpilled = false; - if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { - req.tag.id = req.tag.id % logData->txsTags; - } req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); loop { state TLogPeekStreamReply reply; + state Promise promise; + state Future future(promise.getFuture()); try { - wait(req.reply.onReady() && - store(reply.rep, peekTLog(self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled))); + wait(req.reply.onReady() && peekTLogAndSend(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled)); + ASSERT(future.isReady()); + if(future.isError()) { + throw future.getError(); + } + + reply.rep = future.get(); req.reply.send(reply); begin = reply.rep.end; onlySpilled = reply.rep.onlySpilled; @@ -1911,21 +1931,327 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref } } + ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { - try { - TLogPeekReply reply = - wait(peekTLog(self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); - req.reply.send(reply); - } catch (Error& e) { - if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || - e.code() == error_code_end_of_stream) { - req.reply.sendError(e); - return Void(); - } else { - throw; + state BinaryWriter messages(Unversioned()); + state BinaryWriter messages2(Unversioned()); + state int sequence = -1; + state UID peekId; + state double queueStart = now(); + + if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { + req.tag.id = req.tag.id % logData->txsTags; + } + + if (req.sequence.present()) { + try { + peekId = req.sequence.get().first; + sequence = req.sequence.get().second; + if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + logData->peekTracker.find(peekId) == logData->peekTracker.end()) { + throw operation_obsolete(); + } + auto& trackerData = logData->peekTracker[peekId]; + if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { + trackerData.tag = req.tag; + trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled)); + } + auto seqBegin = trackerData.sequence_version.begin(); + // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. + while (trackerData.sequence_version.size() && + seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + if (seqBegin->second.canBeSet()) { + seqBegin->second.sendError(operation_obsolete()); + } + trackerData.sequence_version.erase(seqBegin); + seqBegin = trackerData.sequence_version.begin(); + } + + if (trackerData.sequence_version.size() && sequence < seqBegin->first) { + throw operation_obsolete(); + } + + Future> fPrevPeekData = trackerData.sequence_version[sequence].getFuture(); + if (fPrevPeekData.isReady()) { + trackerData.unblockedPeeks++; + double t = now() - trackerData.lastUpdate; + if (t > trackerData.idleMax) + trackerData.idleMax = t; + trackerData.idleTime += t; + } + trackerData.lastUpdate = now(); + std::pair prevPeekData = wait(fPrevPeekData); + req.begin = std::max(prevPeekData.first, req.begin); + req.onlySpilled = prevPeekData.second; + wait(yield()); + } catch (Error& e) { + if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { + req.reply.sendError(e); + return Void(); + } else { + throw; + } } } + state double blockStart = now(); + + if (req.returnIfBlocked && logData->version.get() < req.begin) { + req.reply.sendError(end_of_stream()); + if (req.sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequence + 1]; + if (!sequenceData.isSet()) { + sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); + } + } + return Void(); + } + + //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + // Wait until we have something to return that the caller doesn't already have + if (logData->version.get() < req.begin) { + wait(logData->version.whenAtLeast(req.begin)); + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } + + if (req.tag.locality == tagLocalityLogRouter) { + wait(self->concurrentLogRouterReads.take()); + state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); + wait(delay(0.0, TaskPriority::Low)); + } + + if (req.begin <= logData->persistentDataDurableVersion && req.tag.locality != tagLocalityTxs && req.tag != txsTag) { + // Reading spilled data will almost always imply that the storage server is >5s behind the rest + // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up + // slightly faster over keeping the rest of the cluster operating normally. + // txsTag is only ever peeked on recovery, and we would still wish to prioritize requests + // that impact recovery duration. + wait(delay(0, TaskPriority::TLogSpilledPeekReply)); + } + + state double workStart = now(); + + Version poppedVer = poppedVersion(logData, req.tag); + if (poppedVer > req.begin) { + TLogPeekReply rep; + rep.maxKnownVersion = logData->version.get(); + rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; + rep.popped = poppedVer; + rep.end = poppedVer; + rep.onlySpilled = false; + + if (req.sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequence + 1]; + trackerData.lastUpdate = now(); + if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { + req.reply.sendError(operation_obsolete()); + if (!sequenceData.isSet()) + sequenceData.sendError(operation_obsolete()); + return Void(); + } + if (sequenceData.isSet()) { + if (sequenceData.getFuture().get().first != rep.end) { + TEST(true); // tlog peek second attempt ended at a different version + req.reply.sendError(operation_obsolete()); + return Void(); + } + } else { + sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); + } + rep.begin = req.begin; + } + + req.reply.send(rep); + return Void(); + } + + state Version endVersion = logData->version.get() + 1; + state bool onlySpilled = false; + + // grab messages from disk + //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + if (req.begin <= logData->persistentDataDurableVersion) { + // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We + // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if + // an initial attempt to read from disk results in insufficient data and the required data is no longer in + // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the + // result? + + if (req.onlySpilled) { + endVersion = logData->persistentDataDurableVersion + 1; + } else { + peekMessagesFromMemory(logData, req.tag, req.begin, messages2, endVersion); + } + + if (req.tag.locality == tagLocalityTxs || req.tag == txsTag) { + RangeResult kvs = wait(self->persistentData->readRange( + KeyRangeRef(persistTagMessagesKey(logData->logId, req.tag, req.begin), + persistTagMessagesKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)), + SERVER_KNOBS->DESIRED_TOTAL_BYTES, + SERVER_KNOBS->DESIRED_TOTAL_BYTES)); + + for (auto& kv : kvs) { + auto ver = decodeTagMessagesKey(kv.key); + messages << VERSION_HEADER << ver; + messages.serializeBytes(kv.value); + } + + if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { + endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1; + onlySpilled = true; + } else { + messages.serializeBytes(messages2.toValue()); + } + } else { + // FIXME: Limit to approximately DESIRED_TOTATL_BYTES somehow. + RangeResult kvrefs = wait(self->persistentData->readRange( + KeyRangeRef( + persistTagMessageRefsKey(logData->logId, req.tag, req.begin), + persistTagMessageRefsKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)), + SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1)); + + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); + + state std::vector> commitLocations; + state bool earlyEnd = false; + uint32_t mutationBytes = 0; + state uint64_t commitBytes = 0; + state Version firstVersion = std::numeric_limits::max(); + for (int i = 0; i < kvrefs.size() && i < SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK; i++) { + auto& kv = kvrefs[i]; + VectorRef spilledData; + BinaryReader r(kv.value, AssumeVersion(logData->protocolVersion)); + r >> spilledData; + for (const SpilledData& sd : spilledData) { + if (mutationBytes >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { + earlyEnd = true; + break; + } + if (sd.version >= req.begin) { + firstVersion = std::min(firstVersion, sd.version); + const IDiskQueue::location end = sd.start.lo + sd.length; + commitLocations.emplace_back(sd.start, end); + // This isn't perfect, because we aren't accounting for page boundaries, but should be + // close enough. + commitBytes += sd.length; + mutationBytes += sd.mutationBytes; + } + } + if (earlyEnd) + break; + } + earlyEnd = earlyEnd || (kvrefs.size() >= SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1); + wait(self->peekMemoryLimiter.take(TaskPriority::TLogSpilledPeekReply, commitBytes)); + state FlowLock::Releaser memoryReservation(self->peekMemoryLimiter, commitBytes); + state std::vector>> messageReads; + messageReads.reserve(commitLocations.size()); + for (const auto& pair : commitLocations) { + messageReads.push_back(self->rawPersistentQueue->read(pair.first, pair.second, CheckHashes::True)); + } + commitLocations.clear(); + wait(waitForAll(messageReads)); + + state Version lastRefMessageVersion = 0; + state int index = 0; + loop { + if (index >= messageReads.size()) + break; + Standalone queueEntryData = messageReads[index].get(); + uint8_t valid; + const uint32_t length = *(uint32_t*)queueEntryData.begin(); + queueEntryData = queueEntryData.substr(4, queueEntryData.size() - 4); + BinaryReader rd(queueEntryData, IncludeVersion()); + state TLogQueueEntry entry; + rd >> entry >> valid; + ASSERT(valid == 0x01); + ASSERT(length + sizeof(valid) == queueEntryData.size()); + + messages << VERSION_HEADER << entry.version; + + std::vector rawMessages = + wait(parseMessagesForTag(entry.messages, req.tag, logData->logRouterTags)); + for (const StringRef& msg : rawMessages) { + messages.serializeBytes(msg); + } + + lastRefMessageVersion = entry.version; + index++; + } + + messageReads.clear(); + memoryReservation.release(); + + if (earlyEnd) { + endVersion = lastRefMessageVersion + 1; + onlySpilled = true; + } else { + messages.serializeBytes(messages2.toValue()); + } + } + } else { + if (req.onlySpilled) { + endVersion = logData->persistentDataDurableVersion + 1; + } else { + peekMessagesFromMemory(logData, req.tag, req.begin, messages, endVersion); + } + + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); + } + + TLogPeekReply reply; + reply.maxKnownVersion = logData->version.get(); + reply.minKnownCommittedVersion = logData->minKnownCommittedVersion; + reply.messages = messages.toValue(); + reply.end = endVersion; + reply.onlySpilled = onlySpilled; + + //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); + + if (req.sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + trackerData.lastUpdate = now(); + + double queueT = blockStart - queueStart; + double blockT = workStart - blockStart; + double workT = now() - workStart; + + trackerData.totalPeeks++; + trackerData.replyBytes += reply.messages.size(); + + if (queueT > trackerData.queueMax) + trackerData.queueMax = queueT; + if (blockT > trackerData.blockMax) + trackerData.blockMax = blockT; + if (workT > trackerData.workMax) + trackerData.workMax = workT; + + trackerData.queueTime += queueT; + trackerData.blockTime += blockT; + trackerData.workTime += workT; + + auto& sequenceData = trackerData.sequence_version[sequence + 1]; + if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { + req.reply.sendError(operation_obsolete()); + if (!sequenceData.isSet()) + sequenceData.sendError(operation_obsolete()); + return Void(); + } + if (sequenceData.isSet()) { + trackerData.duplicatePeeks++; + if (sequenceData.getFuture().get().first != reply.end) { + TEST(true); // tlog peek second attempt ended at a different version (2) + req.reply.sendError(operation_obsolete()); + return Void(); + } + } else { + sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); + } + reply.begin = req.begin; + } + + req.reply.send(reply); return Void(); } diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index f172a496da..3ca0926135 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1587,7 +1587,8 @@ ACTOR Future> parseMessagesForTag(StringRef commitBlob, T } // Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request -ACTOR Future peekTLog(TLogData* self, +ACTOR template +Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, Reference logData, Version begin, Tag tag, @@ -1607,50 +1608,60 @@ ACTOR Future peekTLog(TLogData* self, // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence requests // STEP: a. mark obsolete sequence requests; b. wait previous sequence requests are handled in order if (sequence.present()) { - peekId = sequence.get().first; - sequenceNum = sequence.get().second; - if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - logData->peekTracker.find(peekId) == logData->peekTracker.end()) { - throw operation_obsolete(); - } - auto& trackerData = logData->peekTracker[peekId]; - if (sequenceNum == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - trackerData.tag = tag; - trackerData.sequence_version[0].send(std::make_pair(begin, reqOnlySpilled)); - } - auto seqBegin = trackerData.sequence_version.begin(); - // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. - while (trackerData.sequence_version.size() && - seqBegin->first <= sequenceNum - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { - if (seqBegin->second.canBeSet()) { - seqBegin->second.sendError(operation_obsolete()); + try { + peekId = sequence.get().first; + sequenceNum = sequence.get().second; + if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + logData->peekTracker.find(peekId) == logData->peekTracker.end()) { + throw operation_obsolete(); + } + auto& trackerData = logData->peekTracker[peekId]; + if (sequenceNum == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { + trackerData.tag = tag; + trackerData.sequence_version[0].send(std::make_pair(begin, reqOnlySpilled)); + } + auto seqBegin = trackerData.sequence_version.begin(); + // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. + while (trackerData.sequence_version.size() && + seqBegin->first <= sequenceNum - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + if (seqBegin->second.canBeSet()) { + seqBegin->second.sendError(operation_obsolete()); + } + trackerData.sequence_version.erase(seqBegin); + seqBegin = trackerData.sequence_version.begin(); + } + + if (trackerData.sequence_version.size() && sequenceNum < seqBegin->first) { + throw operation_obsolete(); + } + + Future> fPrevPeekData = trackerData.sequence_version[sequenceNum].getFuture(); + if (fPrevPeekData.isReady()) { + trackerData.unblockedPeeks++; + double t = now() - trackerData.lastUpdate; + if (t > trackerData.idleMax) + trackerData.idleMax = t; + trackerData.idleTime += t; + } + trackerData.lastUpdate = now(); + std::pair prevPeekData = wait(fPrevPeekData); + begin = std::max(prevPeekData.first, begin); + reqOnlySpilled = prevPeekData.second; + wait(yield()); + } catch (Error& e) { + if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { + replyPromise.sendError(e); + return Void(); + } else { + throw; } - trackerData.sequence_version.erase(seqBegin); - seqBegin = trackerData.sequence_version.begin(); } - - if (trackerData.sequence_version.size() && sequenceNum < seqBegin->first) { - throw operation_obsolete(); - } - - Future> fPrevPeekData = trackerData.sequence_version[sequenceNum].getFuture(); - if (fPrevPeekData.isReady()) { - trackerData.unblockedPeeks++; - double t = now() - trackerData.lastUpdate; - if (t > trackerData.idleMax) - trackerData.idleMax = t; - trackerData.idleTime += t; - } - trackerData.lastUpdate = now(); - std::pair prevPeekData = wait(fPrevPeekData); - begin = std::max(prevPeekData.first, begin); - reqOnlySpilled = prevPeekData.second; - wait(yield()); } state double blockStart = now(); if (returnIfBlocked && logData->version.get() < begin) { + replyPromise.sendError(end_of_stream()); if (sequence.present()) { auto& trackerData = logData->peekTracker[peekId]; auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; @@ -1659,7 +1670,7 @@ ACTOR Future peekTLog(TLogData* self, sequenceData.send(std::make_pair(begin, reqOnlySpilled)); } } - throw end_of_stream(); + return Void(); } //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); @@ -1703,14 +1714,16 @@ ACTOR Future peekTLog(TLogData* self, auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; trackerData.lastUpdate = now(); if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + replyPromise.sendError(operation_obsolete()); if (!sequenceData.isSet()) sequenceData.sendError(operation_obsolete()); - throw operation_obsolete(); + return Void(); } if (sequenceData.isSet()) { if (sequenceData.getFuture().get().first != rep.end) { - TEST(true); // 1 tlog peek second attempt ended at a different version - throw operation_obsolete(); + TEST(true); // xz tlog peek second attempt ended at a different version + replyPromise.sendError(operation_obsolete()); + return Void(); } } else { sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); @@ -1718,7 +1731,8 @@ ACTOR Future peekTLog(TLogData* self, rep.begin = begin; } - return rep; + replyPromise.send(rep); + return Void(); } state Version endVersion = logData->version.get() + 1; @@ -1893,6 +1907,7 @@ ACTOR Future peekTLog(TLogData* self, auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + replyPromise.sendError(operation_obsolete()); if (!sequenceData.isSet()) { // It would technically be more correct to .send({req.begin, req.onlySpilled}), as the next // request might still be in the window of active requests, but LogSystemPeekCursor will @@ -1900,13 +1915,14 @@ ACTOR Future peekTLog(TLogData* self, // response will probably be a waste of CPU. sequenceData.sendError(operation_obsolete()); } - throw operation_obsolete(); + return Void(); } if (sequenceData.isSet()) { trackerData.duplicatePeeks++; if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // 1 tlog peek second attempt ended at a different version (2) - throw operation_obsolete(); + TEST(true); // xz tlog peek second attempt ended at a different version (2) + replyPromise.sendError(operation_obsolete()); + return Void(); } } else { sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); @@ -1914,24 +1930,30 @@ ACTOR Future peekTLog(TLogData* self, reply.begin = begin; } - return reply; + replyPromise.send(reply); + return Void(); } + // This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { self->activePeekStreams++; state Version begin = req.begin; state bool onlySpilled = false; - if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { - req.tag.id = req.tag.id % logData->txsTags; - } req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); loop { state TLogPeekStreamReply reply; + state Promise promise; + state Future future(promise.getFuture()); try { - wait(req.reply.onReady() && - store(reply.rep, peekTLog(self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled))); + wait(req.reply.onReady() && peekTLogAndSend(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled)); + ASSERT(future.isReady()); + if(future.isError()) { + throw future.getError(); + } + + reply.rep = future.get(); req.reply.send(reply); begin = reply.rep.end; onlySpilled = reply.rep.onlySpilled; @@ -1955,23 +1977,342 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref } ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { - try { - TLogPeekReply reply = - wait(peekTLog(self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); - req.reply.send(reply); - } catch (Error& e) { - if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete || - e.code() == error_code_end_of_stream) { - req.reply.sendError(e); - return Void(); - } else { - throw; + state BinaryWriter messages(Unversioned()); + state BinaryWriter messages2(Unversioned()); + state int sequence = -1; + state UID peekId; + state double queueStart = now(); + + if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { + req.tag.id = req.tag.id % logData->txsTags; + } + + if (req.sequence.present()) { + try { + peekId = req.sequence.get().first; + sequence = req.sequence.get().second; + if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + logData->peekTracker.find(peekId) == logData->peekTracker.end()) { + throw operation_obsolete(); + } + auto& trackerData = logData->peekTracker[peekId]; + if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { + trackerData.tag = req.tag; + trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled)); + } + auto seqBegin = trackerData.sequence_version.begin(); + // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. + while (trackerData.sequence_version.size() && + seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + if (seqBegin->second.canBeSet()) { + seqBegin->second.sendError(operation_obsolete()); + } + trackerData.sequence_version.erase(seqBegin); + seqBegin = trackerData.sequence_version.begin(); + } + + if (trackerData.sequence_version.size() && sequence < seqBegin->first) { + throw operation_obsolete(); + } + + Future> fPrevPeekData = trackerData.sequence_version[sequence].getFuture(); + if (fPrevPeekData.isReady()) { + trackerData.unblockedPeeks++; + double t = now() - trackerData.lastUpdate; + if (t > trackerData.idleMax) + trackerData.idleMax = t; + trackerData.idleTime += t; + } + trackerData.lastUpdate = now(); + std::pair prevPeekData = wait(fPrevPeekData); + req.begin = std::max(prevPeekData.first, req.begin); + req.onlySpilled = prevPeekData.second; + wait(yield()); + } catch (Error& e) { + if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { + req.reply.sendError(e); + return Void(); + } else { + throw; + } } } + state double blockStart = now(); + + if (req.returnIfBlocked && logData->version.get() < req.begin) { + req.reply.sendError(end_of_stream()); + if (req.sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequence + 1]; + trackerData.lastUpdate = now(); + if (!sequenceData.isSet()) { + sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); + } + } + return Void(); + } + + //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + // Wait until we have something to return that the caller doesn't already have + if (logData->version.get() < req.begin) { + wait(logData->version.whenAtLeast(req.begin)); + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } + + if (logData->locality != tagLocalitySatellite && req.tag.locality == tagLocalityLogRouter) { + wait(self->concurrentLogRouterReads.take()); + state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); + wait(delay(0.0, TaskPriority::Low)); + } + + if (req.begin <= logData->persistentDataDurableVersion && req.tag.locality != tagLocalityTxs && req.tag != txsTag) { + // Reading spilled data will almost always imply that the storage server is >5s behind the rest + // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up + // slightly faster over keeping the rest of the cluster operating normally. + // txsTag is only ever peeked on recovery, and we would still wish to prioritize requests + // that impact recovery duration. + wait(delay(0, TaskPriority::TLogSpilledPeekReply)); + } + + state double workStart = now(); + + Version poppedVer = poppedVersion(logData, req.tag); + if (poppedVer > req.begin) { + TLogPeekReply rep; + rep.maxKnownVersion = logData->version.get(); + rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; + rep.popped = poppedVer; + rep.end = poppedVer; + rep.onlySpilled = false; + + if (req.sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + auto& sequenceData = trackerData.sequence_version[sequence + 1]; + trackerData.lastUpdate = now(); + if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { + req.reply.sendError(operation_obsolete()); + if (!sequenceData.isSet()) + sequenceData.sendError(operation_obsolete()); + return Void(); + } + if (sequenceData.isSet()) { + if (sequenceData.getFuture().get().first != rep.end) { + TEST(true); // tlog peek second attempt ended at a different version + req.reply.sendError(operation_obsolete()); + return Void(); + } + } else { + sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); + } + rep.begin = req.begin; + } + + req.reply.send(rep); + return Void(); + } + + state Version endVersion = logData->version.get() + 1; + state bool onlySpilled = false; + + // grab messages from disk + //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + if (req.begin <= logData->persistentDataDurableVersion) { + // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We + // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if + // an initial attempt to read from disk results in insufficient data and the required data is no longer in + // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the + // result? + + if (req.onlySpilled) { + endVersion = logData->persistentDataDurableVersion + 1; + } else { + peekMessagesFromMemory(logData, req.tag, req.begin, messages2, endVersion); + } + + if (logData->shouldSpillByValue(req.tag)) { + RangeResult kvs = wait(self->persistentData->readRange( + KeyRangeRef(persistTagMessagesKey(logData->logId, req.tag, req.begin), + persistTagMessagesKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)), + SERVER_KNOBS->DESIRED_TOTAL_BYTES, + SERVER_KNOBS->DESIRED_TOTAL_BYTES)); + + for (auto& kv : kvs) { + auto ver = decodeTagMessagesKey(kv.key); + messages << VERSION_HEADER << ver; + messages.serializeBytes(kv.value); + } + + if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { + endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1; + onlySpilled = true; + } else { + messages.serializeBytes(messages2.toValue()); + } + } else { + // FIXME: Limit to approximately DESIRED_TOTATL_BYTES somehow. + RangeResult kvrefs = wait(self->persistentData->readRange( + KeyRangeRef( + persistTagMessageRefsKey(logData->logId, req.tag, req.begin), + persistTagMessageRefsKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)), + SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1)); + + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); + + state std::vector> commitLocations; + state bool earlyEnd = false; + uint32_t mutationBytes = 0; + state uint64_t commitBytes = 0; + state Version firstVersion = std::numeric_limits::max(); + for (int i = 0; i < kvrefs.size() && i < SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK; i++) { + auto& kv = kvrefs[i]; + VectorRef spilledData; + BinaryReader r(kv.value, AssumeVersion(logData->protocolVersion)); + r >> spilledData; + for (const SpilledData& sd : spilledData) { + if (mutationBytes >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { + earlyEnd = true; + break; + } + if (sd.version >= req.begin) { + firstVersion = std::min(firstVersion, sd.version); + const IDiskQueue::location end = sd.start.lo + sd.length; + commitLocations.emplace_back(sd.start, end); + // This isn't perfect, because we aren't accounting for page boundaries, but should be + // close enough. + commitBytes += sd.length; + mutationBytes += sd.mutationBytes; + } + } + if (earlyEnd) + break; + } + earlyEnd = earlyEnd || (kvrefs.size() >= SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1); + wait(self->peekMemoryLimiter.take(TaskPriority::TLogSpilledPeekReply, commitBytes)); + state FlowLock::Releaser memoryReservation(self->peekMemoryLimiter, commitBytes); + state std::vector>> messageReads; + messageReads.reserve(commitLocations.size()); + for (const auto& pair : commitLocations) { + messageReads.push_back(self->rawPersistentQueue->read(pair.first, pair.second, CheckHashes::True)); + } + commitLocations.clear(); + wait(waitForAll(messageReads)); + + state Version lastRefMessageVersion = 0; + state int index = 0; + loop { + if (index >= messageReads.size()) + break; + Standalone queueEntryData = messageReads[index].get(); + uint8_t valid; + const uint32_t length = *(uint32_t*)queueEntryData.begin(); + queueEntryData = queueEntryData.substr(4, queueEntryData.size() - 4); + BinaryReader rd(queueEntryData, IncludeVersion()); + state TLogQueueEntry entry; + rd >> entry >> valid; + ASSERT(valid == 0x01); + ASSERT(length + sizeof(valid) == queueEntryData.size()); + + messages << VERSION_HEADER << entry.version; + + std::vector rawMessages = + wait(parseMessagesForTag(entry.messages, req.tag, logData->logRouterTags)); + for (const StringRef& msg : rawMessages) { + messages.serializeBytes(msg); + DEBUG_TAGS_AND_MESSAGE("TLogPeekFromDisk", entry.version, msg) + .detail("UID", self->dbgid) + .detail("LogId", logData->logId) + .detail("PeekTag", req.tag); + } + + lastRefMessageVersion = entry.version; + index++; + } + + messageReads.clear(); + memoryReservation.release(); + + if (earlyEnd) { + endVersion = lastRefMessageVersion + 1; + onlySpilled = true; + } else { + messages.serializeBytes(messages2.toValue()); + } + } + } else { + if (req.onlySpilled) { + endVersion = logData->persistentDataDurableVersion + 1; + } else { + peekMessagesFromMemory(logData, req.tag, req.begin, messages, endVersion); + } + + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); + } + + TLogPeekReply reply; + reply.maxKnownVersion = logData->version.get(); + reply.minKnownCommittedVersion = logData->minKnownCommittedVersion; + reply.messages = messages.toValue(); + reply.end = endVersion; + reply.onlySpilled = onlySpilled; + + //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("Tag", req.tag.toString()). + // detail("BeginVer", req.begin).detail("EndVer", reply.end). + // detail("MsgBytes", reply.messages.expectedSize()). + // detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); + + if (req.sequence.present()) { + auto& trackerData = logData->peekTracker[peekId]; + trackerData.lastUpdate = now(); + + double queueT = blockStart - queueStart; + double blockT = workStart - blockStart; + double workT = now() - workStart; + + trackerData.totalPeeks++; + trackerData.replyBytes += reply.messages.size(); + + if (queueT > trackerData.queueMax) + trackerData.queueMax = queueT; + if (blockT > trackerData.blockMax) + trackerData.blockMax = blockT; + if (workT > trackerData.workMax) + trackerData.workMax = workT; + + trackerData.queueTime += queueT; + trackerData.blockTime += blockT; + trackerData.workTime += workT; + + auto& sequenceData = trackerData.sequence_version[sequence + 1]; + if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { + req.reply.sendError(operation_obsolete()); + if (!sequenceData.isSet()) { + // It would technically be more correct to .send({req.begin, req.onlySpilled}), as the next + // request might still be in the window of active requests, but LogSystemPeekCursor will + // throw away all future responses upon getting an operation_obsolete(), so computing a + // response will probably be a waste of CPU. + sequenceData.sendError(operation_obsolete()); + } + return Void(); + } + if (sequenceData.isSet()) { + trackerData.duplicatePeeks++; + if (sequenceData.getFuture().get().first != reply.end) { + TEST(true); // tlog peek second attempt ended at a different version (2) + req.reply.sendError(operation_obsolete()); + return Void(); + } + } else { + sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); + } + reply.begin = req.begin; + } + + req.reply.send(reply); return Void(); } + ACTOR Future doQueueCommit(TLogData* self, Reference logData, std::vector> missingFinalCommit) { From 517ff9801d744a3775c8709eefb699a3dcf5fe7b Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Fri, 30 Jul 2021 19:10:13 -0700 Subject: [PATCH 20/63] add information print --- fdbserver/tester.actor.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/fdbserver/tester.actor.cpp b/fdbserver/tester.actor.cpp index e819267bb1..e3ecd480e7 100644 --- a/fdbserver/tester.actor.cpp +++ b/fdbserver/tester.actor.cpp @@ -1481,7 +1481,9 @@ ACTOR Future runTests(Reference Date: Sat, 31 Jul 2021 09:07:53 -0700 Subject: [PATCH 21/63] clean 100k simulation test. revert changes of fdbrpc.h --- fdbrpc/fdbrpc.h | 16 +- fdbserver/MoveKeys.actor.cpp | 3 +- fdbserver/OldTLogServer_4_6.actor.cpp | 296 ++++---------- fdbserver/OldTLogServer_6_0.actor.cpp | 410 +++++--------------- fdbserver/OldTLogServer_6_2.actor.cpp | 522 +++++-------------------- fdbserver/TLogServer.actor.cpp | 533 +++++--------------------- flow/Knobs.cpp | 2 +- 7 files changed, 376 insertions(+), 1406 deletions(-) diff --git a/fdbrpc/fdbrpc.h b/fdbrpc/fdbrpc.h index c0f39aa0b4..60b4c0168e 100644 --- a/fdbrpc/fdbrpc.h +++ b/fdbrpc/fdbrpc.h @@ -328,16 +328,14 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, NetNotifiedQueueWithAcknowledgements(int futures, int promises, const Endpoint& remoteEndpoint) : NotifiedQueue(futures, promises), FlowReceiver(remoteEndpoint, true) { // A ReplyPromiseStream will be terminated on the server side if the network connection with the client breaks - acknowledgements.failures = - tagError(makeDependent(IFailureMonitor::failureMonitor()).onDisconnectOrFailure(remoteEndpoint), - operation_obsolete()); + acknowledgements.failures = tagError( + makeDependent(IFailureMonitor::failureMonitor()).onDisconnect(remoteEndpoint.getPrimaryAddress()), + operation_obsolete()); } void destroy() override { delete this; } void receive(ArenaObjectReader& reader) override { this->addPromiseRef(); - // TraceEvent(SevDebug, "NetNotifiedQueueWithAcknowledgementsReceive") - // .detail("PromiseRef", this->getPromiseReferenceCount()); ErrorOr> message; reader.deserialize(message); @@ -371,8 +369,6 @@ struct NetNotifiedQueueWithAcknowledgements final : NotifiedQueue, this->send(std::move(message.get().asUnderlyingType())); } this->delPromiseRef(); - // TraceEvent(SevDebug, "NetNotifiedQueueWithAcknowledgementsReceiveEnd") - // .detail("PromiseRef", this->getPromiseReferenceCount()); } T pop() override { @@ -698,18 +694,20 @@ public: template ReplyPromiseStream getReplyStream(const X& value) const { - auto p = getReplyPromiseStream(value); if (queue->isRemoteEndpoint()) { Future disc = makeDependent(IFailureMonitor::failureMonitor()).onDisconnectOrFailure(getEndpoint()); + auto& p = getReplyPromiseStream(value); Reference peer = FlowTransport::transport().sendUnreliable(SerializeSource(value), getEndpoint(), true); // FIXME: defer sending the message until we know the connection is established endStreamOnDisconnect(disc, p, getEndpoint(), peer); + return p; } else { send(value); + auto& p = getReplyPromiseStream(value); + return p; } - return p; } // stream.getReplyUnlessFailedFor( request, double sustainedFailureDuration, double sustainedFailureSlope ) diff --git a/fdbserver/MoveKeys.actor.cpp b/fdbserver/MoveKeys.actor.cpp index 55706e458f..dad17a1a67 100644 --- a/fdbserver/MoveKeys.actor.cpp +++ b/fdbserver/MoveKeys.actor.cpp @@ -815,8 +815,7 @@ ACTOR static Future finishMoveKeys(Database occ, // Wait for a durable quorum of servers in destServers to have keys available (readWrite) // They must also have at least the transaction read version so they can't "forget" the shard - // between - // now and when this transaction commits. + // between now and when this transaction commits. state vector> serverReady; // only for count below state vector> tssReady; // for waiting in parallel with tss state vector tssReadyInterfs; diff --git a/fdbserver/OldTLogServer_4_6.actor.cpp b/fdbserver/OldTLogServer_4_6.actor.cpp index d5fafab6c7..35d142b9f7 100644 --- a/fdbserver/OldTLogServer_4_6.actor.cpp +++ b/fdbserver/OldTLogServer_4_6.actor.cpp @@ -975,51 +975,62 @@ void peekMessagesFromMemory(Reference self, } // Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request -ACTOR Future peekTLog(TLogData* self, - Reference logData, - Version begin, - Tag tag, - bool returnIfBlocked = false, - bool reqOnlySpilled = false, - Optional> sequence = Optional>()) { +ACTOR template +Future tLogPeekMessages(PromiseType replyPromise, + TLogData* self, + Reference logData, + Version reqBegin, + Tag reqTag, + bool reqReturnIfBlocked = false, + bool reqOnlySpilled = false, + Optional> reqSequence = Optional>()) { state BinaryWriter messages(Unversioned()); state BinaryWriter messages2(Unversioned()); - state int sequenceNum = -1; + state int sequence = -1; state UID peekId; - state double queueStart = now(); - state OldTag oldTag = convertTag(tag); + state OldTag oldTag = convertTag(reqTag); - if (sequence.present()) { - peekId = sequence.get().first; - sequenceNum = sequence.get().second; - if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - self->peekTracker.find(peekId) == self->peekTracker.end()) { - throw operation_obsolete(); - } - if (sequenceNum > 0) { - auto& trackerData = self->peekTracker[peekId]; - trackerData.lastUpdate = now(); - Version ver = wait(trackerData.sequence_version[sequenceNum].getFuture()); - begin = std::max(ver, begin); - wait(yield()); + if (reqSequence.present()) { + try { + peekId = reqSequence.get().first; + sequence = reqSequence.get().second; + if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + self->peekTracker.find(peekId) == self->peekTracker.end()) { + throw operation_obsolete(); + } + if (sequence > 0) { + auto& trackerData = self->peekTracker[peekId]; + trackerData.lastUpdate = now(); + Version ver = wait(trackerData.sequence_version[sequence].getFuture()); + reqBegin = std::max(ver, reqBegin); + wait(yield()); + } + } catch (Error& e) { + if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { + replyPromise.sendError(e); + return Void(); + } else { + throw; + } } } - if (returnIfBlocked && logData->version.get() < begin) { - throw end_of_stream(); + if (reqReturnIfBlocked && logData->version.get() < reqBegin) { + replyPromise.sendError(end_of_stream()); + return Void(); } - //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", reqBegin.epoch).detail("ReqBeginSeq", reqBegin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", reqTag1).detail("Tag2", reqTag2); // Wait until we have something to return that the caller doesn't already have - if (logData->version.get() < begin) { - wait(logData->version.whenAtLeast(begin)); + if (logData->version.get() < reqBegin) { + wait(logData->version.whenAtLeast(reqBegin)); wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); } state Version endVersion = logData->version.get() + 1; Version poppedVer = poppedVersion(logData, oldTag); - if (poppedVer > begin) { + if (poppedVer > reqBegin) { TLogPeekReply rep; rep.maxKnownVersion = logData->version.get(); rep.minKnownCommittedVersion = 0; @@ -1027,47 +1038,50 @@ ACTOR Future peekTLog(TLogData* self, rep.end = poppedVer; rep.onlySpilled = false; - if (sequence.present()) { + if (reqSequence.present()) { auto& trackerData = self->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + auto& sequenceData = trackerData.sequence_version[sequence + 1]; trackerData.lastUpdate = now(); - if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { + replyPromise.sendError(operation_obsolete()); if (!sequenceData.isSet()) sequenceData.sendError(operation_obsolete()); - throw operation_obsolete(); + return Void(); } if (sequenceData.isSet()) { if (sequenceData.getFuture().get() != rep.end) { - TEST(true); // 0 tlog peek second attempt ended at a different version - throw operation_obsolete(); + TEST(true); // tlog peek second attempt ended at a different version + replyPromise.sendError(operation_obsolete()); + return Void(); } } else { sequenceData.send(rep.end); } - rep.begin = begin; + rep.begin = reqBegin; } - return rep; + replyPromise.send(rep); + return Void(); } // grab messages from disk - //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - if (begin <= logData->persistentDataDurableVersion) { + //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", reqBegin.epoch).detail("ReqBeginSeq", reqBegin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", reqTag1).detail("Tag2", reqTag2); + if (reqBegin <= logData->persistentDataDurableVersion) { // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if // an initial attempt to read from disk results in insufficient data and the required data is no longer in // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the // result? - peekMessagesFromMemory(logData, tag, begin, messages2, endVersion); + peekMessagesFromMemory(logData, reqTag, reqBegin, messages2, endVersion); RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, oldTag, begin), - persistTagMessagesKey(logData->logId, oldTag, logData->persistentDataDurableVersion + 1)), - SERVER_KNOBS->DESIRED_TOTAL_BYTES, - SERVER_KNOBS->DESIRED_TOTAL_BYTES)); + KeyRangeRef(persistTagMessagesKey(logData->logId, oldTag, reqBegin), + persistTagMessagesKey(logData->logId, oldTag, logData->persistentDataDurableVersion + 1)), + SERVER_KNOBS->DESIRED_TOTAL_BYTES, + SERVER_KNOBS->DESIRED_TOTAL_BYTES)); - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); for (auto& kv : kvs) { auto ver = decodeTagMessagesKey(kv.key); @@ -1079,7 +1093,7 @@ ACTOR Future peekTLog(TLogData* self, uint32_t subVersion; rd >> messageLength >> subVersion; messageLength += sizeof(uint16_t) + sizeof(Tag); - messages << messageLength << subVersion << uint16_t(1) << tag; + messages << messageLength << subVersion << uint16_t(1) << reqTag; messageLength -= (sizeof(subVersion) + sizeof(uint16_t) + sizeof(Tag)); messages.serializeBytes(rd.readBytes(messageLength), messageLength); } @@ -1090,8 +1104,8 @@ ACTOR Future peekTLog(TLogData* self, else messages.serializeBytes(messages2.toValue()); } else { - peekMessagesFromMemory(logData, tag, begin, messages, endVersion); - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); + peekMessagesFromMemory(logData, reqTag, reqBegin, messages, endVersion); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); } TLogPeekReply reply; @@ -1101,40 +1115,48 @@ ACTOR Future peekTLog(TLogData* self, reply.messages = StringRef(reply.arena, messages.toValue()); reply.end = endVersion; - //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); + //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress()); - if (sequence.present()) { + if (reqSequence.present()) { auto& trackerData = self->peekTracker[peekId]; trackerData.lastUpdate = now(); - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + auto& sequenceData = trackerData.sequence_version[sequence + 1]; if (sequenceData.isSet()) { if (sequenceData.getFuture().get() != reply.end) { - TEST(true); // 0 tlog peek second attempt ended at a different version (2) - throw operation_obsolete(); + TEST(true); // tlog peek second attempt ended at a different version (2) + replyPromise.sendError(operation_obsolete()); + return Void(); } } else { sequenceData.send(reply.end); } - reply.begin = begin; + reply.begin = reqBegin; } - return reply; + replyPromise.send(reply); + return Void(); } // This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { self->activePeekStreams++; - TraceEvent(SevDebug, "TLogPeekStream", logData->logId).detail("Token", req.reply.getEndpoint().token); state Version begin = req.begin; state bool onlySpilled = false; - req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); loop { state TLogPeekStreamReply reply; + state Promise promise; + state Future future(promise.getFuture()); try { wait(req.reply.onReady() && - store(reply.rep, peekTLog(self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled))); + tLogPeekMessages(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled)); + ASSERT(future.isReady()); + if (future.isError()) { + throw future.getError(); + } + + reply.rep = future.get(); req.reply.send(reply); begin = reply.rep.end; onlySpilled = reply.rep.onlySpilled; @@ -1157,161 +1179,6 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref } } - -ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { - state BinaryWriter messages(Unversioned()); - state BinaryWriter messages2(Unversioned()); - state int sequence = -1; - state UID peekId; - state OldTag oldTag = convertTag(req.tag); - - if (req.sequence.present()) { - try { - peekId = req.sequence.get().first; - sequence = req.sequence.get().second; - if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - self->peekTracker.find(peekId) == self->peekTracker.end()) { - throw operation_obsolete(); - } - if (sequence > 0) { - auto& trackerData = self->peekTracker[peekId]; - trackerData.lastUpdate = now(); - Version ver = wait(trackerData.sequence_version[sequence].getFuture()); - req.begin = std::max(ver, req.begin); - wait(yield()); - } - } catch (Error& e) { - if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { - req.reply.sendError(e); - return Void(); - } else { - throw; - } - } - } - - if (req.returnIfBlocked && logData->version.get() < req.begin) { - req.reply.sendError(end_of_stream()); - return Void(); - } - - //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - // Wait until we have something to return that the caller doesn't already have - if (logData->version.get() < req.begin) { - wait(logData->version.whenAtLeast(req.begin)); - wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); - } - - state Version endVersion = logData->version.get() + 1; - - Version poppedVer = poppedVersion(logData, oldTag); - if (poppedVer > req.begin) { - TLogPeekReply rep; - rep.maxKnownVersion = logData->version.get(); - rep.minKnownCommittedVersion = 0; - rep.popped = poppedVer; - rep.end = poppedVer; - rep.onlySpilled = false; - - if (req.sequence.present()) { - auto& trackerData = self->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; - trackerData.lastUpdate = now(); - if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - req.reply.sendError(operation_obsolete()); - if (!sequenceData.isSet()) - sequenceData.sendError(operation_obsolete()); - return Void(); - } - if (sequenceData.isSet()) { - if (sequenceData.getFuture().get() != rep.end) { - TEST(true); // tlog peek second attempt ended at a different version - req.reply.sendError(operation_obsolete()); - return Void(); - } - } else { - sequenceData.send(rep.end); - } - rep.begin = req.begin; - } - - req.reply.send(rep); - return Void(); - } - - // grab messages from disk - //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - if (req.begin <= logData->persistentDataDurableVersion) { - // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We - // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if - // an initial attempt to read from disk results in insufficient data and the required data is no longer in - // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the - // result? - - peekMessagesFromMemory(logData, req.tag, req.begin, messages2, endVersion); - - RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, oldTag, req.begin), - persistTagMessagesKey(logData->logId, oldTag, logData->persistentDataDurableVersion + 1)), - SERVER_KNOBS->DESIRED_TOTAL_BYTES, - SERVER_KNOBS->DESIRED_TOTAL_BYTES)); - - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); - - for (auto& kv : kvs) { - auto ver = decodeTagMessagesKey(kv.key); - messages << int32_t(-1) << ver; - - BinaryReader rd(kv.value, Unversioned()); - while (!rd.empty()) { - int32_t messageLength; - uint32_t subVersion; - rd >> messageLength >> subVersion; - messageLength += sizeof(uint16_t) + sizeof(Tag); - messages << messageLength << subVersion << uint16_t(1) << req.tag; - messageLength -= (sizeof(subVersion) + sizeof(uint16_t) + sizeof(Tag)); - messages.serializeBytes(rd.readBytes(messageLength), messageLength); - } - } - - if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) - endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1; - else - messages.serializeBytes(messages2.toValue()); - } else { - peekMessagesFromMemory(logData, req.tag, req.begin, messages, endVersion); - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); - } - - TLogPeekReply reply; - reply.maxKnownVersion = logData->version.get(); - reply.minKnownCommittedVersion = 0; - reply.onlySpilled = false; - reply.messages = messages.toValue(); - reply.end = endVersion; - - //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); - - if (req.sequence.present()) { - auto& trackerData = self->peekTracker[peekId]; - trackerData.lastUpdate = now(); - auto& sequenceData = trackerData.sequence_version[sequence + 1]; - if (sequenceData.isSet()) { - if (sequenceData.getFuture().get() != reply.end) { - TEST(true); // tlog peek second attempt ended at a different version (2) - req.reply.sendError(operation_obsolete()); - return Void(); - } - } else { - sequenceData.send(reply.end); - } - reply.begin = req.begin; - } - - req.reply.send(reply); - return Void(); -} - ACTOR Future doQueueCommit(TLogData* self, Reference logData) { state Version ver = logData->version.get(); state Version commitNumber = self->queueCommitBegin + 1; @@ -1476,7 +1343,8 @@ ACTOR Future serveTLogInterface(TLogData* self, PromiseStream warningCollectorInput) { loop choose { when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) { - logData->addActor.send(tLogPeekMessages(self, req, logData)); + logData->addActor.send(tLogPeekMessages( + req.reply, self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); } when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { TraceEvent(SevDebug, "TLogPeekStream", logData->logId) diff --git a/fdbserver/OldTLogServer_6_0.actor.cpp b/fdbserver/OldTLogServer_6_0.actor.cpp index b6964bf9c9..5c27581b2c 100644 --- a/fdbserver/OldTLogServer_6_0.actor.cpp +++ b/fdbserver/OldTLogServer_6_0.actor.cpp @@ -1208,301 +1208,37 @@ void peekMessagesFromMemory(Reference self, } // Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request -ACTOR Future peekTLog(TLogData* self, - Reference logData, - Version begin, - Tag tag, - bool returnIfBlocked = false, - bool reqOnlySpilled = false, - Optional> sequence = Optional>()) { - state BinaryWriter messages(Unversioned()); - state BinaryWriter messages2(Unversioned()); - state int sequenceNum = -1; - state UID peekId; - state double queueStart = now(); - - if (tag.locality == tagLocalityTxs && tag.id >= logData->txsTags && logData->txsTags > 0) { - tag.id = tag.id % logData->txsTags; - } - - // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence requests - // STEP: a. mark obsolete sequence requests; b. wait previous sequence requests are handled in order - if (sequence.present()) { - peekId = sequence.get().first; - sequenceNum = sequence.get().second; - if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - logData->peekTracker.find(peekId) == logData->peekTracker.end()) { - throw operation_obsolete(); - } - auto& trackerData = logData->peekTracker[peekId]; - if (sequenceNum == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - trackerData.tag = tag; - trackerData.sequence_version[0].send(std::make_pair(begin, reqOnlySpilled)); - } - auto seqBegin = trackerData.sequence_version.begin(); - // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. - while (trackerData.sequence_version.size() && - seqBegin->first <= sequenceNum - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { - if (seqBegin->second.canBeSet()) { - seqBegin->second.sendError(operation_obsolete()); - } - trackerData.sequence_version.erase(seqBegin); - seqBegin = trackerData.sequence_version.begin(); - } - - if (trackerData.sequence_version.size() && sequenceNum < seqBegin->first) { - throw operation_obsolete(); - } - - Future> fPrevPeekData = trackerData.sequence_version[sequenceNum].getFuture(); - if (fPrevPeekData.isReady()) { - trackerData.unblockedPeeks++; - double t = now() - trackerData.lastUpdate; - if (t > trackerData.idleMax) - trackerData.idleMax = t; - trackerData.idleTime += t; - } - trackerData.lastUpdate = now(); - std::pair prevPeekData = wait(fPrevPeekData); - begin = std::max(prevPeekData.first, begin); - reqOnlySpilled = prevPeekData.second; - wait(yield()); - } - - state double blockStart = now(); - - if (returnIfBlocked && logData->version.get() < begin) { - if (sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - trackerData.lastUpdate = now(); - if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(begin, reqOnlySpilled)); - } - } - throw end_of_stream(); - } - - //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - // Wait until we have something to return that the caller doesn't already have - if (logData->version.get() < begin) { - wait(logData->version.whenAtLeast(begin)); - wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); - } - - if (logData->locality != tagLocalitySatellite && tag.locality == tagLocalityLogRouter) { - wait(self->concurrentLogRouterReads.take()); - state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); - wait(delay(0.0, TaskPriority::Low)); - } - - if (begin <= logData->persistentDataDurableVersion && tag.locality != tagLocalityTxs && tag != txsTag) { - // Reading spilled data will almost always imply that the storage server is >5s behind the rest - // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up - // slightly faster over keeping the rest of the cluster operating normally. - // txsTag is only ever peeked on recovery, and we would still wish to prioritize requests - // that impact recovery duration. - wait(delay(0, TaskPriority::TLogSpilledPeekReply)); - } - - state double workStart = now(); - - Version poppedVer = poppedVersion(logData, tag); - if (poppedVer > begin) { - // reply with an empty message and let the next reply start from poppedVer - TLogPeekReply rep; - rep.maxKnownVersion = logData->version.get(); - rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; - rep.popped = poppedVer; - rep.end = poppedVer; - rep.onlySpilled = false; - - // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence - // requests. - if (sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - trackerData.lastUpdate = now(); - if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { - if (!sequenceData.isSet()) - sequenceData.sendError(operation_obsolete()); - throw operation_obsolete(); - } - if (sequenceData.isSet()) { - if (sequenceData.getFuture().get().first != rep.end) { - TEST(true); // 1 tlog peek second attempt ended at a different version - throw operation_obsolete(); - } - } else { - sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); - } - rep.begin = begin; - } - - return rep; - } - - state Version endVersion = logData->version.get() + 1; - state bool onlySpilled = false; - - // grab messages from disk - //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - if (begin <= logData->persistentDataDurableVersion) { - // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We - // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if - // an initial attempt to read from disk results in insufficient data and the required data is no longer in - // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the - // result? - - if (reqOnlySpilled) { - endVersion = logData->persistentDataDurableVersion + 1; - } else { - peekMessagesFromMemory(logData, tag, begin, messages2, endVersion); - } - - RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, tag, begin), - persistTagMessagesKey(logData->logId, tag, logData->persistentDataDurableVersion + 1)), - SERVER_KNOBS->DESIRED_TOTAL_BYTES, - SERVER_KNOBS->DESIRED_TOTAL_BYTES)); - - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().address).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); - - for (auto& kv : kvs) { - auto ver = decodeTagMessagesKey(kv.key); - messages << VERSION_HEADER << ver; - messages.serializeBytes(kv.value); - } - - if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { - endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1; - onlySpilled = true; - } else { - messages.serializeBytes(messages2.toValue()); - } - } else { - peekMessagesFromMemory(logData, tag, begin, messages, endVersion); - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().address).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); - } - - TLogPeekReply reply; - reply.maxKnownVersion = logData->version.get(); - reply.minKnownCommittedVersion = logData->minKnownCommittedVersion; - reply.messages = StringRef(reply.arena, messages.toValue()); - reply.end = endVersion; - reply.onlySpilled = onlySpilled; - - //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", reply.getEndpoint().address); - - if (sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - trackerData.lastUpdate = now(); - - double queueT = blockStart - queueStart; - double blockT = workStart - blockStart; - double workT = now() - workStart; - - trackerData.totalPeeks++; - trackerData.replyBytes += reply.messages.size(); - - if (queueT > trackerData.queueMax) - trackerData.queueMax = queueT; - if (blockT > trackerData.blockMax) - trackerData.blockMax = blockT; - if (workT > trackerData.workMax) - trackerData.workMax = workT; - - trackerData.queueTime += queueT; - trackerData.blockTime += blockT; - trackerData.workTime += workT; - - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { - if (!sequenceData.isSet()) { - // It would technically be more correct to .send({req.begin, req.onlySpilled}), as the next - // request might still be in the window of active requests, but LogSystemPeekCursor will - // throw away all future responses upon getting an operation_obsolete(), so computing a - // response will probably be a waste of CPU. - sequenceData.sendError(operation_obsolete()); - } - throw operation_obsolete(); - } - if (sequenceData.isSet()) { - trackerData.duplicatePeeks++; - if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // 1 tlog peek second attempt ended at a different version (2) - throw operation_obsolete(); - } - } else { - sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); - } - reply.begin = begin; - } - - return reply; -} - -// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover -ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { - self->activePeekStreams++; - - state Version begin = req.begin; - state bool onlySpilled = false; - if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { - req.tag.id = req.tag.id % logData->txsTags; - } - req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); - loop { - state TLogPeekStreamReply reply; - try { - wait(req.reply.onReady() && - store(reply.rep, peekTLog(self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled))); - req.reply.send(reply); - begin = reply.rep.end; - onlySpilled = reply.rep.onlySpilled; - if (reply.rep.end > logData->version.get()) { - wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); - } else { - wait(delay(0, g_network->getCurrentTask())); - } - } catch (Error& e) { - self->activePeekStreams--; - TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); - - if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { - req.reply.sendError(e); - return Void(); - } else { - throw; - } - } - } -} - - -ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { +ACTOR template +Future tLogPeekMessages(PromiseType replyPromise, + TLogData* self, + Reference logData, + Version reqBegin, + Tag reqTag, + bool reqReturnIfBlocked = false, + bool reqOnlySpilled = false, + Optional> reqSequence = Optional>()) { state BinaryWriter messages(Unversioned()); state BinaryWriter messages2(Unversioned()); state int sequence = -1; state UID peekId; state double queueStart = now(); - if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { - req.tag.id = req.tag.id % logData->txsTags; + if (reqTag.locality == tagLocalityTxs && reqTag.id >= logData->txsTags && logData->txsTags > 0) { + reqTag.id = reqTag.id % logData->txsTags; } - if (req.sequence.present()) { + if (reqSequence.present()) { try { - peekId = req.sequence.get().first; - sequence = req.sequence.get().second; + peekId = reqSequence.get().first; + sequence = reqSequence.get().second; if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && logData->peekTracker.find(peekId) == logData->peekTracker.end()) { throw operation_obsolete(); } auto& trackerData = logData->peekTracker[peekId]; if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - trackerData.tag = req.tag; - trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled)); + trackerData.tag = reqTag; + trackerData.sequence_version[0].send(std::make_pair(reqBegin, reqOnlySpilled)); } auto seqBegin = trackerData.sequence_version.begin(); while (trackerData.sequence_version.size() && @@ -1529,12 +1265,12 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen trackerData.lastUpdate = now(); std::pair prevPeekData = wait(fPrevPeekData); - req.begin = std::max(prevPeekData.first, req.begin); - req.onlySpilled = prevPeekData.second; + reqBegin = std::max(prevPeekData.first, reqBegin); + reqOnlySpilled = prevPeekData.second; wait(yield()); } catch (Error& e) { if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { - req.reply.sendError(e); + replyPromise.sendError(e); return Void(); } else { throw; @@ -1544,32 +1280,32 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen state double blockStart = now(); - if (req.returnIfBlocked && logData->version.get() < req.begin) { - req.reply.sendError(end_of_stream()); - if (req.sequence.present()) { + if (reqReturnIfBlocked && logData->version.get() < reqBegin) { + replyPromise.sendError(end_of_stream()); + if (reqSequence.present()) { auto& trackerData = logData->peekTracker[peekId]; auto& sequenceData = trackerData.sequence_version[sequence + 1]; if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); + sequenceData.send(std::make_pair(reqBegin, reqOnlySpilled)); } } return Void(); } - //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", reqBegin.epoch).detail("ReqBeginSeq", reqBegin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", reqTag1).detail("Tag2", reqTag2); // Wait until we have something to return that the caller doesn't already have - if (logData->version.get() < req.begin) { - wait(logData->version.whenAtLeast(req.begin)); + if (logData->version.get() < reqBegin) { + wait(logData->version.whenAtLeast(reqBegin)); wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); } - if (logData->locality != tagLocalitySatellite && req.tag.locality == tagLocalityLogRouter) { + if (logData->locality != tagLocalitySatellite && reqTag.locality == tagLocalityLogRouter) { wait(self->concurrentLogRouterReads.take()); state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); wait(delay(0.0, TaskPriority::Low)); } - if (req.begin <= logData->persistentDataDurableVersion && req.tag.locality != tagLocalityTxs && req.tag != txsTag) { + if (reqBegin <= logData->persistentDataDurableVersion && reqTag.locality != tagLocalityTxs && reqTag != txsTag) { // Reading spilled data will almost always imply that the storage server is >5s behind the rest // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up // slightly faster over keeping the rest of the cluster operating normally. @@ -1580,8 +1316,8 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen state double workStart = now(); - Version poppedVer = poppedVersion(logData, req.tag); - if (poppedVer > req.begin) { + Version poppedVer = poppedVersion(logData, reqTag); + if (poppedVer > reqBegin) { TLogPeekReply rep; rep.maxKnownVersion = logData->version.get(); rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; @@ -1589,12 +1325,12 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen rep.end = poppedVer; rep.onlySpilled = false; - if (req.sequence.present()) { + if (reqSequence.present()) { auto& trackerData = logData->peekTracker[peekId]; auto& sequenceData = trackerData.sequence_version[sequence + 1]; trackerData.lastUpdate = now(); if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - req.reply.sendError(operation_obsolete()); + replyPromise.sendError(operation_obsolete()); if (!sequenceData.isSet()) sequenceData.sendError(operation_obsolete()); return Void(); @@ -1602,16 +1338,16 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen if (sequenceData.isSet()) { if (sequenceData.getFuture().get().first != rep.end) { TEST(true); // tlog peek second attempt ended at a different version - req.reply.sendError(operation_obsolete()); + replyPromise.sendError(operation_obsolete()); return Void(); } } else { sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); } - rep.begin = req.begin; + rep.begin = reqBegin; } - req.reply.send(rep); + replyPromise.send(rep); return Void(); } @@ -1619,27 +1355,27 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen state bool onlySpilled = false; // grab messages from disk - //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - if (req.begin <= logData->persistentDataDurableVersion) { + //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", reqBegin.epoch).detail("ReqBeginSeq", reqBegin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", reqTag1).detail("Tag2", reqTag2); + if (reqBegin <= logData->persistentDataDurableVersion) { // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if // an initial attempt to read from disk results in insufficient data and the required data is no longer in // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the // result? - if (req.onlySpilled) { + if (reqOnlySpilled) { endVersion = logData->persistentDataDurableVersion + 1; } else { - peekMessagesFromMemory(logData, req.tag, req.begin, messages2, endVersion); + peekMessagesFromMemory(logData, reqTag, reqBegin, messages2, endVersion); } RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, req.tag, req.begin), - persistTagMessagesKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)), + KeyRangeRef(persistTagMessagesKey(logData->logId, reqTag, reqBegin), + persistTagMessagesKey(logData->logId, reqTag, logData->persistentDataDurableVersion + 1)), SERVER_KNOBS->DESIRED_TOTAL_BYTES, SERVER_KNOBS->DESIRED_TOTAL_BYTES)); - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().address).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().address).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); for (auto& kv : kvs) { auto ver = decodeTagMessagesKey(kv.key); @@ -1654,20 +1390,20 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen messages.serializeBytes(messages2.toValue()); } } else { - peekMessagesFromMemory(logData, req.tag, req.begin, messages, endVersion); - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().address).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); + peekMessagesFromMemory(logData, reqTag, reqBegin, messages, endVersion); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().address).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); } TLogPeekReply reply; reply.maxKnownVersion = logData->version.get(); reply.minKnownCommittedVersion = logData->minKnownCommittedVersion; - reply.messages = messages.toValue(); + reply.messages = StringRef(reply.arena, messages.toValue()); reply.end = endVersion; reply.onlySpilled = onlySpilled; - //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().address); + //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", replyPromise.getEndpoint().address); - if (req.sequence.present()) { + if (reqSequence.present()) { auto& trackerData = logData->peekTracker[peekId]; trackerData.lastUpdate = now(); @@ -1691,7 +1427,7 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen auto& sequenceData = trackerData.sequence_version[sequence + 1]; if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - req.reply.sendError(operation_obsolete()); + replyPromise.sendError(operation_obsolete()); if (!sequenceData.isSet()) sequenceData.sendError(operation_obsolete()); return Void(); @@ -1700,19 +1436,60 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen trackerData.duplicatePeeks++; if (sequenceData.getFuture().get().first != reply.end) { TEST(true); // tlog peek second attempt ended at a different version (2) - req.reply.sendError(operation_obsolete()); + replyPromise.sendError(operation_obsolete()); return Void(); } } else { sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); } - reply.begin = req.begin; + reply.begin = reqBegin; } - req.reply.send(reply); + replyPromise.send(reply); return Void(); } +// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover +ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { + self->activePeekStreams++; + + state Version begin = req.begin; + state bool onlySpilled = false; + req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); + loop { + state TLogPeekStreamReply reply; + state Promise promise; + state Future future(promise.getFuture()); + try { + wait(req.reply.onReady() && + tLogPeekMessages(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled)); + ASSERT(future.isReady()); + if (future.isError()) { + throw future.getError(); + } + + reply.rep = future.get(); + req.reply.send(reply); + begin = reply.rep.end; + onlySpilled = reply.rep.onlySpilled; + if (reply.rep.end > logData->version.get()) { + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } else { + wait(delay(0, g_network->getCurrentTask())); + } + } catch (Error& e) { + self->activePeekStreams--; + TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); + + if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { + req.reply.sendError(e); + return Void(); + } else { + throw; + } + } + } +} ACTOR Future doQueueCommit(TLogData* self, Reference logData, @@ -2208,7 +1985,8 @@ ACTOR Future serveTLogInterface(TLogData* self, } } when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) { - logData->addActor.send(tLogPeekMessages(self, req, logData)); + logData->addActor.send(tLogPeekMessages( + req.reply, self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); } when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { TraceEvent(SevDebug, "TLogPeekStream", logData->logId) diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index 88029f9eb1..81025d93de 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -1547,67 +1547,66 @@ ACTOR Future> parseMessagesForTag(StringRef commitBlob, T } // Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request -ACTOR template -Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, - Reference logData, - Version begin, - Tag tag, - bool returnIfBlocked = false, - bool reqOnlySpilled = false, - Optional> sequence = Optional>()) { +ACTOR template +Future tLogPeekMessages(PromiseType replyPromise, + TLogData* self, + Reference logData, + Version reqBegin, + Tag reqTag, + bool reqReturnIfBlocked = false, + bool reqOnlySpilled = false, + Optional> reqSequence = Optional>()) { state BinaryWriter messages(Unversioned()); state BinaryWriter messages2(Unversioned()); - state int sequenceNum = -1; + state int sequence = -1; state UID peekId; state double queueStart = now(); - if (tag.locality == tagLocalityTxs && tag.id >= logData->txsTags && logData->txsTags > 0) { - tag.id = tag.id % logData->txsTags; + if (reqTag.locality == tagLocalityTxs && reqTag.id >= logData->txsTags && logData->txsTags > 0) { + reqTag.id = reqTag.id % logData->txsTags; } - // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence requests - // STEP: a. mark obsolete sequence requests; b. wait previous sequence requests are handled in order - if (sequence.present()) { - try{ - peekId = sequence.get().first; - sequenceNum = sequence.get().second; - if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - logData->peekTracker.find(peekId) == logData->peekTracker.end()) { - throw operation_obsolete(); - } - auto& trackerData = logData->peekTracker[peekId]; - if (sequenceNum == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - trackerData.tag = tag; - trackerData.sequence_version[0].send(std::make_pair(begin, reqOnlySpilled)); - } - auto seqBegin = trackerData.sequence_version.begin(); - // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. - while (trackerData.sequence_version.size() && - seqBegin->first <= sequenceNum - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { - if (seqBegin->second.canBeSet()) { - seqBegin->second.sendError(operation_obsolete()); - } - trackerData.sequence_version.erase(seqBegin); - seqBegin = trackerData.sequence_version.begin(); - } + if (reqSequence.present()) { + try { + peekId = reqSequence.get().first; + sequence = reqSequence.get().second; + if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + logData->peekTracker.find(peekId) == logData->peekTracker.end()) { + throw operation_obsolete(); + } + auto& trackerData = logData->peekTracker[peekId]; + if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { + trackerData.tag = reqTag; + trackerData.sequence_version[0].send(std::make_pair(reqBegin, reqOnlySpilled)); + } + auto seqBegin = trackerData.sequence_version.begin(); + // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. + while (trackerData.sequence_version.size() && + seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + if (seqBegin->second.canBeSet()) { + seqBegin->second.sendError(operation_obsolete()); + } + trackerData.sequence_version.erase(seqBegin); + seqBegin = trackerData.sequence_version.begin(); + } - if (trackerData.sequence_version.size() && sequenceNum < seqBegin->first) { - throw operation_obsolete(); - } + if (trackerData.sequence_version.size() && sequence < seqBegin->first) { + throw operation_obsolete(); + } - Future> fPrevPeekData = trackerData.sequence_version[sequenceNum].getFuture(); - if (fPrevPeekData.isReady()) { - trackerData.unblockedPeeks++; - double t = now() - trackerData.lastUpdate; - if (t > trackerData.idleMax) - trackerData.idleMax = t; - trackerData.idleTime += t; - } - trackerData.lastUpdate = now(); - std::pair prevPeekData = wait(fPrevPeekData); - begin = std::max(prevPeekData.first, begin); - reqOnlySpilled = prevPeekData.second; - wait(yield()); + Future> fPrevPeekData = trackerData.sequence_version[sequence].getFuture(); + if (fPrevPeekData.isReady()) { + trackerData.unblockedPeeks++; + double t = now() - trackerData.lastUpdate; + if (t > trackerData.idleMax) + trackerData.idleMax = t; + trackerData.idleTime += t; + } + trackerData.lastUpdate = now(); + std::pair prevPeekData = wait(fPrevPeekData); + reqBegin = std::max(prevPeekData.first, reqBegin); + reqOnlySpilled = prevPeekData.second; + wait(yield()); } catch (Error& e) { if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { replyPromise.sendError(e); @@ -1620,33 +1619,32 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, state double blockStart = now(); - if (returnIfBlocked && logData->version.get() < begin) { - replyPromise.sendError(end_of_stream()); - if (sequence.present()) { + if (reqReturnIfBlocked && logData->version.get() < reqBegin) { + replyPromise.sendError(end_of_stream()); + if (reqSequence.present()) { auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - trackerData.lastUpdate = now(); + auto& sequenceData = trackerData.sequence_version[sequence + 1]; if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(begin, reqOnlySpilled)); + sequenceData.send(std::make_pair(reqBegin, reqOnlySpilled)); } } return Void(); } - //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", reqBegin.epoch).detail("ReqBeginSeq", reqBegin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", reqTag1).detail("Tag2", reqTag2); // Wait until we have something to return that the caller doesn't already have - if (logData->version.get() < begin) { - wait(logData->version.whenAtLeast(begin)); + if (logData->version.get() < reqBegin) { + wait(logData->version.whenAtLeast(reqBegin)); wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); } - if (logData->locality != tagLocalitySatellite && tag.locality == tagLocalityLogRouter) { + if (reqTag.locality == tagLocalityLogRouter) { wait(self->concurrentLogRouterReads.take()); state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); wait(delay(0.0, TaskPriority::Low)); } - if (begin <= logData->persistentDataDurableVersion && tag.locality != tagLocalityTxs && tag != txsTag) { + if (reqBegin <= logData->persistentDataDurableVersion && reqTag.locality != tagLocalityTxs && reqTag != txsTag) { // Reading spilled data will almost always imply that the storage server is >5s behind the rest // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up // slightly faster over keeping the rest of the cluster operating normally. @@ -1657,9 +1655,8 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, state double workStart = now(); - Version poppedVer = poppedVersion(logData, tag); - if (poppedVer > begin) { - // reply with an empty message and let the next reply start from poppedVer + Version poppedVer = poppedVersion(logData, reqTag); + if (poppedVer > reqBegin) { TLogPeekReply rep; rep.maxKnownVersion = logData->version.get(); rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; @@ -1667,13 +1664,11 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, rep.end = poppedVer; rep.onlySpilled = false; - // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence - // requests. - if (sequence.present()) { + if (reqSequence.present()) { auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + auto& sequenceData = trackerData.sequence_version[sequence + 1]; trackerData.lastUpdate = now(); - if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { replyPromise.sendError(operation_obsolete()); if (!sequenceData.isSet()) sequenceData.sendError(operation_obsolete()); @@ -1681,14 +1676,14 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, } if (sequenceData.isSet()) { if (sequenceData.getFuture().get().first != rep.end) { - TEST(true); // 1 tlog peek second attempt ended at a different version + TEST(true); // tlog peek second attempt ended at a different version replyPromise.sendError(operation_obsolete()); return Void(); } } else { sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); } - rep.begin = begin; + rep.begin = reqBegin; } replyPromise.send(rep); @@ -1699,8 +1694,8 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, state bool onlySpilled = false; // grab messages from disk - //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - if (begin <= logData->persistentDataDurableVersion) { + //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", reqBegin.epoch).detail("ReqBeginSeq", reqBegin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", reqTag1).detail("Tag2", reqTag2); + if (reqBegin <= logData->persistentDataDurableVersion) { // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if // an initial attempt to read from disk results in insufficient data and the required data is no longer in @@ -1710,13 +1705,13 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, if (reqOnlySpilled) { endVersion = logData->persistentDataDurableVersion + 1; } else { - peekMessagesFromMemory(logData, tag, begin, messages2, endVersion); + peekMessagesFromMemory(logData, reqTag, reqBegin, messages2, endVersion); } - if (tag.locality == tagLocalityTxs || tag == txsTag) { + if (reqTag.locality == tagLocalityTxs || reqTag == txsTag) { RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, tag, begin), - persistTagMessagesKey(logData->logId, tag, logData->persistentDataDurableVersion + 1)), + KeyRangeRef(persistTagMessagesKey(logData->logId, reqTag, reqBegin), + persistTagMessagesKey(logData->logId, reqTag, logData->persistentDataDurableVersion + 1)), SERVER_KNOBS->DESIRED_TOTAL_BYTES, SERVER_KNOBS->DESIRED_TOTAL_BYTES)); @@ -1735,11 +1730,12 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, } else { // FIXME: Limit to approximately DESIRED_TOTATL_BYTES somehow. RangeResult kvrefs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessageRefsKey(logData->logId, tag, begin), - persistTagMessageRefsKey(logData->logId, tag, logData->persistentDataDurableVersion + 1)), + KeyRangeRef( + persistTagMessageRefsKey(logData->logId, reqTag, reqBegin), + persistTagMessageRefsKey(logData->logId, reqTag, logData->persistentDataDurableVersion + 1)), SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1)); - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); state std::vector> commitLocations; state bool earlyEnd = false; @@ -1756,7 +1752,7 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, earlyEnd = true; break; } - if (sd.version >= begin) { + if (sd.version >= reqBegin) { firstVersion = std::min(firstVersion, sd.version); const IDiskQueue::location end = sd.start.lo + sd.length; commitLocations.emplace_back(sd.start, end); @@ -1798,7 +1794,7 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, messages << VERSION_HEADER << entry.version; std::vector rawMessages = - wait(parseMessagesForTag(entry.messages, tag, logData->logRouterTags)); + wait(parseMessagesForTag(entry.messages, reqTag, logData->logRouterTags)); for (const StringRef& msg : rawMessages) { messages.serializeBytes(msg); } @@ -1821,10 +1817,10 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, if (reqOnlySpilled) { endVersion = logData->persistentDataDurableVersion + 1; } else { - peekMessagesFromMemory(logData, tag, begin, messages, endVersion); + peekMessagesFromMemory(logData, reqTag, reqBegin, messages, endVersion); } - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); } TLogPeekReply reply; @@ -1834,12 +1830,9 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, reply.end = endVersion; reply.onlySpilled = onlySpilled; - //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("Tag", req.tag.toString()). - // detail("BeginVer", req.begin).detail("EndVer", reply.end). - // detail("MsgBytes", reply.messages.expectedSize()). - // detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); + //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress()); - if (sequence.present()) { + if (reqSequence.present()) { auto& trackerData = logData->peekTracker[peekId]; trackerData.lastUpdate = now(); @@ -1861,29 +1854,24 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, trackerData.blockTime += blockT; trackerData.workTime += workT; - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + auto& sequenceData = trackerData.sequence_version[sequence + 1]; + if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { replyPromise.sendError(operation_obsolete()); - if (!sequenceData.isSet()) { - // It would technically be more correct to .send({req.begin, req.onlySpilled}), as the next - // request might still be in the window of active requests, but LogSystemPeekCursor will - // throw away all future responses upon getting an operation_obsolete(), so computing a - // response will probably be a waste of CPU. + if (!sequenceData.isSet()) sequenceData.sendError(operation_obsolete()); - } return Void(); } if (sequenceData.isSet()) { trackerData.duplicatePeeks++; if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // 1 tlog peek second attempt ended at a different version (2) + TEST(true); // tlog peek second attempt ended at a different version (2) replyPromise.sendError(operation_obsolete()); return Void(); } } else { sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); } - reply.begin = begin; + reply.begin = reqBegin; } replyPromise.send(reply); @@ -1902,9 +1890,10 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref state Promise promise; state Future future(promise.getFuture()); try { - wait(req.reply.onReady() && peekTLogAndSend(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled)); + wait(req.reply.onReady() && + tLogPeekMessages(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled)); ASSERT(future.isReady()); - if(future.isError()) { + if (future.isError()) { throw future.getError(); } @@ -1931,330 +1920,6 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref } } - -ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { - state BinaryWriter messages(Unversioned()); - state BinaryWriter messages2(Unversioned()); - state int sequence = -1; - state UID peekId; - state double queueStart = now(); - - if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { - req.tag.id = req.tag.id % logData->txsTags; - } - - if (req.sequence.present()) { - try { - peekId = req.sequence.get().first; - sequence = req.sequence.get().second; - if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - logData->peekTracker.find(peekId) == logData->peekTracker.end()) { - throw operation_obsolete(); - } - auto& trackerData = logData->peekTracker[peekId]; - if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - trackerData.tag = req.tag; - trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled)); - } - auto seqBegin = trackerData.sequence_version.begin(); - // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. - while (trackerData.sequence_version.size() && - seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { - if (seqBegin->second.canBeSet()) { - seqBegin->second.sendError(operation_obsolete()); - } - trackerData.sequence_version.erase(seqBegin); - seqBegin = trackerData.sequence_version.begin(); - } - - if (trackerData.sequence_version.size() && sequence < seqBegin->first) { - throw operation_obsolete(); - } - - Future> fPrevPeekData = trackerData.sequence_version[sequence].getFuture(); - if (fPrevPeekData.isReady()) { - trackerData.unblockedPeeks++; - double t = now() - trackerData.lastUpdate; - if (t > trackerData.idleMax) - trackerData.idleMax = t; - trackerData.idleTime += t; - } - trackerData.lastUpdate = now(); - std::pair prevPeekData = wait(fPrevPeekData); - req.begin = std::max(prevPeekData.first, req.begin); - req.onlySpilled = prevPeekData.second; - wait(yield()); - } catch (Error& e) { - if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { - req.reply.sendError(e); - return Void(); - } else { - throw; - } - } - } - - state double blockStart = now(); - - if (req.returnIfBlocked && logData->version.get() < req.begin) { - req.reply.sendError(end_of_stream()); - if (req.sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; - if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); - } - } - return Void(); - } - - //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - // Wait until we have something to return that the caller doesn't already have - if (logData->version.get() < req.begin) { - wait(logData->version.whenAtLeast(req.begin)); - wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); - } - - if (req.tag.locality == tagLocalityLogRouter) { - wait(self->concurrentLogRouterReads.take()); - state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); - wait(delay(0.0, TaskPriority::Low)); - } - - if (req.begin <= logData->persistentDataDurableVersion && req.tag.locality != tagLocalityTxs && req.tag != txsTag) { - // Reading spilled data will almost always imply that the storage server is >5s behind the rest - // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up - // slightly faster over keeping the rest of the cluster operating normally. - // txsTag is only ever peeked on recovery, and we would still wish to prioritize requests - // that impact recovery duration. - wait(delay(0, TaskPriority::TLogSpilledPeekReply)); - } - - state double workStart = now(); - - Version poppedVer = poppedVersion(logData, req.tag); - if (poppedVer > req.begin) { - TLogPeekReply rep; - rep.maxKnownVersion = logData->version.get(); - rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; - rep.popped = poppedVer; - rep.end = poppedVer; - rep.onlySpilled = false; - - if (req.sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; - trackerData.lastUpdate = now(); - if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - req.reply.sendError(operation_obsolete()); - if (!sequenceData.isSet()) - sequenceData.sendError(operation_obsolete()); - return Void(); - } - if (sequenceData.isSet()) { - if (sequenceData.getFuture().get().first != rep.end) { - TEST(true); // tlog peek second attempt ended at a different version - req.reply.sendError(operation_obsolete()); - return Void(); - } - } else { - sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); - } - rep.begin = req.begin; - } - - req.reply.send(rep); - return Void(); - } - - state Version endVersion = logData->version.get() + 1; - state bool onlySpilled = false; - - // grab messages from disk - //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - if (req.begin <= logData->persistentDataDurableVersion) { - // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We - // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if - // an initial attempt to read from disk results in insufficient data and the required data is no longer in - // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the - // result? - - if (req.onlySpilled) { - endVersion = logData->persistentDataDurableVersion + 1; - } else { - peekMessagesFromMemory(logData, req.tag, req.begin, messages2, endVersion); - } - - if (req.tag.locality == tagLocalityTxs || req.tag == txsTag) { - RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, req.tag, req.begin), - persistTagMessagesKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)), - SERVER_KNOBS->DESIRED_TOTAL_BYTES, - SERVER_KNOBS->DESIRED_TOTAL_BYTES)); - - for (auto& kv : kvs) { - auto ver = decodeTagMessagesKey(kv.key); - messages << VERSION_HEADER << ver; - messages.serializeBytes(kv.value); - } - - if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { - endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1; - onlySpilled = true; - } else { - messages.serializeBytes(messages2.toValue()); - } - } else { - // FIXME: Limit to approximately DESIRED_TOTATL_BYTES somehow. - RangeResult kvrefs = wait(self->persistentData->readRange( - KeyRangeRef( - persistTagMessageRefsKey(logData->logId, req.tag, req.begin), - persistTagMessageRefsKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)), - SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1)); - - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); - - state std::vector> commitLocations; - state bool earlyEnd = false; - uint32_t mutationBytes = 0; - state uint64_t commitBytes = 0; - state Version firstVersion = std::numeric_limits::max(); - for (int i = 0; i < kvrefs.size() && i < SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK; i++) { - auto& kv = kvrefs[i]; - VectorRef spilledData; - BinaryReader r(kv.value, AssumeVersion(logData->protocolVersion)); - r >> spilledData; - for (const SpilledData& sd : spilledData) { - if (mutationBytes >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { - earlyEnd = true; - break; - } - if (sd.version >= req.begin) { - firstVersion = std::min(firstVersion, sd.version); - const IDiskQueue::location end = sd.start.lo + sd.length; - commitLocations.emplace_back(sd.start, end); - // This isn't perfect, because we aren't accounting for page boundaries, but should be - // close enough. - commitBytes += sd.length; - mutationBytes += sd.mutationBytes; - } - } - if (earlyEnd) - break; - } - earlyEnd = earlyEnd || (kvrefs.size() >= SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1); - wait(self->peekMemoryLimiter.take(TaskPriority::TLogSpilledPeekReply, commitBytes)); - state FlowLock::Releaser memoryReservation(self->peekMemoryLimiter, commitBytes); - state std::vector>> messageReads; - messageReads.reserve(commitLocations.size()); - for (const auto& pair : commitLocations) { - messageReads.push_back(self->rawPersistentQueue->read(pair.first, pair.second, CheckHashes::True)); - } - commitLocations.clear(); - wait(waitForAll(messageReads)); - - state Version lastRefMessageVersion = 0; - state int index = 0; - loop { - if (index >= messageReads.size()) - break; - Standalone queueEntryData = messageReads[index].get(); - uint8_t valid; - const uint32_t length = *(uint32_t*)queueEntryData.begin(); - queueEntryData = queueEntryData.substr(4, queueEntryData.size() - 4); - BinaryReader rd(queueEntryData, IncludeVersion()); - state TLogQueueEntry entry; - rd >> entry >> valid; - ASSERT(valid == 0x01); - ASSERT(length + sizeof(valid) == queueEntryData.size()); - - messages << VERSION_HEADER << entry.version; - - std::vector rawMessages = - wait(parseMessagesForTag(entry.messages, req.tag, logData->logRouterTags)); - for (const StringRef& msg : rawMessages) { - messages.serializeBytes(msg); - } - - lastRefMessageVersion = entry.version; - index++; - } - - messageReads.clear(); - memoryReservation.release(); - - if (earlyEnd) { - endVersion = lastRefMessageVersion + 1; - onlySpilled = true; - } else { - messages.serializeBytes(messages2.toValue()); - } - } - } else { - if (req.onlySpilled) { - endVersion = logData->persistentDataDurableVersion + 1; - } else { - peekMessagesFromMemory(logData, req.tag, req.begin, messages, endVersion); - } - - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); - } - - TLogPeekReply reply; - reply.maxKnownVersion = logData->version.get(); - reply.minKnownCommittedVersion = logData->minKnownCommittedVersion; - reply.messages = messages.toValue(); - reply.end = endVersion; - reply.onlySpilled = onlySpilled; - - //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("EndVer", reply.end).detail("MsgBytes", reply.messages.expectedSize()).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); - - if (req.sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - trackerData.lastUpdate = now(); - - double queueT = blockStart - queueStart; - double blockT = workStart - blockStart; - double workT = now() - workStart; - - trackerData.totalPeeks++; - trackerData.replyBytes += reply.messages.size(); - - if (queueT > trackerData.queueMax) - trackerData.queueMax = queueT; - if (blockT > trackerData.blockMax) - trackerData.blockMax = blockT; - if (workT > trackerData.workMax) - trackerData.workMax = workT; - - trackerData.queueTime += queueT; - trackerData.blockTime += blockT; - trackerData.workTime += workT; - - auto& sequenceData = trackerData.sequence_version[sequence + 1]; - if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - req.reply.sendError(operation_obsolete()); - if (!sequenceData.isSet()) - sequenceData.sendError(operation_obsolete()); - return Void(); - } - if (sequenceData.isSet()) { - trackerData.duplicatePeeks++; - if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // tlog peek second attempt ended at a different version (2) - req.reply.sendError(operation_obsolete()); - return Void(); - } - } else { - sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); - } - reply.begin = req.begin; - } - - req.reply.send(reply); - return Void(); -} - ACTOR Future watchDegraded(TLogData* self) { if (g_network->isSimulated() && g_simulator.speedUpSimulation) { return Void(); @@ -2765,7 +2430,8 @@ ACTOR Future serveTLogInterface(TLogData* self, } } when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) { - logData->addActor.send(tLogPeekMessages(self, req, logData)); + logData->addActor.send(tLogPeekMessages( + req.reply, self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); } when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { TraceEvent(SevDebug, "TLogPeekStream", logData->logId) diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index 3ca0926135..aa2afd5406 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1587,67 +1587,66 @@ ACTOR Future> parseMessagesForTag(StringRef commitBlob, T } // Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request -ACTOR template -Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, - Reference logData, - Version begin, - Tag tag, - bool returnIfBlocked = false, - bool reqOnlySpilled = false, - Optional> sequence = Optional>()) { +ACTOR template +Future tLogPeekMessages(PromiseType replyPromise, + TLogData* self, + Reference logData, + Version reqBegin, + Tag reqTag, + bool reqReturnIfBlocked = false, + bool reqOnlySpilled = false, + Optional> reqSequence = Optional>()) { state BinaryWriter messages(Unversioned()); state BinaryWriter messages2(Unversioned()); - state int sequenceNum = -1; + state int sequence = -1; state UID peekId; state double queueStart = now(); - if (tag.locality == tagLocalityTxs && tag.id >= logData->txsTags && logData->txsTags > 0) { - tag.id = tag.id % logData->txsTags; + if (reqTag.locality == tagLocalityTxs && reqTag.id >= logData->txsTags && logData->txsTags > 0) { + reqTag.id = reqTag.id % logData->txsTags; } - // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence requests - // STEP: a. mark obsolete sequence requests; b. wait previous sequence requests are handled in order - if (sequence.present()) { - try { - peekId = sequence.get().first; - sequenceNum = sequence.get().second; - if (sequenceNum >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - logData->peekTracker.find(peekId) == logData->peekTracker.end()) { - throw operation_obsolete(); - } - auto& trackerData = logData->peekTracker[peekId]; - if (sequenceNum == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - trackerData.tag = tag; - trackerData.sequence_version[0].send(std::make_pair(begin, reqOnlySpilled)); - } - auto seqBegin = trackerData.sequence_version.begin(); - // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. - while (trackerData.sequence_version.size() && - seqBegin->first <= sequenceNum - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { - if (seqBegin->second.canBeSet()) { - seqBegin->second.sendError(operation_obsolete()); - } - trackerData.sequence_version.erase(seqBegin); - seqBegin = trackerData.sequence_version.begin(); - } + if (reqSequence.present()) { + try { + peekId = reqSequence.get().first; + sequence = reqSequence.get().second; + if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && + logData->peekTracker.find(peekId) == logData->peekTracker.end()) { + throw operation_obsolete(); + } + auto& trackerData = logData->peekTracker[peekId]; + if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { + trackerData.tag = reqTag; + trackerData.sequence_version[0].send(std::make_pair(reqBegin, reqOnlySpilled)); + } + auto seqBegin = trackerData.sequence_version.begin(); + // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. + while (trackerData.sequence_version.size() && + seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + if (seqBegin->second.canBeSet()) { + seqBegin->second.sendError(operation_obsolete()); + } + trackerData.sequence_version.erase(seqBegin); + seqBegin = trackerData.sequence_version.begin(); + } - if (trackerData.sequence_version.size() && sequenceNum < seqBegin->first) { - throw operation_obsolete(); - } + if (trackerData.sequence_version.size() && sequence < seqBegin->first) { + throw operation_obsolete(); + } - Future> fPrevPeekData = trackerData.sequence_version[sequenceNum].getFuture(); - if (fPrevPeekData.isReady()) { - trackerData.unblockedPeeks++; - double t = now() - trackerData.lastUpdate; - if (t > trackerData.idleMax) - trackerData.idleMax = t; - trackerData.idleTime += t; - } - trackerData.lastUpdate = now(); - std::pair prevPeekData = wait(fPrevPeekData); - begin = std::max(prevPeekData.first, begin); - reqOnlySpilled = prevPeekData.second; - wait(yield()); + Future> fPrevPeekData = trackerData.sequence_version[sequence].getFuture(); + if (fPrevPeekData.isReady()) { + trackerData.unblockedPeeks++; + double t = now() - trackerData.lastUpdate; + if (t > trackerData.idleMax) + trackerData.idleMax = t; + trackerData.idleTime += t; + } + trackerData.lastUpdate = now(); + std::pair prevPeekData = wait(fPrevPeekData); + reqBegin = std::max(prevPeekData.first, reqBegin); + reqOnlySpilled = prevPeekData.second; + wait(yield()); } catch (Error& e) { if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { replyPromise.sendError(e); @@ -1660,33 +1659,33 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, state double blockStart = now(); - if (returnIfBlocked && logData->version.get() < begin) { - replyPromise.sendError(end_of_stream()); - if (sequence.present()) { + if (reqReturnIfBlocked && logData->version.get() < reqBegin) { + replyPromise.sendError(end_of_stream()); + if (reqSequence.present()) { auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + auto& sequenceData = trackerData.sequence_version[sequence + 1]; trackerData.lastUpdate = now(); if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(begin, reqOnlySpilled)); + sequenceData.send(std::make_pair(reqBegin, reqOnlySpilled)); } } return Void(); } - //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); + //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", reqBegin.epoch).detail("ReqBeginSeq", reqBegin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", reqTag1).detail("Tag2", reqTag2); // Wait until we have something to return that the caller doesn't already have - if (logData->version.get() < begin) { - wait(logData->version.whenAtLeast(begin)); + if (logData->version.get() < reqBegin) { + wait(logData->version.whenAtLeast(reqBegin)); wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); } - if (logData->locality != tagLocalitySatellite && tag.locality == tagLocalityLogRouter) { + if (logData->locality != tagLocalitySatellite && reqTag.locality == tagLocalityLogRouter) { wait(self->concurrentLogRouterReads.take()); state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); wait(delay(0.0, TaskPriority::Low)); } - if (begin <= logData->persistentDataDurableVersion && tag.locality != tagLocalityTxs && tag != txsTag) { + if (reqBegin <= logData->persistentDataDurableVersion && reqTag.locality != tagLocalityTxs && reqTag != txsTag) { // Reading spilled data will almost always imply that the storage server is >5s behind the rest // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up // slightly faster over keeping the rest of the cluster operating normally. @@ -1697,9 +1696,8 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, state double workStart = now(); - Version poppedVer = poppedVersion(logData, tag); - if (poppedVer > begin) { - // reply with an empty message and let the next reply start from poppedVer + Version poppedVer = poppedVersion(logData, reqTag); + if (poppedVer > reqBegin) { TLogPeekReply rep; rep.maxKnownVersion = logData->version.get(); rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; @@ -1707,13 +1705,11 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, rep.end = poppedVer; rep.onlySpilled = false; - // TODO: once the fake stream is replaced by ReplyPromiseStream, we can remove the code handling sequence - // requests. - if (sequence.present()) { + if (reqSequence.present()) { auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; + auto& sequenceData = trackerData.sequence_version[sequence + 1]; trackerData.lastUpdate = now(); - if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { replyPromise.sendError(operation_obsolete()); if (!sequenceData.isSet()) sequenceData.sendError(operation_obsolete()); @@ -1721,14 +1717,14 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, } if (sequenceData.isSet()) { if (sequenceData.getFuture().get().first != rep.end) { - TEST(true); // xz tlog peek second attempt ended at a different version + TEST(true); // tlog peek second attempt ended at a different version replyPromise.sendError(operation_obsolete()); return Void(); } } else { sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); } - rep.begin = begin; + rep.begin = reqBegin; } replyPromise.send(rep); @@ -1739,8 +1735,8 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, state bool onlySpilled = false; // grab messages from disk - //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - if (begin <= logData->persistentDataDurableVersion) { + //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", reqBegin.epoch).detail("ReqBeginSeq", reqBegin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", reqTag1).detail("Tag2", reqTag2); + if (reqBegin <= logData->persistentDataDurableVersion) { // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if // an initial attempt to read from disk results in insufficient data and the required data is no longer in @@ -1750,13 +1746,13 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, if (reqOnlySpilled) { endVersion = logData->persistentDataDurableVersion + 1; } else { - peekMessagesFromMemory(logData, tag, begin, messages2, endVersion); + peekMessagesFromMemory(logData, reqTag, reqBegin, messages2, endVersion); } - if (logData->shouldSpillByValue(tag)) { + if (logData->shouldSpillByValue(reqTag)) { RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, tag, begin), - persistTagMessagesKey(logData->logId, tag, logData->persistentDataDurableVersion + 1)), + KeyRangeRef(persistTagMessagesKey(logData->logId, reqTag, reqBegin), + persistTagMessagesKey(logData->logId, reqTag, logData->persistentDataDurableVersion + 1)), SERVER_KNOBS->DESIRED_TOTAL_BYTES, SERVER_KNOBS->DESIRED_TOTAL_BYTES)); @@ -1775,11 +1771,12 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, } else { // FIXME: Limit to approximately DESIRED_TOTATL_BYTES somehow. RangeResult kvrefs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessageRefsKey(logData->logId, tag, begin), - persistTagMessageRefsKey(logData->logId, tag, logData->persistentDataDurableVersion + 1)), + KeyRangeRef( + persistTagMessageRefsKey(logData->logId, reqTag, reqBegin), + persistTagMessageRefsKey(logData->logId, reqTag, logData->persistentDataDurableVersion + 1)), SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1)); - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); state std::vector> commitLocations; state bool earlyEnd = false; @@ -1796,7 +1793,7 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, earlyEnd = true; break; } - if (sd.version >= begin) { + if (sd.version >= reqBegin) { firstVersion = std::min(firstVersion, sd.version); const IDiskQueue::location end = sd.start.lo + sd.length; commitLocations.emplace_back(sd.start, end); @@ -1838,13 +1835,13 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, messages << VERSION_HEADER << entry.version; std::vector rawMessages = - wait(parseMessagesForTag(entry.messages, tag, logData->logRouterTags)); + wait(parseMessagesForTag(entry.messages, reqTag, logData->logRouterTags)); for (const StringRef& msg : rawMessages) { messages.serializeBytes(msg); DEBUG_TAGS_AND_MESSAGE("TLogPeekFromDisk", entry.version, msg) .detail("UID", self->dbgid) .detail("LogId", logData->logId) - .detail("PeekTag", tag); + .detail("PeekTag", reqTag); } lastRefMessageVersion = entry.version; @@ -1865,10 +1862,10 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, if (reqOnlySpilled) { endVersion = logData->persistentDataDurableVersion + 1; } else { - peekMessagesFromMemory(logData, tag, begin, messages, endVersion); + peekMessagesFromMemory(logData, reqTag, reqBegin, messages, endVersion); } - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); + //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); } TLogPeekReply reply; @@ -1878,12 +1875,12 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, reply.end = endVersion; reply.onlySpilled = onlySpilled; - //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("Tag", req.tag.toString()). - // detail("BeginVer", req.begin).detail("EndVer", reply.end). + //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("Tag", reqTag.toString()). + // detail("BeginVer", reqBegin).detail("EndVer", reply.end). // detail("MsgBytes", reply.messages.expectedSize()). - // detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); + // detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress()); - if (sequence.present()) { + if (reqSequence.present()) { auto& trackerData = logData->peekTracker[peekId]; trackerData.lastUpdate = now(); @@ -1905,11 +1902,11 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, trackerData.blockTime += blockT; trackerData.workTime += workT; - auto& sequenceData = trackerData.sequence_version[sequenceNum + 1]; - if (trackerData.sequence_version.size() && sequenceNum + 1 < trackerData.sequence_version.begin()->first) { + auto& sequenceData = trackerData.sequence_version[sequence + 1]; + if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { replyPromise.sendError(operation_obsolete()); if (!sequenceData.isSet()) { - // It would technically be more correct to .send({req.begin, req.onlySpilled}), as the next + // It would technically be more correct to .send({reqBegin, reqOnlySpilled}), as the next // request might still be in the window of active requests, but LogSystemPeekCursor will // throw away all future responses upon getting an operation_obsolete(), so computing a // response will probably be a waste of CPU. @@ -1920,21 +1917,20 @@ Future peekTLogAndSend(PromiseType replyPromise, TLogData* self, if (sequenceData.isSet()) { trackerData.duplicatePeeks++; if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // xz tlog peek second attempt ended at a different version (2) + TEST(true); // tlog peek second attempt ended at a different version (2) replyPromise.sendError(operation_obsolete()); return Void(); } } else { sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); } - reply.begin = begin; + reply.begin = reqBegin; } replyPromise.send(reply); return Void(); } - // This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Reference logData) { self->activePeekStreams++; @@ -1947,9 +1943,10 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref state Promise promise; state Future future(promise.getFuture()); try { - wait(req.reply.onReady() && peekTLogAndSend(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled)); + wait(req.reply.onReady() && + tLogPeekMessages(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled)); ASSERT(future.isReady()); - if(future.isError()) { + if (future.isError()) { throw future.getError(); } @@ -1976,343 +1973,6 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref } } -ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Reference logData) { - state BinaryWriter messages(Unversioned()); - state BinaryWriter messages2(Unversioned()); - state int sequence = -1; - state UID peekId; - state double queueStart = now(); - - if (req.tag.locality == tagLocalityTxs && req.tag.id >= logData->txsTags && logData->txsTags > 0) { - req.tag.id = req.tag.id % logData->txsTags; - } - - if (req.sequence.present()) { - try { - peekId = req.sequence.get().first; - sequence = req.sequence.get().second; - if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - logData->peekTracker.find(peekId) == logData->peekTracker.end()) { - throw operation_obsolete(); - } - auto& trackerData = logData->peekTracker[peekId]; - if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - trackerData.tag = req.tag; - trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled)); - } - auto seqBegin = trackerData.sequence_version.begin(); - // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. - while (trackerData.sequence_version.size() && - seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { - if (seqBegin->second.canBeSet()) { - seqBegin->second.sendError(operation_obsolete()); - } - trackerData.sequence_version.erase(seqBegin); - seqBegin = trackerData.sequence_version.begin(); - } - - if (trackerData.sequence_version.size() && sequence < seqBegin->first) { - throw operation_obsolete(); - } - - Future> fPrevPeekData = trackerData.sequence_version[sequence].getFuture(); - if (fPrevPeekData.isReady()) { - trackerData.unblockedPeeks++; - double t = now() - trackerData.lastUpdate; - if (t > trackerData.idleMax) - trackerData.idleMax = t; - trackerData.idleTime += t; - } - trackerData.lastUpdate = now(); - std::pair prevPeekData = wait(fPrevPeekData); - req.begin = std::max(prevPeekData.first, req.begin); - req.onlySpilled = prevPeekData.second; - wait(yield()); - } catch (Error& e) { - if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { - req.reply.sendError(e); - return Void(); - } else { - throw; - } - } - } - - state double blockStart = now(); - - if (req.returnIfBlocked && logData->version.get() < req.begin) { - req.reply.sendError(end_of_stream()); - if (req.sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; - trackerData.lastUpdate = now(); - if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); - } - } - return Void(); - } - - //TraceEvent("TLogPeekMessages0", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - // Wait until we have something to return that the caller doesn't already have - if (logData->version.get() < req.begin) { - wait(logData->version.whenAtLeast(req.begin)); - wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); - } - - if (logData->locality != tagLocalitySatellite && req.tag.locality == tagLocalityLogRouter) { - wait(self->concurrentLogRouterReads.take()); - state FlowLock::Releaser globalReleaser(self->concurrentLogRouterReads); - wait(delay(0.0, TaskPriority::Low)); - } - - if (req.begin <= logData->persistentDataDurableVersion && req.tag.locality != tagLocalityTxs && req.tag != txsTag) { - // Reading spilled data will almost always imply that the storage server is >5s behind the rest - // of the cluster. We shouldn't prioritize spending CPU on helping this server catch up - // slightly faster over keeping the rest of the cluster operating normally. - // txsTag is only ever peeked on recovery, and we would still wish to prioritize requests - // that impact recovery duration. - wait(delay(0, TaskPriority::TLogSpilledPeekReply)); - } - - state double workStart = now(); - - Version poppedVer = poppedVersion(logData, req.tag); - if (poppedVer > req.begin) { - TLogPeekReply rep; - rep.maxKnownVersion = logData->version.get(); - rep.minKnownCommittedVersion = logData->minKnownCommittedVersion; - rep.popped = poppedVer; - rep.end = poppedVer; - rep.onlySpilled = false; - - if (req.sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - auto& sequenceData = trackerData.sequence_version[sequence + 1]; - trackerData.lastUpdate = now(); - if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - req.reply.sendError(operation_obsolete()); - if (!sequenceData.isSet()) - sequenceData.sendError(operation_obsolete()); - return Void(); - } - if (sequenceData.isSet()) { - if (sequenceData.getFuture().get().first != rep.end) { - TEST(true); // tlog peek second attempt ended at a different version - req.reply.sendError(operation_obsolete()); - return Void(); - } - } else { - sequenceData.send(std::make_pair(rep.end, rep.onlySpilled)); - } - rep.begin = req.begin; - } - - req.reply.send(rep); - return Void(); - } - - state Version endVersion = logData->version.get() + 1; - state bool onlySpilled = false; - - // grab messages from disk - //TraceEvent("TLogPeekMessages", self->dbgid).detail("ReqBeginEpoch", req.begin.epoch).detail("ReqBeginSeq", req.begin.sequence).detail("Epoch", self->epoch()).detail("PersistentDataSeq", self->persistentDataSequence).detail("Tag1", req.tag1).detail("Tag2", req.tag2); - if (req.begin <= logData->persistentDataDurableVersion) { - // Just in case the durable version changes while we are waiting for the read, we grab this data from memory. We - // may or may not actually send it depending on whether we get enough data from disk. SOMEDAY: Only do this if - // an initial attempt to read from disk results in insufficient data and the required data is no longer in - // memory SOMEDAY: Should we only send part of the messages we collected, to actually limit the size of the - // result? - - if (req.onlySpilled) { - endVersion = logData->persistentDataDurableVersion + 1; - } else { - peekMessagesFromMemory(logData, req.tag, req.begin, messages2, endVersion); - } - - if (logData->shouldSpillByValue(req.tag)) { - RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, req.tag, req.begin), - persistTagMessagesKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)), - SERVER_KNOBS->DESIRED_TOTAL_BYTES, - SERVER_KNOBS->DESIRED_TOTAL_BYTES)); - - for (auto& kv : kvs) { - auto ver = decodeTagMessagesKey(kv.key); - messages << VERSION_HEADER << ver; - messages.serializeBytes(kv.value); - } - - if (kvs.expectedSize() >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { - endVersion = decodeTagMessagesKey(kvs.end()[-1].key) + 1; - onlySpilled = true; - } else { - messages.serializeBytes(messages2.toValue()); - } - } else { - // FIXME: Limit to approximately DESIRED_TOTATL_BYTES somehow. - RangeResult kvrefs = wait(self->persistentData->readRange( - KeyRangeRef( - persistTagMessageRefsKey(logData->logId, req.tag, req.begin), - persistTagMessageRefsKey(logData->logId, req.tag, logData->persistentDataDurableVersion + 1)), - SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1)); - - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); - - state std::vector> commitLocations; - state bool earlyEnd = false; - uint32_t mutationBytes = 0; - state uint64_t commitBytes = 0; - state Version firstVersion = std::numeric_limits::max(); - for (int i = 0; i < kvrefs.size() && i < SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK; i++) { - auto& kv = kvrefs[i]; - VectorRef spilledData; - BinaryReader r(kv.value, AssumeVersion(logData->protocolVersion)); - r >> spilledData; - for (const SpilledData& sd : spilledData) { - if (mutationBytes >= SERVER_KNOBS->DESIRED_TOTAL_BYTES) { - earlyEnd = true; - break; - } - if (sd.version >= req.begin) { - firstVersion = std::min(firstVersion, sd.version); - const IDiskQueue::location end = sd.start.lo + sd.length; - commitLocations.emplace_back(sd.start, end); - // This isn't perfect, because we aren't accounting for page boundaries, but should be - // close enough. - commitBytes += sd.length; - mutationBytes += sd.mutationBytes; - } - } - if (earlyEnd) - break; - } - earlyEnd = earlyEnd || (kvrefs.size() >= SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_BATCHES_PER_PEEK + 1); - wait(self->peekMemoryLimiter.take(TaskPriority::TLogSpilledPeekReply, commitBytes)); - state FlowLock::Releaser memoryReservation(self->peekMemoryLimiter, commitBytes); - state std::vector>> messageReads; - messageReads.reserve(commitLocations.size()); - for (const auto& pair : commitLocations) { - messageReads.push_back(self->rawPersistentQueue->read(pair.first, pair.second, CheckHashes::True)); - } - commitLocations.clear(); - wait(waitForAll(messageReads)); - - state Version lastRefMessageVersion = 0; - state int index = 0; - loop { - if (index >= messageReads.size()) - break; - Standalone queueEntryData = messageReads[index].get(); - uint8_t valid; - const uint32_t length = *(uint32_t*)queueEntryData.begin(); - queueEntryData = queueEntryData.substr(4, queueEntryData.size() - 4); - BinaryReader rd(queueEntryData, IncludeVersion()); - state TLogQueueEntry entry; - rd >> entry >> valid; - ASSERT(valid == 0x01); - ASSERT(length + sizeof(valid) == queueEntryData.size()); - - messages << VERSION_HEADER << entry.version; - - std::vector rawMessages = - wait(parseMessagesForTag(entry.messages, req.tag, logData->logRouterTags)); - for (const StringRef& msg : rawMessages) { - messages.serializeBytes(msg); - DEBUG_TAGS_AND_MESSAGE("TLogPeekFromDisk", entry.version, msg) - .detail("UID", self->dbgid) - .detail("LogId", logData->logId) - .detail("PeekTag", req.tag); - } - - lastRefMessageVersion = entry.version; - index++; - } - - messageReads.clear(); - memoryReservation.release(); - - if (earlyEnd) { - endVersion = lastRefMessageVersion + 1; - onlySpilled = true; - } else { - messages.serializeBytes(messages2.toValue()); - } - } - } else { - if (req.onlySpilled) { - endVersion = logData->persistentDataDurableVersion + 1; - } else { - peekMessagesFromMemory(logData, req.tag, req.begin, messages, endVersion); - } - - //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()).detail("MessageBytes", messages.getLength()).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowSeq", self->sequence.getNextSequence()); - } - - TLogPeekReply reply; - reply.maxKnownVersion = logData->version.get(); - reply.minKnownCommittedVersion = logData->minKnownCommittedVersion; - reply.messages = messages.toValue(); - reply.end = endVersion; - reply.onlySpilled = onlySpilled; - - //TraceEvent("TlogPeek", self->dbgid).detail("LogId", logData->logId).detail("Tag", req.tag.toString()). - // detail("BeginVer", req.begin).detail("EndVer", reply.end). - // detail("MsgBytes", reply.messages.expectedSize()). - // detail("ForAddress", req.reply.getEndpoint().getPrimaryAddress()); - - if (req.sequence.present()) { - auto& trackerData = logData->peekTracker[peekId]; - trackerData.lastUpdate = now(); - - double queueT = blockStart - queueStart; - double blockT = workStart - blockStart; - double workT = now() - workStart; - - trackerData.totalPeeks++; - trackerData.replyBytes += reply.messages.size(); - - if (queueT > trackerData.queueMax) - trackerData.queueMax = queueT; - if (blockT > trackerData.blockMax) - trackerData.blockMax = blockT; - if (workT > trackerData.workMax) - trackerData.workMax = workT; - - trackerData.queueTime += queueT; - trackerData.blockTime += blockT; - trackerData.workTime += workT; - - auto& sequenceData = trackerData.sequence_version[sequence + 1]; - if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - req.reply.sendError(operation_obsolete()); - if (!sequenceData.isSet()) { - // It would technically be more correct to .send({req.begin, req.onlySpilled}), as the next - // request might still be in the window of active requests, but LogSystemPeekCursor will - // throw away all future responses upon getting an operation_obsolete(), so computing a - // response will probably be a waste of CPU. - sequenceData.sendError(operation_obsolete()); - } - return Void(); - } - if (sequenceData.isSet()) { - trackerData.duplicatePeeks++; - if (sequenceData.getFuture().get().first != reply.end) { - TEST(true); // tlog peek second attempt ended at a different version (2) - req.reply.sendError(operation_obsolete()); - return Void(); - } - } else { - sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); - } - reply.begin = req.begin; - } - - req.reply.send(reply); - return Void(); -} - - ACTOR Future doQueueCommit(TLogData* self, Reference logData, std::vector> missingFinalCommit) { @@ -2814,7 +2474,8 @@ ACTOR Future serveTLogInterface(TLogData* self, logData->addActor.send(tLogPeekStream(self, req, logData)); } when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) { - logData->addActor.send(tLogPeekMessages(self, req, logData)); + logData->addActor.send(tLogPeekMessages( + req.reply, self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); } when(TLogPopRequest req = waitNext(tli.popMessages.getFuture())) { logData->addActor.send(tLogPop(self, req, logData)); diff --git a/flow/Knobs.cpp b/flow/Knobs.cpp index 25c6e32b80..7a8f1e24f7 100644 --- a/flow/Knobs.cpp +++ b/flow/Knobs.cpp @@ -184,7 +184,7 @@ void FlowKnobs::initialize(Randomize randomize, IsSimulated isSimulated) { init( FAST_NETWORK_LATENCY, 800e-6 ); init( SLOW_NETWORK_LATENCY, 100e-3 ); init( MAX_CLOGGING_LATENCY, 0 ); if( randomize && BUGGIFY ) MAX_CLOGGING_LATENCY = 0.1 * deterministicRandom()->random01(); - init( MAX_BUGGIFIED_DELAY, 0 ); // if( randomize && BUGGIFY ) MAX_BUGGIFIED_DELAY = 0.2 * deterministicRandom()->random01(); + init( MAX_BUGGIFIED_DELAY, 0 ); if( randomize && BUGGIFY ) MAX_BUGGIFIED_DELAY = 0.2 * deterministicRandom()->random01(); init( SIM_CONNECT_ERROR_MODE, deterministicRandom()->randomInt(0,3) ); //Tracefiles From ae2268f9f2bc1d0c89c1a65e54f336e1f000f114 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Sat, 31 Jul 2021 22:07:43 -0700 Subject: [PATCH 22/63] 200k simulation: check stream sequence; delay in GetMore loop --- fdbserver/LogSystemPeekCursor.actor.cpp | 12 ++++++++++-- fdbserver/OldTLogServer_4_6.actor.cpp | 1 + fdbserver/OldTLogServer_6_0.actor.cpp | 1 + fdbserver/OldTLogServer_6_2.actor.cpp | 1 + fdbserver/TLogServer.actor.cpp | 1 + 5 files changed, 14 insertions(+), 2 deletions(-) diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index c210f89f29..f719c7c83a 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -333,6 +333,7 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T loop { try { + state Version expectedBegin = self->messageVersion.version; state Future fPeekReply = self->peekReplyStream.present() ? map(waitAndForward(self->peekReplyStream.get().getFuture()), [](const TLogPeekStreamReply& r) { return r.rep; }) @@ -350,7 +351,11 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T self->interf->get().interf().peekStreamMessages.getEndpoint().getPrimaryAddress(), fPeekReply) : Never())) { + if (res.begin.get() != expectedBegin) { + throw operation_obsolete(); + } updateCursorWithReply(self, res); + expectedBegin = res.end; TraceEvent("SPC_GetMoreB", self->randomID) .detail("Has", self->hasMessage()) .detail("End", res.end) @@ -364,8 +369,11 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T } } catch (Error& e) { TraceEvent(SevDebug, "SPC_GetMoreB_Error", self->randomID).detail("Error", e.what()); + + self->peekReplyStream.reset(); if (e.code() == error_code_connection_failed || e.code() == error_code_operation_obsolete) { - self->peekReplyStream.reset(); + // NOTE: delay in order to avoid the endless retry loop block other tasks + wait(delay(0)); } else if (e.code() == error_code_end_of_stream) { self->end.reset(self->messageVersion.version); return Void(); @@ -418,7 +426,7 @@ Future ILogSystem::ServerPeekCursor::getMore(TaskPriority taskID) { return Void(); if (!more.isValid() || more.isReady()) { // TODO: remove locality check when log router support streaming peek - if (false && usePeekStream && tag.locality >= 0) { + if (usePeekStream && tag.locality >= 0) { more = serverPeekStreamGetMore(this, taskID); } else if (parallelGetMore || onlySpilled || futureResults.size()) { more = serverPeekParallelGetMore(this, taskID); diff --git a/fdbserver/OldTLogServer_4_6.actor.cpp b/fdbserver/OldTLogServer_4_6.actor.cpp index 35d142b9f7..ea0f6ba22e 100644 --- a/fdbserver/OldTLogServer_4_6.actor.cpp +++ b/fdbserver/OldTLogServer_4_6.actor.cpp @@ -1157,6 +1157,7 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref } reply.rep = future.get(); + reply.rep.begin = begin; req.reply.send(reply); begin = reply.rep.end; onlySpilled = reply.rep.onlySpilled; diff --git a/fdbserver/OldTLogServer_6_0.actor.cpp b/fdbserver/OldTLogServer_6_0.actor.cpp index 5c27581b2c..fa08f0e3f3 100644 --- a/fdbserver/OldTLogServer_6_0.actor.cpp +++ b/fdbserver/OldTLogServer_6_0.actor.cpp @@ -1469,6 +1469,7 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref } reply.rep = future.get(); + reply.rep.begin = begin; req.reply.send(reply); begin = reply.rep.end; onlySpilled = reply.rep.onlySpilled; diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index 81025d93de..181960cec9 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -1898,6 +1898,7 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref } reply.rep = future.get(); + reply.rep.begin = begin; req.reply.send(reply); begin = reply.rep.end; onlySpilled = reply.rep.onlySpilled; diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index aa2afd5406..746064ce91 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1951,6 +1951,7 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref } reply.rep = future.get(); + reply.rep.begin = begin; req.reply.send(reply); begin = reply.rep.end; onlySpilled = reply.rep.onlySpilled; From fd74a16f35e2e3a9fe21f4917e165bdbaecdcc7f Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Mon, 2 Aug 2021 14:24:20 -0700 Subject: [PATCH 23/63] format code --- fdbclient/ServerKnobs.cpp | 4 ++-- fdbclient/ServerKnobs.h | 4 ++-- fdbserver/LogRouter.actor.cpp | 9 +++++---- fdbserver/LogSystemPeekCursor.actor.cpp | 7 ++++--- fdbserver/TLogInterface.h | 4 ++-- flow/error_definitions.h | 1 - 6 files changed, 15 insertions(+), 14 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 36d599ba03..290452012a 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -64,8 +64,8 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( TLOG_MESSAGE_BLOCK_BYTES, 10e6 ); init( TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR, double(TLOG_MESSAGE_BLOCK_BYTES) / (TLOG_MESSAGE_BLOCK_BYTES - MAX_MESSAGE_SIZE) ); //1.0121466709838096006362758832473 init( PEEK_TRACKER_EXPIRATION_TIME, 600 ); if( randomize && BUGGIFY ) PEEK_TRACKER_EXPIRATION_TIME = deterministicRandom()->coinflip() ? 0.1 : 120; - init( PEEK_USEING_STREAMING, true ); - init( PARALLEL_GET_MORE_REQUESTS, 32 ); if( randomize && BUGGIFY ) PARALLEL_GET_MORE_REQUESTS = 2; + init( PEEK_USEING_STREAMING, true ); + init( PARALLEL_GET_MORE_REQUESTS, 32 ); if( randomize && BUGGIFY ) PARALLEL_GET_MORE_REQUESTS = 2; init( MULTI_CURSOR_PRE_FETCH_LIMIT, 10 ); init( MAX_QUEUE_COMMIT_BYTES, 15e6 ); if( randomize && BUGGIFY ) MAX_QUEUE_COMMIT_BYTES = 5000; init( DESIRED_OUTSTANDING_MESSAGES, 5000 ); if( randomize && BUGGIFY ) DESIRED_OUTSTANDING_MESSAGES = deterministicRandom()->randomInt(0,100); diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index ea7ca9ca3a..23b3049668 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -41,8 +41,8 @@ public: // often, so that versions always advance smoothly // TLogs - bool PEEK_USEING_STREAMING; - double TLOG_TIMEOUT; // tlog OR commit proxy failure - master's reaction time + bool PEEK_USEING_STREAMING; + double TLOG_TIMEOUT; // tlog OR commit proxy failure - master's reaction time double TLOG_SLOW_REJOIN_WARN_TIMEOUT_SECS; // Warns if a tlog takes too long to rejoin double RECOVERY_TLOG_SMART_QUORUM_DELAY; // smaller might be better for bug amplification double TLOG_STORAGE_MIN_UPDATE_INTERVAL; diff --git a/fdbserver/LogRouter.actor.cpp b/fdbserver/LogRouter.actor.cpp index 06806e649b..aae2edb05b 100644 --- a/fdbserver/LogRouter.actor.cpp +++ b/fdbserver/LogRouter.actor.cpp @@ -441,10 +441,11 @@ Version poppedVersion(LogRouterData* self, Tag tag) { return tagData->popped; } +// TODO: enable streaming peek log from log router // This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover -ACTOR Future logRouterPeekStream(LogRouterData* self, TLogPeekStreamRequest req) { - return Void(); -} +// ACTOR Future logRouterPeekStream(LogRouterData* self, TLogPeekStreamRequest req) { +// return Void(); +// } ACTOR Future logRouterPeekMessages(LogRouterData* self, TLogPeekRequest req) { state BinaryWriter messages(Unversioned()); @@ -652,7 +653,7 @@ ACTOR Future logRouterCore(TLogInterface interf, } when(TLogPeekStreamRequest req = waitNext(interf.peekStreamMessages.getFuture())) { // addActor.send(logRouterPeekStream(&logRouterData, req)); - // FIXME: temporarily disable streaming peek from LogRouter + // FIXME: currently LogRouter doesn't support streaming peek request TraceEvent(SevError, "LogRouterPeekStream", logRouterData.dbgid) .detail("Token", interf.peekStreamMessages.getEndpoint().token); req.reply.sendError(operation_failed()); diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 88faa618d8..953f5debdd 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -52,7 +52,8 @@ ILogSystem::ServerPeekCursor::ServerPeekCursor(ReferencerandomUniqueID()), returnIfBlocked(returnIfBlocked), onlySpilled(false), parallelGetMore(parallelGetMore), sequence(0), lastReset(0), - resetCheck(Void()), slowReplies(0), fastReplies(0), unknownReplies(0), usePeekStream(SERVER_KNOBS->PEEK_USEING_STREAMING) { + resetCheck(Void()), slowReplies(0), fastReplies(0), unknownReplies(0), + usePeekStream(SERVER_KNOBS->PEEK_USEING_STREAMING) { this->results.maxKnownVersion = 0; this->results.minKnownCommittedVersion = 0; TraceEvent("SPC_Starting", randomID) @@ -361,8 +362,8 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T .detail("End", res.end) .detail("Popped", res.popped.present() ? res.popped.get() : 0); - // NOTE: delay is needed here since TLog need to be scheduled to response if there are TLog and SS - // on the same machine + // NOTE: delay is necessary here since ReplyPromiseStream delivers reply on high priority. Here we + // change the priority to the intended one. wait(delay(0, taskID)); return Void(); } diff --git a/fdbserver/TLogInterface.h b/fdbserver/TLogInterface.h index 9f147189fe..322c7ddb35 100644 --- a/fdbserver/TLogInterface.h +++ b/fdbserver/TLogInterface.h @@ -83,8 +83,8 @@ struct TLogInterface { streams.push_back(disablePopRequest.getReceiver()); streams.push_back(enablePopRequest.getReceiver()); streams.push_back(snapRequest.getReceiver()); - streams.push_back(peekStreamMessages.getReceiver(TaskPriority::TLogPeek)); - FlowTransport::transport().addEndpoints(streams); + streams.push_back(peekStreamMessages.getReceiver(TaskPriority::TLogPeek)); + FlowTransport::transport().addEndpoints(streams); } template diff --git a/flow/error_definitions.h b/flow/error_definitions.h index bfe57bfd06..b69801cfd7 100755 --- a/flow/error_definitions.h +++ b/flow/error_definitions.h @@ -33,7 +33,6 @@ // clang-format off ERROR( success, 0, "Success" ) ERROR( end_of_stream, 1, "End of stream" ) -ERROR( no_action_needed, 2, "No action needed" ) ERROR( operation_failed, 1000, "Operation failed") ERROR( wrong_shard_server, 1001, "Shard is not available from this server") ERROR( operation_obsolete, 1002, "Operation result no longer necessary") From 9986d2b0b6acfb4557fc71b33d87cf65ff78a87e Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Mon, 2 Aug 2021 22:33:17 -0700 Subject: [PATCH 24/63] change log severity --- fdbserver/LogSystemPeekCursor.actor.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 77ee0296f4..36f62cd6a5 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -56,11 +56,10 @@ ILogSystem::ServerPeekCursor::ServerPeekCursor(Referenceresults.maxKnownVersion = 0; this->results.minKnownCommittedVersion = 0; - TraceEvent("SPC_Starting", randomID) + TraceEvent(SevDebug, "SPC_Starting", randomID) .detail("Tag", tag.toString()) .detail("Begin", begin) - .detail("End", end) - .backtrace(); + .detail("End", end); } ILogSystem::ServerPeekCursor::ServerPeekCursor(TLogPeekReply const& results, @@ -357,7 +356,7 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T } updateCursorWithReply(self, res); expectedBegin = res.end; - TraceEvent("SPC_GetMoreB", self->randomID) + TraceEvent(SevDebug, "SPC_GetMoreB", self->randomID) .detail("Has", self->hasMessage()) .detail("End", res.end) .detail("Popped", res.popped.present() ? res.popped.get() : 0); From 80a5120df8e0bb80f6a8b56c4cdcf4a568af68e8 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Thu, 5 Aug 2021 19:51:17 -0700 Subject: [PATCH 25/63] support LogRouter peek from TLog --- fdbserver/LogSystemPeekCursor.actor.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 36f62cd6a5..935da90b8b 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -368,13 +368,13 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T } } } catch (Error& e) { - TraceEvent(SevDebug, "SPC_GetMoreB_Error", self->randomID).detail("Error", e.what()); - - self->peekReplyStream.reset(); + TraceEvent(SevDebug, "SPC_GetMoreB_Error").error(e, true); if (e.code() == error_code_connection_failed || e.code() == error_code_operation_obsolete) { // NOTE: delay in order to avoid the endless retry loop block other tasks + self->peekReplyStream.reset(); wait(delay(0)); } else if (e.code() == error_code_end_of_stream) { + self->peekReplyStream.reset(); self->end.reset(self->messageVersion.version); return Void(); } else { @@ -425,8 +425,8 @@ Future ILogSystem::ServerPeekCursor::getMore(TaskPriority taskID) { if (hasMessage() && !parallelGetMore) return Void(); if (!more.isValid() || more.isReady()) { - // TODO: remove locality check when log router support streaming peek - if (usePeekStream && tag.locality >= 0) { + // TODO: add tagLocalityRemoteLog when log router support streaming peek + if (usePeekStream && (tag.locality >= 0 || tag.locality == tagLocalityLogRouter)) { more = serverPeekStreamGetMore(this, taskID); } else if (parallelGetMore || onlySpilled || futureResults.size()) { more = serverPeekParallelGetMore(this, taskID); From 2263626cdcc59a5e984ff48f62e54cb5b7630d4e Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Sat, 7 Aug 2021 09:53:22 -0700 Subject: [PATCH 26/63] 200k test clean: enable remote Log pull from LogRouter --- fdbserver/LogRouter.actor.cpp | 125 ++++++++++++++++-------- fdbserver/LogSystemPeekCursor.actor.cpp | 2 +- fdbserver/OldTLogServer_4_6.actor.cpp | 35 +++---- fdbserver/OldTLogServer_6_0.actor.cpp | 37 +++---- fdbserver/OldTLogServer_6_2.actor.cpp | 11 +-- fdbserver/TLogServer.actor.cpp | 7 +- 6 files changed, 119 insertions(+), 98 deletions(-) diff --git a/fdbserver/LogRouter.actor.cpp b/fdbserver/LogRouter.actor.cpp index aae2edb05b..f26ea539af 100644 --- a/fdbserver/LogRouter.actor.cpp +++ b/fdbserver/LogRouter.actor.cpp @@ -441,33 +441,35 @@ Version poppedVersion(LogRouterData* self, Tag tag) { return tagData->popped; } -// TODO: enable streaming peek log from log router -// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover -// ACTOR Future logRouterPeekStream(LogRouterData* self, TLogPeekStreamRequest req) { -// return Void(); -// } - -ACTOR Future logRouterPeekMessages(LogRouterData* self, TLogPeekRequest req) { +// Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request +ACTOR template +Future logRouterPeekMessages(PromiseType replyPromise, + LogRouterData* self, + Version reqBegin, + Tag reqTag, + bool reqReturnIfBlocked = false, + bool reqOnlySpilled = false, + Optional> reqSequence = Optional>()) { state BinaryWriter messages(Unversioned()); state int sequence = -1; state UID peekId; - if (req.sequence.present()) { + if (reqSequence.present()) { try { - peekId = req.sequence.get().first; - sequence = req.sequence.get().second; + peekId = reqSequence.get().first; + sequence = reqSequence.get().second; if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - self->peekTracker.find(peekId) == self->peekTracker.end()) { + self->peekTracker.find(peekId) == self->peekTracker.end()) { throw operation_obsolete(); } auto& trackerData = self->peekTracker[peekId]; if (sequence == 0 && trackerData.sequence_version.find(0) == trackerData.sequence_version.end()) { - trackerData.sequence_version[0].send(std::make_pair(req.begin, req.onlySpilled)); + trackerData.sequence_version[0].send(std::make_pair(reqBegin, reqOnlySpilled)); } auto seqBegin = trackerData.sequence_version.begin(); // The peek cursor and this comparison need to agree about the maximum number of in-flight requests. while (trackerData.sequence_version.size() && - seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { if (seqBegin->second.canBeSet()) { seqBegin->second.sendError(operation_obsolete()); } @@ -481,12 +483,12 @@ ACTOR Future logRouterPeekMessages(LogRouterData* self, TLogPeekRequest re trackerData.lastUpdate = now(); std::pair prevPeekData = wait(trackerData.sequence_version[sequence].getFuture()); - req.begin = prevPeekData.first; - req.onlySpilled = prevPeekData.second; + reqBegin = prevPeekData.first; + reqOnlySpilled = prevPeekData.second; wait(yield()); } catch (Error& e) { if (e.code() == error_code_timed_out || e.code() == error_code_operation_obsolete) { - req.reply.sendError(e); + replyPromise.sendError(e); return Void(); } else { throw; @@ -494,62 +496,62 @@ ACTOR Future logRouterPeekMessages(LogRouterData* self, TLogPeekRequest re } } - //TraceEvent("LogRouterPeek1", self->dbgid).detail("From", req.reply.getEndpoint().getPrimaryAddress()).detail("Ver", self->version.get()).detail("Begin", req.begin); - if (req.returnIfBlocked && self->version.get() < req.begin) { + //TraceEvent("LogRouterPeek1", self->dbgid).detail("From", replyPromise.getEndpoint().getPrimaryAddress()).detail("Ver", self->version.get()).detail("Begin", reqBegin); + if (reqReturnIfBlocked && self->version.get() < reqBegin) { //TraceEvent("LogRouterPeek2", self->dbgid); - req.reply.sendError(end_of_stream()); - if (req.sequence.present()) { + replyPromise.sendError(end_of_stream()); + if (reqSequence.present()) { auto& trackerData = self->peekTracker[peekId]; auto& sequenceData = trackerData.sequence_version[sequence + 1]; if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); + sequenceData.send(std::make_pair(reqBegin, reqOnlySpilled)); } } return Void(); } - if (self->version.get() < req.begin) { - wait(self->version.whenAtLeast(req.begin)); + if (self->version.get() < reqBegin) { + wait(self->version.whenAtLeast(reqBegin)); wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); } - Version poppedVer = poppedVersion(self, req.tag); + Version poppedVer = poppedVersion(self, reqTag); - if (poppedVer > req.begin || req.begin < self->startVersion) { + if (poppedVer > reqBegin || reqBegin < self->startVersion) { // This should only happen if a packet is sent multiple times and the reply is not needed. // Since we are using popped differently, do not send a reply. TraceEvent(SevWarnAlways, "LogRouterPeekPopped", self->dbgid) - .detail("Begin", req.begin) - .detail("Popped", poppedVer) - .detail("Start", self->startVersion); - req.reply.send(Never()); - if (req.sequence.present()) { + .detail("Begin", reqBegin) + .detail("Popped", poppedVer) + .detail("Start", self->startVersion); + replyPromise.send(Never()); + if (reqSequence.present()) { auto& trackerData = self->peekTracker[peekId]; auto& sequenceData = trackerData.sequence_version[sequence + 1]; if (!sequenceData.isSet()) { - sequenceData.send(std::make_pair(req.begin, req.onlySpilled)); + sequenceData.send(std::make_pair(reqBegin, reqOnlySpilled)); } } return Void(); } Version endVersion = self->version.get() + 1; - peekMessagesFromMemory(self, req.tag, req.begin, messages, endVersion); + peekMessagesFromMemory(self, reqTag, reqBegin, messages, endVersion); TLogPeekReply reply; reply.maxKnownVersion = self->version.get(); reply.minKnownCommittedVersion = self->poppedVersion; - reply.messages = messages.toValue(); + reply.messages = StringRef(reply.arena, messages.toValue()); reply.popped = self->minPopped.get() >= self->startVersion ? self->minPopped.get() : 0; reply.end = endVersion; reply.onlySpilled = false; - if (req.sequence.present()) { + if (reqSequence.present()) { auto& trackerData = self->peekTracker[peekId]; trackerData.lastUpdate = now(); auto& sequenceData = trackerData.sequence_version[sequence + 1]; if (trackerData.sequence_version.size() && sequence + 1 < trackerData.sequence_version.begin()->first) { - req.reply.sendError(operation_obsolete()); + replyPromise.sendError(operation_obsolete()); if (!sequenceData.isSet()) sequenceData.sendError(operation_obsolete()); return Void(); @@ -557,20 +559,58 @@ ACTOR Future logRouterPeekMessages(LogRouterData* self, TLogPeekRequest re if (sequenceData.isSet()) { if (sequenceData.getFuture().get().first != reply.end) { TEST(true); // tlog peek second attempt ended at a different version - req.reply.sendError(operation_obsolete()); + replyPromise.sendError(operation_obsolete()); return Void(); } } else { sequenceData.send(std::make_pair(reply.end, reply.onlySpilled)); } - reply.begin = req.begin; + reply.begin = reqBegin; } - req.reply.send(reply); + replyPromise.send(reply); //TraceEvent("LogRouterPeek4", self->dbgid); return Void(); } +// TODO: enable streaming peek log from log router +// This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover +ACTOR Future logRouterPeekStream(LogRouterData* self, TLogPeekStreamRequest req) { + self->activePeekStreams++; + + state Version begin = req.begin; + state bool onlySpilled = false; + req.reply.setByteLimit(std::min(SERVER_KNOBS->MAXIMUM_PEEK_BYTES, req.limitBytes)); + loop { + state TLogPeekStreamReply reply; + state Promise promise; + state Future future(promise.getFuture()); + try { + wait(req.reply.onReady() && store(reply.rep, future) && + logRouterPeekMessages(promise, self, begin, req.tag, req.returnIfBlocked, onlySpilled)); + + reply.rep.begin = begin; + req.reply.send(reply); + begin = reply.rep.end; + onlySpilled = reply.rep.onlySpilled; + if (reply.rep.end > self->version.get()) { + wait(delay(SERVER_KNOBS->TLOG_PEEK_DELAY, g_network->getCurrentTask())); + } else { + wait(delay(0, g_network->getCurrentTask())); + } + } catch (Error& e) { + self->activePeekStreams--; + TraceEvent(SevDebug, "TLogPeekStreamEnd", self->dbgid).error(e, true); + + if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { + req.reply.sendError(e); + return Void(); + } else { + throw; + } + } + } +} ACTOR Future cleanupPeekTrackers(LogRouterData* self) { loop { @@ -649,14 +689,15 @@ ACTOR Future logRouterCore(TLogInterface interf, logRouterData.logSystem->set(ILogSystem::fromServerDBInfo(logRouterData.dbgid, db->get(), true)); } when(TLogPeekRequest req = waitNext(interf.peekMessages.getFuture())) { - addActor.send(logRouterPeekMessages(&logRouterData, req)); + addActor.send(logRouterPeekMessages( + req.reply, &logRouterData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); } when(TLogPeekStreamRequest req = waitNext(interf.peekStreamMessages.getFuture())) { - // addActor.send(logRouterPeekStream(&logRouterData, req)); // FIXME: currently LogRouter doesn't support streaming peek request - TraceEvent(SevError, "LogRouterPeekStream", logRouterData.dbgid) + TraceEvent(SevDebug, "LogRouterPeekStream", logRouterData.dbgid) .detail("Token", interf.peekStreamMessages.getEndpoint().token); - req.reply.sendError(operation_failed()); + addActor.send(logRouterPeekStream(&logRouterData, req)); + // req.reply.sendError(operation_failed()); } when(TLogPopRequest req = waitNext(interf.popMessages.getFuture())) { // Request from remote tLog to pop data from LR diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 935da90b8b..26cb4b6eec 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -426,7 +426,7 @@ Future ILogSystem::ServerPeekCursor::getMore(TaskPriority taskID) { return Void(); if (!more.isValid() || more.isReady()) { // TODO: add tagLocalityRemoteLog when log router support streaming peek - if (usePeekStream && (tag.locality >= 0 || tag.locality == tagLocalityLogRouter)) { + if (usePeekStream && (tag.locality >= 0 || tag.locality == tagLocalityLogRouter || tag.locality == tagLocalityRemoteLog)) { more = serverPeekStreamGetMore(this, taskID); } else if (parallelGetMore || onlySpilled || futureResults.size()) { more = serverPeekParallelGetMore(this, taskID); diff --git a/fdbserver/OldTLogServer_4_6.actor.cpp b/fdbserver/OldTLogServer_4_6.actor.cpp index bf0bf1639a..f50744cc1a 100644 --- a/fdbserver/OldTLogServer_4_6.actor.cpp +++ b/fdbserver/OldTLogServer_4_6.actor.cpp @@ -976,13 +976,13 @@ void peekMessagesFromMemory(Reference self, // Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request ACTOR template Future tLogPeekMessages(PromiseType replyPromise, - TLogData* self, - Reference logData, - Version reqBegin, - Tag reqTag, - bool reqReturnIfBlocked = false, - bool reqOnlySpilled = false, - Optional> reqSequence = Optional>()) { + TLogData* self, + Reference logData, + Version reqBegin, + Tag reqTag, + bool reqReturnIfBlocked = false, + bool reqOnlySpilled = false, + Optional> reqSequence = Optional>()) { state BinaryWriter messages(Unversioned()); state BinaryWriter messages2(Unversioned()); state int sequence = -1; @@ -994,7 +994,7 @@ Future tLogPeekMessages(PromiseType replyPromise, peekId = reqSequence.get().first; sequence = reqSequence.get().second; if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - self->peekTracker.find(peekId) == self->peekTracker.end()) { + self->peekTracker.find(peekId) == self->peekTracker.end()) { throw operation_obsolete(); } if (sequence > 0) { @@ -1075,10 +1075,10 @@ Future tLogPeekMessages(PromiseType replyPromise, peekMessagesFromMemory(logData, reqTag, reqBegin, messages2, endVersion); RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, oldTag, reqBegin), - persistTagMessagesKey(logData->logId, oldTag, logData->persistentDataDurableVersion + 1)), - SERVER_KNOBS->DESIRED_TOTAL_BYTES, - SERVER_KNOBS->DESIRED_TOTAL_BYTES)); + KeyRangeRef(persistTagMessagesKey(logData->logId, oldTag, reqBegin), + persistTagMessagesKey(logData->logId, oldTag, logData->persistentDataDurableVersion + 1)), + SERVER_KNOBS->DESIRED_TOTAL_BYTES, + SERVER_KNOBS->DESIRED_TOTAL_BYTES)); //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().getPrimaryAddress()).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); @@ -1148,14 +1148,9 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref state Promise promise; state Future future(promise.getFuture()); try { - wait(req.reply.onReady() && - tLogPeekMessages(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled)); - ASSERT(future.isReady()); - if (future.isError()) { - throw future.getError(); - } + wait(req.reply.onReady() && store(reply.rep, future) && + tLogPeekMessages(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled)); - reply.rep = future.get(); reply.rep.begin = begin; req.reply.send(reply); begin = reply.rep.end; @@ -1344,7 +1339,7 @@ ACTOR Future serveTLogInterface(TLogData* self, loop choose { when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) { logData->addActor.send(tLogPeekMessages( - req.reply, self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); + req.reply, self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); } when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { TraceEvent(SevDebug, "TLogPeekStream", logData->logId) diff --git a/fdbserver/OldTLogServer_6_0.actor.cpp b/fdbserver/OldTLogServer_6_0.actor.cpp index d6d2daf7c0..5d77b4e12f 100644 --- a/fdbserver/OldTLogServer_6_0.actor.cpp +++ b/fdbserver/OldTLogServer_6_0.actor.cpp @@ -1209,13 +1209,13 @@ void peekMessagesFromMemory(Reference self, // Common logics to peek TLog and create TLogPeekReply that serves both streaming peek or normal peek request ACTOR template Future tLogPeekMessages(PromiseType replyPromise, - TLogData* self, - Reference logData, - Version reqBegin, - Tag reqTag, - bool reqReturnIfBlocked = false, - bool reqOnlySpilled = false, - Optional> reqSequence = Optional>()) { + TLogData* self, + Reference logData, + Version reqBegin, + Tag reqTag, + bool reqReturnIfBlocked = false, + bool reqOnlySpilled = false, + Optional> reqSequence = Optional>()) { state BinaryWriter messages(Unversioned()); state BinaryWriter messages2(Unversioned()); state int sequence = -1; @@ -1231,7 +1231,7 @@ Future tLogPeekMessages(PromiseType replyPromise, peekId = reqSequence.get().first; sequence = reqSequence.get().second; if (sequence >= SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS && - logData->peekTracker.find(peekId) == logData->peekTracker.end()) { + logData->peekTracker.find(peekId) == logData->peekTracker.end()) { throw operation_obsolete(); } auto& trackerData = logData->peekTracker[peekId]; @@ -1241,7 +1241,7 @@ Future tLogPeekMessages(PromiseType replyPromise, } auto seqBegin = trackerData.sequence_version.begin(); while (trackerData.sequence_version.size() && - seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { + seqBegin->first <= sequence - SERVER_KNOBS->PARALLEL_GET_MORE_REQUESTS) { if (seqBegin->second.canBeSet()) { seqBegin->second.sendError(operation_obsolete()); } @@ -1369,10 +1369,10 @@ Future tLogPeekMessages(PromiseType replyPromise, } RangeResult kvs = wait(self->persistentData->readRange( - KeyRangeRef(persistTagMessagesKey(logData->logId, reqTag, reqBegin), - persistTagMessagesKey(logData->logId, reqTag, logData->persistentDataDurableVersion + 1)), - SERVER_KNOBS->DESIRED_TOTAL_BYTES, - SERVER_KNOBS->DESIRED_TOTAL_BYTES)); + KeyRangeRef(persistTagMessagesKey(logData->logId, reqTag, reqBegin), + persistTagMessagesKey(logData->logId, reqTag, logData->persistentDataDurableVersion + 1)), + SERVER_KNOBS->DESIRED_TOTAL_BYTES, + SERVER_KNOBS->DESIRED_TOTAL_BYTES)); //TraceEvent("TLogPeekResults", self->dbgid).detail("ForAddress", replyPromise.getEndpoint().address).detail("Tag1Results", s1).detail("Tag2Results", s2).detail("Tag1ResultsLim", kv1.size()).detail("Tag2ResultsLim", kv2.size()).detail("Tag1ResultsLast", kv1.size() ? kv1[0].key : "").detail("Tag2ResultsLast", kv2.size() ? kv2[0].key : "").detail("Limited", limited).detail("NextEpoch", next_pos.epoch).detail("NextSeq", next_pos.sequence).detail("NowEpoch", self->epoch()).detail("NowSeq", self->sequence.getNextSequence()); @@ -1460,14 +1460,9 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref state Promise promise; state Future future(promise.getFuture()); try { - wait(req.reply.onReady() && - tLogPeekMessages(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled)); - ASSERT(future.isReady()); - if (future.isError()) { - throw future.getError(); - } + wait(req.reply.onReady() && store(reply.rep, future) && + tLogPeekMessages(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled)); - reply.rep = future.get(); reply.rep.begin = begin; req.reply.send(reply); begin = reply.rep.end; @@ -1986,7 +1981,7 @@ ACTOR Future serveTLogInterface(TLogData* self, } when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) { logData->addActor.send(tLogPeekMessages( - req.reply, self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); + req.reply, self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); } when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { TraceEvent(SevDebug, "TLogPeekStream", logData->logId) diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index 9419b4cd9e..687ea0e638 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -1889,14 +1889,9 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref state Promise promise; state Future future(promise.getFuture()); try { - wait(req.reply.onReady() && - tLogPeekMessages(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled)); - ASSERT(future.isReady()); - if (future.isError()) { - throw future.getError(); - } + wait(req.reply.onReady() && store(reply.rep, future) && + tLogPeekMessages(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled)); - reply.rep = future.get(); reply.rep.begin = begin; req.reply.send(reply); begin = reply.rep.end; @@ -2431,7 +2426,7 @@ ACTOR Future serveTLogInterface(TLogData* self, } when(TLogPeekRequest req = waitNext(tli.peekMessages.getFuture())) { logData->addActor.send(tLogPeekMessages( - req.reply, self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); + req.reply, self, logData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); } when(TLogPeekStreamRequest req = waitNext(tli.peekStreamMessages.getFuture())) { TraceEvent(SevDebug, "TLogPeekStream", logData->logId) diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index b20d9f41cf..fea695645c 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1942,14 +1942,9 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref state Promise promise; state Future future(promise.getFuture()); try { - wait(req.reply.onReady() && + wait(req.reply.onReady() && store(reply.rep, future) && tLogPeekMessages(promise, self, logData, begin, req.tag, req.returnIfBlocked, onlySpilled)); - ASSERT(future.isReady()); - if (future.isError()) { - throw future.getError(); - } - reply.rep = future.get(); reply.rep.begin = begin; req.reply.send(reply); begin = reply.rep.end; From 618c72f7e2027de1eb467609d572fd335bbd0f71 Mon Sep 17 00:00:00 2001 From: Chaoguang Lin Date: Wed, 11 Aug 2021 01:07:36 +0000 Subject: [PATCH 27/63] Draft to use template to rewrite ThrottleApi --- fdbcli/ThrottleCommand.actor.cpp | 40 +++++++------- fdbcli/fdbcli.actor.cpp | 2 +- fdbclient/CMakeLists.txt | 2 +- fdbclient/CommitProxyInterface.h | 2 +- fdbclient/IClientApi.h | 4 ++ fdbclient/NativeAPI.actor.cpp | 4 ++ fdbclient/NativeAPI.actor.h | 8 ++- fdbclient/ReadYourWrites.h | 2 + fdbclient/StorageServerInterface.h | 2 +- fdbclient/TagThrottle.actor.cpp | 39 +++++++------- .../{TagThrottle.h => TagThrottle.actor.h} | 52 +++++++++++++++++-- fdbserver/Ratekeeper.actor.cpp | 2 +- fdbserver/workloads/TagThrottleApi.actor.cpp | 7 +-- .../workloads/WriteTagThrottling.actor.cpp | 5 +- flow/ThreadHelper.actor.h | 6 +++ 15 files changed, 123 insertions(+), 54 deletions(-) rename fdbclient/{TagThrottle.h => TagThrottle.actor.h} (82%) diff --git a/fdbcli/ThrottleCommand.actor.cpp b/fdbcli/ThrottleCommand.actor.cpp index 7692c17b69..89a0cf6c97 100644 --- a/fdbcli/ThrottleCommand.actor.cpp +++ b/fdbcli/ThrottleCommand.actor.cpp @@ -21,7 +21,7 @@ #include "fdbcli/fdbcli.actor.h" #include "fdbclient/IClientApi.h" -#include "fdbclient/TagThrottle.h" +#include "fdbclient/TagThrottle.actor.h" #include "fdbclient/Knobs.h" #include "fdbclient/SystemData.h" #include "fdbclient/CommitTransaction.h" @@ -258,26 +258,26 @@ ACTOR Future unthrottleTags(Reference db, } } -ACTOR Future enableAuto(Reference db, bool enabled) { - state Reference tr = db->createTransaction(); +// ACTOR Future enableAuto(Reference db, bool enabled) { +// state Reference tr = db->createTransaction(); - loop { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - try { - Optional value = wait(safeThreadFutureToFuture(tr->get(tagThrottleAutoEnabledKey))); - if (!value.present() || (enabled && value.get() != LiteralStringRef("1")) || - (!enabled && value.get() != LiteralStringRef("0"))) { - tr->set(tagThrottleAutoEnabledKey, LiteralStringRef(enabled ? "1" : "0")); - signalThrottleChange(tr); +// loop { +// tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); +// try { +// Optional value = wait(safeThreadFutureToFuture(tr->get(tagThrottleAutoEnabledKey))); +// if (!value.present() || (enabled && value.get() != LiteralStringRef("1")) || +// (!enabled && value.get() != LiteralStringRef("0"))) { +// tr->set(tagThrottleAutoEnabledKey, LiteralStringRef(enabled ? "1" : "0")); +// signalThrottleChange(tr); - wait(safeThreadFutureToFuture(tr->commit())); - } - return Void(); - } catch (Error& e) { - wait(safeThreadFutureToFuture(tr->onError(e))); - } - } -} +// wait(safeThreadFutureToFuture(tr->commit())); +// } +// return Void(); +// } catch (Error& e) { +// wait(safeThreadFutureToFuture(tr->onError(e))); +// } +// } +// } ACTOR Future unthrottleMatchingThrottles(Reference db, KeyRef beginKey, @@ -626,7 +626,7 @@ ACTOR Future throttleCommandActor(Reference db, std::vector using FutureT = ThreadFuture; }; // An interface that represents a connection to a cluster made by a client @@ -115,6 +117,8 @@ public: virtual ThreadFuture forceRecoveryWithDataLoss(const StringRef& dcid) = 0; // Management API, create snapshot virtual ThreadFuture createSnapshot(const StringRef& uid, const StringRef& snapshot_command) = 0; + + using TransactionT = ITransaction; }; // An interface that presents the top-level FDB client API as exposed through the C bindings diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 0a6a474249..c2c8ecd6d5 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -1773,6 +1773,10 @@ Database Database::createDatabase(std::string connFileName, return Database::createDatabase(rccf, apiVersion, internal, clientLocality); } +Reference Database::createTransaction() { + return makeReference(*this); +} + Reference DatabaseContext::getWatchMetadata(KeyRef key) const { const auto it = watchMap.find(key); if (it == watchMap.end()) diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h index 3636b30c46..4b18c72376 100644 --- a/fdbclient/NativeAPI.actor.h +++ b/fdbclient/NativeAPI.actor.h @@ -76,7 +76,9 @@ struct NetworkOptions { NetworkOptions(); }; -class Database { +class ReadYourWritesTransaction; // workaround cyclic dependency + +class Database : public ThreadSafeReferenceCounted { public: enum { API_VERSION_LATEST = -1 }; @@ -109,6 +111,9 @@ public: const UniqueOrderedOptionList& getTransactionDefaults() const; + using TransactionT = ReadYourWritesTransaction; + Reference createTransaction(); + private: Reference db; }; @@ -169,7 +174,6 @@ private: void clear(); }; -class ReadYourWritesTransaction; // workaround cyclic dependency struct TransactionInfo { Optional debugID; TaskPriority taskID; diff --git a/fdbclient/ReadYourWrites.h b/fdbclient/ReadYourWrites.h index 53431e00ed..e24b02b13a 100644 --- a/fdbclient/ReadYourWrites.h +++ b/fdbclient/ReadYourWrites.h @@ -175,6 +175,8 @@ public: void setSpecialKeySpaceErrorMsg(const std::string& msg) { specialKeySpaceErrorMsg = msg; } Transaction& getTransaction() { return tr; } + template using FutureT = Future; + private: friend class RYWImpl; diff --git a/fdbclient/StorageServerInterface.h b/fdbclient/StorageServerInterface.h index 0873b9b6b9..ccae14168e 100644 --- a/fdbclient/StorageServerInterface.h +++ b/fdbclient/StorageServerInterface.h @@ -30,7 +30,7 @@ #include "fdbrpc/Stats.h" #include "fdbrpc/TimedRequest.h" #include "fdbrpc/TSSComparison.h" -#include "fdbclient/TagThrottle.h" +#include "fdbclient/TagThrottle.actor.h" #include "flow/UnitTest.h" // Dead code, removed in the next protocol version diff --git a/fdbclient/TagThrottle.actor.cpp b/fdbclient/TagThrottle.actor.cpp index 76adbb5431..f6d803e8cf 100644 --- a/fdbclient/TagThrottle.actor.cpp +++ b/fdbclient/TagThrottle.actor.cpp @@ -18,10 +18,12 @@ * limitations under the License. */ -#include "fdbclient/TagThrottle.h" +#include "fdbclient/TagThrottle.actor.h" #include "fdbclient/CommitProxyInterface.h" #include "fdbclient/DatabaseContext.h" +#include "flow/FastRef.h" +#include "flow/ThreadHelper.actor.h" #include "flow/actorcompiler.h" // has to be last include void TagSet::addTag(TransactionTagRef tag) { @@ -406,23 +408,24 @@ Future expire(Database db) { db, tagThrottleKeys.begin, tagThrottleKeys.end, Optional(), true); } -ACTOR Future enableAuto(Database db, bool enabled) { - state Transaction tr(db); +// ACTOR template +// Future enableAuto(Reference db, bool enabled) { +// state Reference tr = db->createTransaction(); - loop { - try { - Optional value = wait(tr.get(tagThrottleAutoEnabledKey)); - if (!value.present() || (enabled && value.get() != LiteralStringRef("1")) || - (!enabled && value.get() != LiteralStringRef("0"))) { - tr.set(tagThrottleAutoEnabledKey, LiteralStringRef(enabled ? "1" : "0")); - signalThrottleChange(tr); +// loop { +// try { +// Optional value = wait(safeThreadFutureToFuture(tr->get(tagThrottleAutoEnabledKey))); +// if (!value.present() || (enabled && value.get() != LiteralStringRef("1")) || +// (!enabled && value.get() != LiteralStringRef("0"))) { +// tr->set(tagThrottleAutoEnabledKey, LiteralStringRef(enabled ? "1" : "0")); +// signalThrottleChange(tr); - wait(tr.commit()); - } - return Void(); - } catch (Error& e) { - wait(tr.onError(e)); - } - } -} +// wait(safeThreadFutureToFuture(tr->commit())); +// } +// return Void(); +// } catch (Error& e) { +// wait(safeThreadFutureToFuture(tr->onError(e))); +// } +// } +// } } // namespace ThrottleApi diff --git a/fdbclient/TagThrottle.h b/fdbclient/TagThrottle.actor.h similarity index 82% rename from fdbclient/TagThrottle.h rename to fdbclient/TagThrottle.actor.h index 333f841f36..2f44858fcb 100644 --- a/fdbclient/TagThrottle.h +++ b/fdbclient/TagThrottle.actor.h @@ -1,5 +1,5 @@ /* - * TagThrottle.h + * TagThrottle.actor.h * * This source file is part of the FoundationDB open source project * @@ -18,15 +18,23 @@ * limitations under the License. */ -#ifndef FDBCLIENT_TAG_THROTTLE_H -#define FDBCLIENT_TAG_THROTTLE_H +#if defined(NO_INTELLISENSE) && !defined(FDBCLIENT_TAG_THROTTLE_ACTOR_G_H) +#define FDBCLIENT_TAG_THROTTLE_ACTOR_G_H +#include "fdbclient/TagThrottle.actor.g.h" +#elif !defined(FDBCLIENT_TAG_THROTTLE_ACTOR_H) +#define FDBCLIENT_TAG_THROTTLE_ACTOR_H #pragma once #include "flow/Error.h" #include "flow/flow.h" #include "flow/network.h" +#include "flow/ThreadHelper.actor.h" +#include "fdbclient/FDBOptions.g.h" #include "fdbclient/FDBTypes.h" +#include "fdbclient/CommitTransaction.h" +// #include "fdbclient/SystemData.h" +#include "flow/actorcompiler.h" // This must be the last #include. #include @@ -228,6 +236,15 @@ struct ClientTrCommitCostEstimation { } }; +// Keys to view and control tag throttling +extern const KeyRangeRef tagThrottleKeys; +extern const KeyRef tagThrottleKeysPrefix; +extern const KeyRef tagThrottleAutoKeysPrefix; +extern const KeyRef tagThrottleSignalKey; +extern const KeyRef tagThrottleAutoEnabledKey; +extern const KeyRef tagThrottleLimitKey; +extern const KeyRef tagThrottleCountKey; + namespace ThrottleApi { Future> getThrottledTags(Database const& db, int const& limit, @@ -251,7 +268,34 @@ Future unthrottleTags(Database const& db, Future unthrottleAll(Database db, Optional throttleType, Optional priority); Future expire(Database db); -Future enableAuto(Database const& db, bool const& enabled); +template +void signalThrottleChange(Reference tr) { + tr->atomicOp( + tagThrottleSignalKey, LiteralStringRef("XXXXXXXXXX\x00\x00\x00\x00"), MutationRef::SetVersionstampedValue); +} + +ACTOR template +Future enableAuto(Reference db, bool enabled) { + state Reference tr = db->createTransaction(); + + loop { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + try { + Optional value = wait(safeThreadFutureToFuture(tr->get(tagThrottleAutoEnabledKey))); + if (!value.present() || (enabled && value.get() != LiteralStringRef("1")) || + (!enabled && value.get() != LiteralStringRef("0"))) { + tr->set(tagThrottleAutoEnabledKey, LiteralStringRef(enabled ? "1" : "0")); + signalThrottleChange(tr); + + wait(safeThreadFutureToFuture(tr->commit())); + } + return Void(); + } catch (Error& e) { + wait(safeThreadFutureToFuture(tr->onError(e))); + } + } +} + }; // namespace ThrottleApi template diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index 7fd2acaee7..54bd4f9ca3 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -24,7 +24,7 @@ #include "fdbrpc/Smoother.h" #include "fdbrpc/simulator.h" #include "fdbclient/ReadYourWrites.h" -#include "fdbclient/TagThrottle.h" +#include "fdbclient/TagThrottle.actor.h" #include "fdbserver/Knobs.h" #include "fdbserver/DataDistribution.actor.h" #include "fdbserver/RatekeeperInterface.h" diff --git a/fdbserver/workloads/TagThrottleApi.actor.cpp b/fdbserver/workloads/TagThrottleApi.actor.cpp index 235279ca96..005322007e 100644 --- a/fdbserver/workloads/TagThrottleApi.actor.cpp +++ b/fdbserver/workloads/TagThrottleApi.actor.cpp @@ -19,7 +19,7 @@ */ #include "fdbclient/NativeAPI.actor.h" -#include "fdbclient/TagThrottle.h" +#include "fdbclient/TagThrottle.actor.h" #include "fdbserver/Knobs.h" #include "fdbserver/TesterInterface.actor.h" #include "fdbserver/workloads/workloads.actor.h" @@ -227,15 +227,16 @@ struct TagThrottleApiWorkload : TestWorkload { } ACTOR Future enableAutoThrottling(TagThrottleApiWorkload* self, Database cx) { + state Reference db = Reference::addRef(&cx); if (deterministicRandom()->coinflip()) { - wait(ThrottleApi::enableAuto(cx, true)); + wait(ThrottleApi::enableAuto(db, true)); self->autoThrottleEnabled = true; if (deterministicRandom()->coinflip()) { bool unthrottled = wait(ThrottleApi::unthrottleAll(cx, TagThrottleType::AUTO, Optional())); } } else { - wait(ThrottleApi::enableAuto(cx, false)); + wait(ThrottleApi::enableAuto(db, false)); self->autoThrottleEnabled = false; } diff --git a/fdbserver/workloads/WriteTagThrottling.actor.cpp b/fdbserver/workloads/WriteTagThrottling.actor.cpp index 2d28ba5958..e188eab9ae 100644 --- a/fdbserver/workloads/WriteTagThrottling.actor.cpp +++ b/fdbserver/workloads/WriteTagThrottling.actor.cpp @@ -23,7 +23,7 @@ #include "fdbserver/workloads/BulkSetup.actor.h" #include "fdbserver/WorkerInterface.actor.h" #include "fdbclient/NativeAPI.actor.h" -#include "fdbclient/TagThrottle.h" +#include "fdbclient/TagThrottle.actor.h" #include "flow/actorcompiler.h" // This must be the last #include. constexpr int SAMPLE_SIZE = 10000; @@ -94,13 +94,14 @@ struct WriteTagThrottlingWorkload : KVWorkload { std::string description() const override { return WriteTagThrottlingWorkload::NAME; } ACTOR static Future _setup(Database cx, WriteTagThrottlingWorkload* self) { + state Reference db = Reference::addRef(&cx); ASSERT(CLIENT_KNOBS->MAX_TAGS_PER_TRANSACTION >= MIN_TAGS_PER_TRANSACTION && CLIENT_KNOBS->MAX_TRANSACTION_TAG_LENGTH >= MIN_TRANSACTION_TAG_LENGTH); if (self->populateData) { wait(bulkSetup(cx, self, self->keyCount, Promise())); } if (self->clientId == 0) { - wait(ThrottleApi::enableAuto(cx, true)); + wait(ThrottleApi::enableAuto(db, true)); } return Void(); } diff --git a/flow/ThreadHelper.actor.h b/flow/ThreadHelper.actor.h index 5a5fd549c1..b1e40604f1 100644 --- a/flow/ThreadHelper.actor.h +++ b/flow/ThreadHelper.actor.h @@ -627,6 +627,12 @@ Future safeThreadFutureToFuture(ThreadFuture threadFuture) { return threadFuture.get(); } +template +Future safeThreadFutureToFuture(Future future) { + // do nothing + return future; +} + // Helper actor. Do not use directly! namespace internal_thread_helper { From bf0d0598dc97fc5696de7d3f8e14bfc05f8a55e8 Mon Sep 17 00:00:00 2001 From: Chaoguang Lin Date: Wed, 11 Aug 2021 23:59:25 +0000 Subject: [PATCH 28/63] Refactor all ThrottleAPI functions using template and remove duplicate copies --- fdbcli/ThrottleCommand.actor.cpp | 340 +----------------- fdbclient/IClientApi.h | 2 +- fdbclient/NativeAPI.actor.h | 2 +- fdbclient/TagThrottle.actor.cpp | 317 ---------------- fdbclient/TagThrottle.actor.h | 330 +++++++++++++++-- fdbserver/Ratekeeper.actor.cpp | 11 +- fdbserver/workloads/TagThrottleApi.actor.cpp | 20 +- .../workloads/WriteTagThrottling.actor.cpp | 5 +- flow/ThreadHelper.actor.h | 1 + 9 files changed, 340 insertions(+), 688 deletions(-) diff --git a/fdbcli/ThrottleCommand.actor.cpp b/fdbcli/ThrottleCommand.actor.cpp index 89a0cf6c97..d09cf263e6 100644 --- a/fdbcli/ThrottleCommand.actor.cpp +++ b/fdbcli/ThrottleCommand.actor.cpp @@ -32,334 +32,6 @@ #include "flow/genericactors.actor.h" #include "flow/actorcompiler.h" // This must be the last #include. -namespace { - -// Helper functions copied from TagThrottle.actor.cpp -// The only difference is transactions are changed to go through MultiversionTransaction, -// instead of the native Transaction(i.e., RYWTransaction) - -ACTOR Future getValidAutoEnabled(Reference tr) { - state bool result; - loop { - Optional value = wait(safeThreadFutureToFuture(tr->get(tagThrottleAutoEnabledKey))); - if (!value.present()) { - tr->reset(); - wait(delay(CLIENT_KNOBS->DEFAULT_BACKOFF)); - continue; - } else if (value.get() == LiteralStringRef("1")) { - result = true; - } else if (value.get() == LiteralStringRef("0")) { - result = false; - } else { - TraceEvent(SevWarnAlways, "InvalidAutoTagThrottlingValue").detail("Value", value.get()); - tr->reset(); - wait(delay(CLIENT_KNOBS->DEFAULT_BACKOFF)); - continue; - } - return result; - }; -} - -ACTOR Future> getThrottledTags(Reference db, - int limit, - bool containsRecommend = false) { - state Reference tr = db->createTransaction(); - state bool reportAuto = containsRecommend; - loop { - tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS); - try { - if (!containsRecommend) { - wait(store(reportAuto, getValidAutoEnabled(tr))); - } - state ThreadFuture f = tr->getRange( - reportAuto ? tagThrottleKeys : KeyRangeRef(tagThrottleKeysPrefix, tagThrottleAutoKeysPrefix), limit); - RangeResult throttles = wait(safeThreadFutureToFuture(f)); - std::vector results; - for (auto throttle : throttles) { - results.push_back(TagThrottleInfo(TagThrottleKey::fromKey(throttle.key), - TagThrottleValue::fromValue(throttle.value))); - } - return results; - } catch (Error& e) { - wait(safeThreadFutureToFuture(tr->onError(e))); - } - } -} - -ACTOR Future> getRecommendedTags(Reference db, int limit) { - state Reference tr = db->createTransaction(); - loop { - tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS); - try { - bool enableAuto = wait(getValidAutoEnabled(tr)); - if (enableAuto) { - return std::vector(); - } - state ThreadFuture f = - tr->getRange(KeyRangeRef(tagThrottleAutoKeysPrefix, tagThrottleKeys.end), limit); - RangeResult throttles = wait(safeThreadFutureToFuture(f)); - std::vector results; - for (auto throttle : throttles) { - results.push_back(TagThrottleInfo(TagThrottleKey::fromKey(throttle.key), - TagThrottleValue::fromValue(throttle.value))); - } - return results; - } catch (Error& e) { - wait(safeThreadFutureToFuture(tr->onError(e))); - } - } -} - -ACTOR Future updateThrottleCount(Reference tr, int64_t delta) { - state ThreadFuture> countVal = tr->get(tagThrottleCountKey); - state ThreadFuture> limitVal = tr->get(tagThrottleLimitKey); - - wait(success(safeThreadFutureToFuture(countVal)) && success(safeThreadFutureToFuture(limitVal))); - - int64_t count = 0; - int64_t limit = 0; - - if (countVal.get().present()) { - BinaryReader reader(countVal.get().get(), Unversioned()); - reader >> count; - } - - if (limitVal.get().present()) { - BinaryReader reader(limitVal.get().get(), Unversioned()); - reader >> limit; - } - - count += delta; - - if (count > limit) { - throw too_many_tag_throttles(); - } - - BinaryWriter writer(Unversioned()); - writer << count; - - tr->set(tagThrottleCountKey, writer.toValue()); - return Void(); -} - -void signalThrottleChange(Reference tr) { - tr->atomicOp( - tagThrottleSignalKey, LiteralStringRef("XXXXXXXXXX\x00\x00\x00\x00"), MutationRef::SetVersionstampedValue); -} - -ACTOR Future throttleTags(Reference db, - TagSet tags, - double tpsRate, - double initialDuration, - TagThrottleType throttleType, - TransactionPriority priority, - Optional expirationTime = Optional(), - Optional reason = Optional()) { - state Reference tr = db->createTransaction(); - state Key key = TagThrottleKey(tags, throttleType, priority).toKey(); - - ASSERT(initialDuration > 0); - - if (throttleType == TagThrottleType::MANUAL) { - reason = TagThrottledReason::MANUAL; - } - TagThrottleValue throttle(tpsRate, - expirationTime.present() ? expirationTime.get() : 0, - initialDuration, - reason.present() ? reason.get() : TagThrottledReason::UNSET); - BinaryWriter wr(IncludeVersion(ProtocolVersion::withTagThrottleValueReason())); - wr << throttle; - state Value value = wr.toValue(); - - loop { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - try { - if (throttleType == TagThrottleType::MANUAL) { - Optional oldThrottle = wait(safeThreadFutureToFuture(tr->get(key))); - if (!oldThrottle.present()) { - wait(updateThrottleCount(tr, 1)); - } - } - - tr->set(key, value); - - if (throttleType == TagThrottleType::MANUAL) { - signalThrottleChange(tr); - } - - wait(safeThreadFutureToFuture(tr->commit())); - return Void(); - } catch (Error& e) { - wait(safeThreadFutureToFuture(tr->onError(e))); - } - } -} - -ACTOR Future unthrottleTags(Reference db, - TagSet tags, - Optional throttleType, - Optional priority) { - state Reference tr = db->createTransaction(); - - state std::vector keys; - for (auto p : allTransactionPriorities) { - if (!priority.present() || priority.get() == p) { - if (!throttleType.present() || throttleType.get() == TagThrottleType::AUTO) { - keys.push_back(TagThrottleKey(tags, TagThrottleType::AUTO, p).toKey()); - } - if (!throttleType.present() || throttleType.get() == TagThrottleType::MANUAL) { - keys.push_back(TagThrottleKey(tags, TagThrottleType::MANUAL, p).toKey()); - } - } - } - - state bool removed = false; - - loop { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - try { - state std::vector>> values; - values.reserve(keys.size()); - for (auto key : keys) { - values.push_back(safeThreadFutureToFuture(tr->get(key))); - } - - wait(waitForAll(values)); - - int delta = 0; - for (int i = 0; i < values.size(); ++i) { - if (values[i].get().present()) { - if (TagThrottleKey::fromKey(keys[i]).throttleType == TagThrottleType::MANUAL) { - delta -= 1; - } - - tr->clear(keys[i]); - - // Report that we are removing this tag if we ever see it present. - // This protects us from getting confused if the transaction is maybe committed. - // It's ok if someone else actually ends up removing this tag at the same time - // and we aren't the ones to actually do it. - removed = true; - } - } - - if (delta != 0) { - wait(updateThrottleCount(tr, delta)); - } - if (removed) { - signalThrottleChange(tr); - wait(safeThreadFutureToFuture(tr->commit())); - } - - return removed; - } catch (Error& e) { - wait(safeThreadFutureToFuture(tr->onError(e))); - } - } -} - -// ACTOR Future enableAuto(Reference db, bool enabled) { -// state Reference tr = db->createTransaction(); - -// loop { -// tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); -// try { -// Optional value = wait(safeThreadFutureToFuture(tr->get(tagThrottleAutoEnabledKey))); -// if (!value.present() || (enabled && value.get() != LiteralStringRef("1")) || -// (!enabled && value.get() != LiteralStringRef("0"))) { -// tr->set(tagThrottleAutoEnabledKey, LiteralStringRef(enabled ? "1" : "0")); -// signalThrottleChange(tr); - -// wait(safeThreadFutureToFuture(tr->commit())); -// } -// return Void(); -// } catch (Error& e) { -// wait(safeThreadFutureToFuture(tr->onError(e))); -// } -// } -// } - -ACTOR Future unthrottleMatchingThrottles(Reference db, - KeyRef beginKey, - KeyRef endKey, - Optional priority, - bool onlyExpiredThrottles) { - state Reference tr = db->createTransaction(); - - state KeySelector begin = firstGreaterOrEqual(beginKey); - state KeySelector end = firstGreaterOrEqual(endKey); - - state bool removed = false; - - loop { - tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); - try { - // holds memory of the RangeResult - state ThreadFuture f = tr->getRange(begin, end, 1000); - state RangeResult tags = wait(safeThreadFutureToFuture(f)); - state uint64_t unthrottledTags = 0; - uint64_t manualUnthrottledTags = 0; - for (auto tag : tags) { - if (onlyExpiredThrottles) { - double expirationTime = TagThrottleValue::fromValue(tag.value).expirationTime; - if (expirationTime == 0 || expirationTime > now()) { - continue; - } - } - - TagThrottleKey key = TagThrottleKey::fromKey(tag.key); - if (priority.present() && key.priority != priority.get()) { - continue; - } - - if (key.throttleType == TagThrottleType::MANUAL) { - ++manualUnthrottledTags; - } - - removed = true; - tr->clear(tag.key); - unthrottledTags++; - } - - if (manualUnthrottledTags > 0) { - wait(updateThrottleCount(tr, -manualUnthrottledTags)); - } - - if (unthrottledTags > 0) { - signalThrottleChange(tr); - } - - wait(safeThreadFutureToFuture(tr->commit())); - - if (!tags.more) { - return removed; - } - - ASSERT(tags.size() > 0); - begin = KeySelector(firstGreaterThan(tags[tags.size() - 1].key), tags.arena()); - } catch (Error& e) { - wait(safeThreadFutureToFuture(tr->onError(e))); - } - } -} - -Future unthrottleAll(Reference db, - Optional tagThrottleType, - Optional priority) { - KeyRef begin = tagThrottleKeys.begin; - KeyRef end = tagThrottleKeys.end; - - if (tagThrottleType.present() && tagThrottleType == TagThrottleType::AUTO) { - begin = tagThrottleAutoKeysPrefix; - } else if (tagThrottleType.present() && tagThrottleType == TagThrottleType::MANUAL) { - end = tagThrottleAutoKeysPrefix; - } - - return unthrottleMatchingThrottles(db, begin, end, priority, false); -} - -} // namespace - namespace fdb_cli { ACTOR Future throttleCommandActor(Reference db, std::vector tokens) { @@ -403,11 +75,11 @@ ACTOR Future throttleCommandActor(Reference db, std::vector tags; if (reportThrottled && reportRecommended) { - wait(store(tags, getThrottledTags(db, throttleListLimit, true))); + wait(store(tags, ThrottleApi::getThrottledTags(db, throttleListLimit, true))); } else if (reportThrottled) { - wait(store(tags, getThrottledTags(db, throttleListLimit))); + wait(store(tags, ThrottleApi::getThrottledTags(db, throttleListLimit))); } else if (reportRecommended) { - wait(store(tags, getRecommendedTags(db, throttleListLimit))); + wait(store(tags, ThrottleApi::getRecommendedTags(db, throttleListLimit))); } bool anyLogged = false; @@ -509,7 +181,7 @@ ACTOR Future throttleCommandActor(Reference db, std::vector throttleCommandActor(Reference db, std::vector 0) { - bool success = wait(unthrottleTags(db, tags, throttleType, priority)); + bool success = wait(ThrottleApi::unthrottleTags(db, tags, throttleType, priority)); if (success) { printf("Unthrottled tag `%s'%s\n", tokens[3].toString().c_str(), priorityString.c_str()); } else { @@ -596,7 +268,7 @@ ACTOR Future throttleCommandActor(Reference db, std::vector using FutureT = ThreadFuture; + template using FutureT = ThreadFuture; }; // An interface that represents a connection to a cluster made by a client diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h index 4b18c72376..02139aa73b 100644 --- a/fdbclient/NativeAPI.actor.h +++ b/fdbclient/NativeAPI.actor.h @@ -78,7 +78,7 @@ struct NetworkOptions { class ReadYourWritesTransaction; // workaround cyclic dependency -class Database : public ThreadSafeReferenceCounted { +class Database final : public ThreadSafeReferenceCounted { public: enum { API_VERSION_LATEST = -1 }; diff --git a/fdbclient/TagThrottle.actor.cpp b/fdbclient/TagThrottle.actor.cpp index f6d803e8cf..c874c0d7c3 100644 --- a/fdbclient/TagThrottle.actor.cpp +++ b/fdbclient/TagThrottle.actor.cpp @@ -112,320 +112,3 @@ TagThrottleValue TagThrottleValue::fromValue(const ValueRef& value) { reader >> throttleValue; return throttleValue; } - -namespace ThrottleApi { -ACTOR Future getValidAutoEnabled(Transaction* tr, Database db) { - state bool result; - loop { - Optional value = wait(tr->get(tagThrottleAutoEnabledKey)); - if (!value.present()) { - tr->reset(); - wait(delay(CLIENT_KNOBS->DEFAULT_BACKOFF)); - continue; - } else if (value.get() == LiteralStringRef("1")) { - result = true; - } else if (value.get() == LiteralStringRef("0")) { - result = false; - } else { - TraceEvent(SevWarnAlways, "InvalidAutoTagThrottlingValue", db->dbId).detail("Value", value.get()); - tr->reset(); - wait(delay(CLIENT_KNOBS->DEFAULT_BACKOFF)); - continue; - } - return result; - }; -} - -void signalThrottleChange(Transaction& tr) { - tr.atomicOp( - tagThrottleSignalKey, LiteralStringRef("XXXXXXXXXX\x00\x00\x00\x00"), MutationRef::SetVersionstampedValue); -} - -ACTOR Future updateThrottleCount(Transaction* tr, int64_t delta) { - state Future> countVal = tr->get(tagThrottleCountKey); - state Future> limitVal = tr->get(tagThrottleLimitKey); - - wait(success(countVal) && success(limitVal)); - - int64_t count = 0; - int64_t limit = 0; - - if (countVal.get().present()) { - BinaryReader reader(countVal.get().get(), Unversioned()); - reader >> count; - } - - if (limitVal.get().present()) { - BinaryReader reader(limitVal.get().get(), Unversioned()); - reader >> limit; - } - - count += delta; - - if (count > limit) { - throw too_many_tag_throttles(); - } - - BinaryWriter writer(Unversioned()); - writer << count; - - tr->set(tagThrottleCountKey, writer.toValue()); - return Void(); -} - -ACTOR Future> getThrottledTags(Database db, int limit, bool containsRecommend) { - state Transaction tr(db); - state bool reportAuto = containsRecommend; - loop { - try { - if (!containsRecommend) { - wait(store(reportAuto, getValidAutoEnabled(&tr, db))); - } - RangeResult throttles = wait(tr.getRange( - reportAuto ? tagThrottleKeys : KeyRangeRef(tagThrottleKeysPrefix, tagThrottleAutoKeysPrefix), limit)); - std::vector results; - for (auto throttle : throttles) { - results.push_back(TagThrottleInfo(TagThrottleKey::fromKey(throttle.key), - TagThrottleValue::fromValue(throttle.value))); - } - return results; - } catch (Error& e) { - wait(tr.onError(e)); - } - } -} - -ACTOR Future> getRecommendedTags(Database db, int limit) { - state Transaction tr(db); - loop { - try { - bool enableAuto = wait(getValidAutoEnabled(&tr, db)); - if (enableAuto) { - return std::vector(); - } - - RangeResult throttles = - wait(tr.getRange(KeyRangeRef(tagThrottleAutoKeysPrefix, tagThrottleKeys.end), limit)); - std::vector results; - for (auto throttle : throttles) { - results.push_back(TagThrottleInfo(TagThrottleKey::fromKey(throttle.key), - TagThrottleValue::fromValue(throttle.value))); - } - return results; - } catch (Error& e) { - wait(tr.onError(e)); - } - } -} - -ACTOR Future throttleTags(Database db, - TagSet tags, - double tpsRate, - double initialDuration, - TagThrottleType throttleType, - TransactionPriority priority, - Optional expirationTime, - Optional reason) { - state Transaction tr(db); - state Key key = TagThrottleKey(tags, throttleType, priority).toKey(); - - ASSERT(initialDuration > 0); - - if (throttleType == TagThrottleType::MANUAL) { - reason = TagThrottledReason::MANUAL; - } - TagThrottleValue throttle(tpsRate, - expirationTime.present() ? expirationTime.get() : 0, - initialDuration, - reason.present() ? reason.get() : TagThrottledReason::UNSET); - BinaryWriter wr(IncludeVersion(ProtocolVersion::withTagThrottleValueReason())); - wr << throttle; - state Value value = wr.toValue(); - - loop { - try { - if (throttleType == TagThrottleType::MANUAL) { - Optional oldThrottle = wait(tr.get(key)); - if (!oldThrottle.present()) { - wait(updateThrottleCount(&tr, 1)); - } - } - - tr.set(key, value); - - if (throttleType == TagThrottleType::MANUAL) { - signalThrottleChange(tr); - } - - wait(tr.commit()); - return Void(); - } catch (Error& e) { - wait(tr.onError(e)); - } - } -} - -ACTOR Future unthrottleTags(Database db, - TagSet tags, - Optional throttleType, - Optional priority) { - state Transaction tr(db); - - state std::vector keys; - for (auto p : allTransactionPriorities) { - if (!priority.present() || priority.get() == p) { - if (!throttleType.present() || throttleType.get() == TagThrottleType::AUTO) { - keys.push_back(TagThrottleKey(tags, TagThrottleType::AUTO, p).toKey()); - } - if (!throttleType.present() || throttleType.get() == TagThrottleType::MANUAL) { - keys.push_back(TagThrottleKey(tags, TagThrottleType::MANUAL, p).toKey()); - } - } - } - - state bool removed = false; - - loop { - try { - state std::vector>> values; - values.reserve(keys.size()); - for (auto key : keys) { - values.push_back(tr.get(key)); - } - - wait(waitForAll(values)); - - int delta = 0; - for (int i = 0; i < values.size(); ++i) { - if (values[i].get().present()) { - if (TagThrottleKey::fromKey(keys[i]).throttleType == TagThrottleType::MANUAL) { - delta -= 1; - } - - tr.clear(keys[i]); - - // Report that we are removing this tag if we ever see it present. - // This protects us from getting confused if the transaction is maybe committed. - // It's ok if someone else actually ends up removing this tag at the same time - // and we aren't the ones to actually do it. - removed = true; - } - } - - if (delta != 0) { - wait(updateThrottleCount(&tr, delta)); - } - if (removed) { - signalThrottleChange(tr); - wait(tr.commit()); - } - - return removed; - } catch (Error& e) { - wait(tr.onError(e)); - } - } -} - -ACTOR Future unthrottleMatchingThrottles(Database db, - KeyRef beginKey, - KeyRef endKey, - Optional priority, - bool onlyExpiredThrottles) { - state Transaction tr(db); - - state KeySelector begin = firstGreaterOrEqual(beginKey); - state KeySelector end = firstGreaterOrEqual(endKey); - - state bool removed = false; - - loop { - try { - state RangeResult tags = wait(tr.getRange(begin, end, 1000)); - state uint64_t unthrottledTags = 0; - uint64_t manualUnthrottledTags = 0; - for (auto tag : tags) { - if (onlyExpiredThrottles) { - double expirationTime = TagThrottleValue::fromValue(tag.value).expirationTime; - if (expirationTime == 0 || expirationTime > now()) { - continue; - } - } - - TagThrottleKey key = TagThrottleKey::fromKey(tag.key); - if (priority.present() && key.priority != priority.get()) { - continue; - } - - if (key.throttleType == TagThrottleType::MANUAL) { - ++manualUnthrottledTags; - } - - removed = true; - tr.clear(tag.key); - unthrottledTags++; - } - - if (manualUnthrottledTags > 0) { - wait(updateThrottleCount(&tr, -manualUnthrottledTags)); - } - - if (unthrottledTags > 0) { - signalThrottleChange(tr); - } - - wait(tr.commit()); - - if (!tags.more) { - return removed; - } - - ASSERT(tags.size() > 0); - begin = KeySelector(firstGreaterThan(tags[tags.size() - 1].key), tags.arena()); - } catch (Error& e) { - wait(tr.onError(e)); - } - } -} - -Future unthrottleAll(Database db, - Optional tagThrottleType, - Optional priority) { - KeyRef begin = tagThrottleKeys.begin; - KeyRef end = tagThrottleKeys.end; - - if (tagThrottleType.present() && tagThrottleType == TagThrottleType::AUTO) { - begin = tagThrottleAutoKeysPrefix; - } else if (tagThrottleType.present() && tagThrottleType == TagThrottleType::MANUAL) { - end = tagThrottleAutoKeysPrefix; - } - - return unthrottleMatchingThrottles(db, begin, end, priority, false); -} - -Future expire(Database db) { - return unthrottleMatchingThrottles( - db, tagThrottleKeys.begin, tagThrottleKeys.end, Optional(), true); -} - -// ACTOR template -// Future enableAuto(Reference db, bool enabled) { -// state Reference tr = db->createTransaction(); - -// loop { -// try { -// Optional value = wait(safeThreadFutureToFuture(tr->get(tagThrottleAutoEnabledKey))); -// if (!value.present() || (enabled && value.get() != LiteralStringRef("1")) || -// (!enabled && value.get() != LiteralStringRef("0"))) { -// tr->set(tagThrottleAutoEnabledKey, LiteralStringRef(enabled ? "1" : "0")); -// signalThrottleChange(tr); - -// wait(safeThreadFutureToFuture(tr->commit())); -// } -// return Void(); -// } catch (Error& e) { -// wait(safeThreadFutureToFuture(tr->onError(e))); -// } -// } -// } -} // namespace ThrottleApi diff --git a/fdbclient/TagThrottle.actor.h b/fdbclient/TagThrottle.actor.h index 2f44858fcb..9a471a266a 100644 --- a/fdbclient/TagThrottle.actor.h +++ b/fdbclient/TagThrottle.actor.h @@ -33,11 +33,8 @@ #include "fdbclient/FDBOptions.g.h" #include "fdbclient/FDBTypes.h" #include "fdbclient/CommitTransaction.h" -// #include "fdbclient/SystemData.h" #include "flow/actorcompiler.h" // This must be the last #include. -#include - class Database; namespace ThrottleApi {} @@ -246,35 +243,320 @@ extern const KeyRef tagThrottleLimitKey; extern const KeyRef tagThrottleCountKey; namespace ThrottleApi { -Future> getThrottledTags(Database const& db, - int const& limit, - bool const& containsRecommend = false); -Future> getRecommendedTags(Database const& db, int const& limit); -Future throttleTags(Database const& db, - TagSet const& tags, - double const& tpsRate, - double const& initialDuration, - TagThrottleType const& throttleType, - TransactionPriority const& priority, - Optional const& expirationTime = Optional(), - Optional const& reason = Optional()); +ACTOR template +Future getValidAutoEnabled(Reference tr) { + state bool result; + loop { + Optional value = wait(safeThreadFutureToFuture(tr->get(tagThrottleAutoEnabledKey))); + if (!value.present()) { + tr->reset(); + wait(delay(CLIENT_KNOBS->DEFAULT_BACKOFF)); + continue; + } else if (value.get() == LiteralStringRef("1")) { + result = true; + } else if (value.get() == LiteralStringRef("0")) { + result = false; + } else { + TraceEvent(SevWarnAlways, "InvalidAutoTagThrottlingValue").detail("Value", value.get()); + tr->reset(); + wait(delay(CLIENT_KNOBS->DEFAULT_BACKOFF)); + continue; + } + return result; + }; +} -Future unthrottleTags(Database const& db, - TagSet const& tags, - Optional const& throttleType, - Optional const& priority); +ACTOR template +Future> getRecommendedTags(Reference db, int limit) { + state Reference tr = db->createTransaction(); + loop { + tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS); + try { + bool enableAuto = wait(getValidAutoEnabled(tr)); + if (enableAuto) { + return std::vector(); + } + state typename DB::TransactionT::template FutureT f = + tr->getRange(KeyRangeRef(tagThrottleAutoKeysPrefix, tagThrottleKeys.end), limit); + RangeResult throttles = wait(safeThreadFutureToFuture(f)); + std::vector results; + for (auto throttle : throttles) { + results.push_back(TagThrottleInfo(TagThrottleKey::fromKey(throttle.key), + TagThrottleValue::fromValue(throttle.value))); + } + return results; + } catch (Error& e) { + wait(safeThreadFutureToFuture(tr->onError(e))); + } + } +} -Future unthrottleAll(Database db, Optional throttleType, Optional priority); -Future expire(Database db); +ACTOR template +Future> getThrottledTags(Reference db, int limit, bool containsRecommend = false) { + state Reference tr = db->createTransaction(); + state bool reportAuto = containsRecommend; + loop { + tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS); + try { + if (!containsRecommend) { + wait(store(reportAuto, getValidAutoEnabled(tr))); + } + state typename DB::TransactionT::template FutureT f = tr->getRange( + reportAuto ? tagThrottleKeys : KeyRangeRef(tagThrottleKeysPrefix, tagThrottleAutoKeysPrefix), limit); + RangeResult throttles = wait(safeThreadFutureToFuture(f)); + std::vector results; + for (auto throttle : throttles) { + results.push_back(TagThrottleInfo(TagThrottleKey::fromKey(throttle.key), + TagThrottleValue::fromValue(throttle.value))); + } + return results; + } catch (Error& e) { + wait(safeThreadFutureToFuture(tr->onError(e))); + } + } +} -template +template void signalThrottleChange(Reference tr) { tr->atomicOp( tagThrottleSignalKey, LiteralStringRef("XXXXXXXXXX\x00\x00\x00\x00"), MutationRef::SetVersionstampedValue); } -ACTOR template +ACTOR template +Future updateThrottleCount(Reference tr, int64_t delta) { + state typename Tr::template FutureT> countVal = tr->get(tagThrottleCountKey); + state typename Tr::template FutureT> limitVal = tr->get(tagThrottleLimitKey); + + wait(success(safeThreadFutureToFuture(countVal)) && success(safeThreadFutureToFuture(limitVal))); + + int64_t count = 0; + int64_t limit = 0; + + if (countVal.get().present()) { + BinaryReader reader(countVal.get().get(), Unversioned()); + reader >> count; + } + + if (limitVal.get().present()) { + BinaryReader reader(limitVal.get().get(), Unversioned()); + reader >> limit; + } + + count += delta; + + if (count > limit) { + throw too_many_tag_throttles(); + } + + BinaryWriter writer(Unversioned()); + writer << count; + + tr->set(tagThrottleCountKey, writer.toValue()); + return Void(); +} + +ACTOR template +Future unthrottleMatchingThrottles(Reference db, + KeyRef beginKey, + KeyRef endKey, + Optional priority, + bool onlyExpiredThrottles) { + state Reference tr = db->createTransaction(); + + state KeySelector begin = firstGreaterOrEqual(beginKey); + state KeySelector end = firstGreaterOrEqual(endKey); + + state bool removed = false; + + loop { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + try { + // holds memory of the RangeResult + state typename DB::TransactionT::template FutureT f = tr->getRange(begin, end, 1000); + state RangeResult tags = wait(safeThreadFutureToFuture(f)); + state uint64_t unthrottledTags = 0; + uint64_t manualUnthrottledTags = 0; + for (auto tag : tags) { + if (onlyExpiredThrottles) { + double expirationTime = TagThrottleValue::fromValue(tag.value).expirationTime; + if (expirationTime == 0 || expirationTime > now()) { + continue; + } + } + + TagThrottleKey key = TagThrottleKey::fromKey(tag.key); + if (priority.present() && key.priority != priority.get()) { + continue; + } + + if (key.throttleType == TagThrottleType::MANUAL) { + ++manualUnthrottledTags; + } + + removed = true; + tr->clear(tag.key); + unthrottledTags++; + } + + if (manualUnthrottledTags > 0) { + wait(updateThrottleCount(tr, -manualUnthrottledTags)); + } + + if (unthrottledTags > 0) { + signalThrottleChange(tr); + } + + wait(safeThreadFutureToFuture(tr->commit())); + + if (!tags.more) { + return removed; + } + + ASSERT(tags.size() > 0); + begin = KeySelector(firstGreaterThan(tags[tags.size() - 1].key), tags.arena()); + } catch (Error& e) { + wait(safeThreadFutureToFuture(tr->onError(e))); + } + } +} + +template +Future expire(DB db) { + return unthrottleMatchingThrottles( + db, tagThrottleKeys.begin, tagThrottleKeys.end, Optional(), true); +} + +template +Future unthrottleAll(Reference db, + Optional tagThrottleType, + Optional priority) { + KeyRef begin = tagThrottleKeys.begin; + KeyRef end = tagThrottleKeys.end; + + if (tagThrottleType.present() && tagThrottleType == TagThrottleType::AUTO) { + begin = tagThrottleAutoKeysPrefix; + } else if (tagThrottleType.present() && tagThrottleType == TagThrottleType::MANUAL) { + end = tagThrottleAutoKeysPrefix; + } + + return unthrottleMatchingThrottles(db, begin, end, priority, false); +} + +ACTOR template +Future unthrottleTags(Reference db, + TagSet tags, + Optional throttleType, + Optional priority) { + state Reference tr = db->createTransaction(); + + state std::vector keys; + for (auto p : allTransactionPriorities) { + if (!priority.present() || priority.get() == p) { + if (!throttleType.present() || throttleType.get() == TagThrottleType::AUTO) { + keys.push_back(TagThrottleKey(tags, TagThrottleType::AUTO, p).toKey()); + } + if (!throttleType.present() || throttleType.get() == TagThrottleType::MANUAL) { + keys.push_back(TagThrottleKey(tags, TagThrottleType::MANUAL, p).toKey()); + } + } + } + + state bool removed = false; + + loop { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + try { + state std::vector>> values; + values.reserve(keys.size()); + for (auto key : keys) { + values.push_back(safeThreadFutureToFuture(tr->get(key))); + } + + wait(waitForAll(values)); + + int delta = 0; + for (int i = 0; i < values.size(); ++i) { + if (values[i].get().present()) { + if (TagThrottleKey::fromKey(keys[i]).throttleType == TagThrottleType::MANUAL) { + delta -= 1; + } + + tr->clear(keys[i]); + + // Report that we are removing this tag if we ever see it present. + // This protects us from getting confused if the transaction is maybe committed. + // It's ok if someone else actually ends up removing this tag at the same time + // and we aren't the ones to actually do it. + removed = true; + } + } + + if (delta != 0) { + wait(updateThrottleCount(tr, delta)); + } + if (removed) { + signalThrottleChange(tr); + wait(safeThreadFutureToFuture(tr->commit())); + } + + return removed; + } catch (Error& e) { + wait(safeThreadFutureToFuture(tr->onError(e))); + } + } +} + +ACTOR template +Future throttleTags(Reference db, + TagSet tags, + double tpsRate, + double initialDuration, + TagThrottleType throttleType, + TransactionPriority priority, + Optional expirationTime = Optional(), + Optional reason = Optional()) { + state Reference tr = db->createTransaction(); + state Key key = TagThrottleKey(tags, throttleType, priority).toKey(); + + ASSERT(initialDuration > 0); + + if (throttleType == TagThrottleType::MANUAL) { + reason = TagThrottledReason::MANUAL; + } + TagThrottleValue throttle(tpsRate, + expirationTime.present() ? expirationTime.get() : 0, + initialDuration, + reason.present() ? reason.get() : TagThrottledReason::UNSET); + BinaryWriter wr(IncludeVersion(ProtocolVersion::withTagThrottleValueReason())); + wr << throttle; + state Value value = wr.toValue(); + + loop { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + try { + if (throttleType == TagThrottleType::MANUAL) { + Optional oldThrottle = wait(safeThreadFutureToFuture(tr->get(key))); + if (!oldThrottle.present()) { + wait(updateThrottleCount(tr, 1)); + } + } + + tr->set(key, value); + + if (throttleType == TagThrottleType::MANUAL) { + signalThrottleChange(tr); + } + + wait(safeThreadFutureToFuture(tr->commit())); + return Void(); + } catch (Error& e) { + wait(safeThreadFutureToFuture(tr->onError(e))); + } + } +} + +ACTOR template Future enableAuto(Reference db, bool enabled) { state Reference tr = db->createTransaction(); @@ -306,4 +588,6 @@ using PrioritizedTransactionTagMap = std::map using UIDTransactionTagMap = std::unordered_map>; + +#include "flow/unactorcompiler.h" #endif diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index 54bd4f9ca3..847862803c 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -595,8 +595,12 @@ struct RatekeeperData { SERVER_KNOBS->MAX_TL_SS_VERSION_DIFFERENCE_BATCH, SERVER_KNOBS->TARGET_DURABILITY_LAG_VERSIONS_BATCH), autoThrottlingEnabled(false) { - expiredTagThrottleCleanup = - recurring([this]() { ThrottleApi::expire(this->db); }, SERVER_KNOBS->TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL); + expiredTagThrottleCleanup = recurring( + [this]() { + Reference db = makeReference(this->db); + ThrottleApi::expire(db); + }, + SERVER_KNOBS->TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL); } }; @@ -942,7 +946,8 @@ void tryAutoThrottleTag(RatekeeperData* self, TagSet tags; tags.addTag(tag); - self->addActor.send(ThrottleApi::throttleTags(self->db, + Reference db = makeReference(self->db); + self->addActor.send(ThrottleApi::throttleTags(db, tags, clientRate.get(), SERVER_KNOBS->AUTO_TAG_THROTTLE_DURATION, diff --git a/fdbserver/workloads/TagThrottleApi.actor.cpp b/fdbserver/workloads/TagThrottleApi.actor.cpp index 005322007e..2a4094989a 100644 --- a/fdbserver/workloads/TagThrottleApi.actor.cpp +++ b/fdbserver/workloads/TagThrottleApi.actor.cpp @@ -78,12 +78,13 @@ struct TagThrottleApiWorkload : TestWorkload { state TransactionPriority priority = deterministicRandom()->randomChoice(allTransactionPriorities); state double rate = deterministicRandom()->random01() * 20; state double duration = 1 + deterministicRandom()->random01() * 19; + state Reference db = makeReference(cx); TagSet tagSet; tagSet.addTag(tag); try { - wait(ThrottleApi::throttleTags(cx, + wait(ThrottleApi::throttleTags(db, tagSet, rate, duration, @@ -137,7 +138,8 @@ struct TagThrottleApiWorkload : TestWorkload { } } - bool removed = wait(ThrottleApi::unthrottleTags(cx, tagSet, throttleType, priority)); + state Reference db = makeReference(cx); + bool removed = wait(ThrottleApi::unthrottleTags(db, tagSet, throttleType, priority)); if (removed) { ASSERT(erased || !throttleType.present() || throttleType.get() == TagThrottleType::AUTO); } else { @@ -151,7 +153,9 @@ struct TagThrottleApiWorkload : TestWorkload { TagThrottleApiWorkload* self, Database cx, std::map, TagThrottleInfo> const* manuallyThrottledTags) { - std::vector tags = wait(ThrottleApi::getThrottledTags(cx, CLIENT_KNOBS->TOO_MANY)); + + state Reference db = makeReference(cx); + std::vector tags = wait(ThrottleApi::getThrottledTags(db, CLIENT_KNOBS->TOO_MANY)); int manualThrottledTags = 0; int activeAutoThrottledTags = 0; @@ -184,7 +188,8 @@ struct TagThrottleApiWorkload : TestWorkload { } ACTOR Future getRecommendedTags(TagThrottleApiWorkload* self, Database cx) { - std::vector tags = wait(ThrottleApi::getRecommendedTags(cx, CLIENT_KNOBS->TOO_MANY)); + state Reference db = makeReference(cx); + std::vector tags = wait(ThrottleApi::getRecommendedTags(db, CLIENT_KNOBS->TOO_MANY)); for (auto& tag : tags) { ASSERT(tag.throttleType == TagThrottleType::AUTO); @@ -200,7 +205,8 @@ struct TagThrottleApiWorkload : TestWorkload { deterministicRandom()->coinflip() ? Optional() : deterministicRandom()->randomChoice(allTransactionPriorities); - bool unthrottled = wait(ThrottleApi::unthrottleAll(cx, throttleType, priority)); + state Reference db = makeReference(cx); + bool unthrottled = wait(ThrottleApi::unthrottleAll(db, throttleType, priority)); if (!throttleType.present() || throttleType.get() == TagThrottleType::MANUAL) { bool unthrottleExpected = false; bool empty = manuallyThrottledTags->empty(); @@ -227,13 +233,13 @@ struct TagThrottleApiWorkload : TestWorkload { } ACTOR Future enableAutoThrottling(TagThrottleApiWorkload* self, Database cx) { - state Reference db = Reference::addRef(&cx); + state Reference db = makeReference(cx); if (deterministicRandom()->coinflip()) { wait(ThrottleApi::enableAuto(db, true)); self->autoThrottleEnabled = true; if (deterministicRandom()->coinflip()) { bool unthrottled = - wait(ThrottleApi::unthrottleAll(cx, TagThrottleType::AUTO, Optional())); + wait(ThrottleApi::unthrottleAll(db, TagThrottleType::AUTO, Optional())); } } else { wait(ThrottleApi::enableAuto(db, false)); diff --git a/fdbserver/workloads/WriteTagThrottling.actor.cpp b/fdbserver/workloads/WriteTagThrottling.actor.cpp index e188eab9ae..a6ff0f5d7a 100644 --- a/fdbserver/workloads/WriteTagThrottling.actor.cpp +++ b/fdbserver/workloads/WriteTagThrottling.actor.cpp @@ -94,7 +94,7 @@ struct WriteTagThrottlingWorkload : KVWorkload { std::string description() const override { return WriteTagThrottlingWorkload::NAME; } ACTOR static Future _setup(Database cx, WriteTagThrottlingWorkload* self) { - state Reference db = Reference::addRef(&cx); + state Reference db = makeReference(cx); ASSERT(CLIENT_KNOBS->MAX_TAGS_PER_TRANSACTION >= MIN_TAGS_PER_TRANSACTION && CLIENT_KNOBS->MAX_TRANSACTION_TAG_LENGTH >= MIN_TRANSACTION_TAG_LENGTH); if (self->populateData) { @@ -307,9 +307,10 @@ struct WriteTagThrottlingWorkload : KVWorkload { } ACTOR static Future throttledTagUpdater(Database cx, WriteTagThrottlingWorkload* self) { state std::vector tags; + state Reference db = makeReference(cx); loop { wait(delay(1.0)); - wait(store(tags, ThrottleApi::getThrottledTags(cx, CLIENT_KNOBS->TOO_MANY, true))); + wait(store(tags, ThrottleApi::getThrottledTags(db, CLIENT_KNOBS->TOO_MANY, true))); self->recordThrottledTags(tags); }; } diff --git a/flow/ThreadHelper.actor.h b/flow/ThreadHelper.actor.h index b1e40604f1..91dce5468e 100644 --- a/flow/ThreadHelper.actor.h +++ b/flow/ThreadHelper.actor.h @@ -627,6 +627,7 @@ Future safeThreadFutureToFuture(ThreadFuture threadFuture) { return threadFuture.get(); } +// do nothing, just for template functions' calls template Future safeThreadFutureToFuture(Future future) { // do nothing From a97570bd065bc1796788d27fb1586457ccc4949c Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Wed, 11 Aug 2021 18:26:00 -0700 Subject: [PATCH 29/63] solve mis-spelling, trace log and format problems --- fdbclient/ServerKnobs.cpp | 2 +- fdbclient/ServerKnobs.h | 2 +- fdbserver/LogRouter.actor.cpp | 5 +++-- fdbserver/LogSystemPeekCursor.actor.cpp | 11 +++++------ fdbserver/OldTLogServer_4_6.actor.cpp | 4 +++- fdbserver/OldTLogServer_6_0.actor.cpp | 4 +++- fdbserver/OldTLogServer_6_2.actor.cpp | 4 +++- fdbserver/TLogInterface.h | 4 ++-- fdbserver/TLogServer.actor.cpp | 4 +++- flow/Trace.h | 1 + 10 files changed, 25 insertions(+), 16 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 290452012a..1e90c5a38b 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -64,7 +64,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( TLOG_MESSAGE_BLOCK_BYTES, 10e6 ); init( TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR, double(TLOG_MESSAGE_BLOCK_BYTES) / (TLOG_MESSAGE_BLOCK_BYTES - MAX_MESSAGE_SIZE) ); //1.0121466709838096006362758832473 init( PEEK_TRACKER_EXPIRATION_TIME, 600 ); if( randomize && BUGGIFY ) PEEK_TRACKER_EXPIRATION_TIME = deterministicRandom()->coinflip() ? 0.1 : 120; - init( PEEK_USEING_STREAMING, true ); + init( PEEK_USING_STREAMING, true ); init( PARALLEL_GET_MORE_REQUESTS, 32 ); if( randomize && BUGGIFY ) PARALLEL_GET_MORE_REQUESTS = 2; init( MULTI_CURSOR_PRE_FETCH_LIMIT, 10 ); init( MAX_QUEUE_COMMIT_BYTES, 15e6 ); if( randomize && BUGGIFY ) MAX_QUEUE_COMMIT_BYTES = 5000; diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index 23b3049668..c905720898 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -41,7 +41,7 @@ public: // often, so that versions always advance smoothly // TLogs - bool PEEK_USEING_STREAMING; + bool PEEK_USING_STREAMING; double TLOG_TIMEOUT; // tlog OR commit proxy failure - master's reaction time double TLOG_SLOW_REJOIN_WARN_TIMEOUT_SECS; // Warns if a tlog takes too long to rejoin double RECOVERY_TLOG_SMART_QUORUM_DELAY; // smaller might be better for bug amplification diff --git a/fdbserver/LogRouter.actor.cpp b/fdbserver/LogRouter.actor.cpp index f26ea539af..22348fe8b3 100644 --- a/fdbserver/LogRouter.actor.cpp +++ b/fdbserver/LogRouter.actor.cpp @@ -573,7 +573,6 @@ Future logRouterPeekMessages(PromiseType replyPromise, return Void(); } -// TODO: enable streaming peek log from log router // This actor keep pushing TLogPeekStreamReply until it's removed from the cluster or should recover ACTOR Future logRouterPeekStream(LogRouterData* self, TLogPeekStreamRequest req) { self->activePeekStreams++; @@ -600,7 +599,9 @@ ACTOR Future logRouterPeekStream(LogRouterData* self, TLogPeekStreamReques } } catch (Error& e) { self->activePeekStreams--; - TraceEvent(SevDebug, "TLogPeekStreamEnd", self->dbgid).error(e, true); + TraceEvent(SevDebug, "TLogPeekStreamEnd", self->dbgid) + .detail("PeerAddr", req.reply.getEndpoint().getPrimaryAddress()) + .error(e, true); if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { req.reply.sendError(e); diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 769fd2ee57..4ab5aaf56f 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -38,7 +38,7 @@ ACTOR Future tryEstablishPeekStream(ILogSystem::ServerPeekCursor* self) { self->peekReplyStream = self->interf->get().interf().peekStreamMessages.getReplyStream(TLogPeekStreamRequest( self->messageVersion.version, self->tag, self->returnIfBlocked, std::numeric_limits::max())); TraceEvent(SevDebug, "SPC_StreamCreated", self->randomID) - .detail("PeerAddress", self->interf->get().interf().peekStreamMessages.getEndpoint().getPrimaryAddress()) + .detail("PeerAddr", self->interf->get().interf().peekStreamMessages.getEndpoint().getPrimaryAddress()) .detail("PeerToken", self->interf->get().interf().peekStreamMessages.getEndpoint().token); return Void(); } @@ -52,11 +52,11 @@ ILogSystem::ServerPeekCursor::ServerPeekCursor(ReferencerandomUniqueID()), returnIfBlocked(returnIfBlocked), onlySpilled(false), parallelGetMore(parallelGetMore), - usePeekStream(SERVER_KNOBS->PEEK_USEING_STREAMING), sequence(0), lastReset(0), resetCheck(Void()), slowReplies(0), + usePeekStream(SERVER_KNOBS->PEEK_USING_STREAMING), sequence(0), lastReset(0), resetCheck(Void()), slowReplies(0), fastReplies(0), unknownReplies(0) { this->results.maxKnownVersion = 0; this->results.minKnownCommittedVersion = 0; - TraceEvent(SevDebug, "SPC_Starting", randomID) + DisabledTraceEvent(SevDebug, "SPC_Starting", randomID) .detail("Tag", tag.toString()) .detail("Begin", begin) .detail("End", end); @@ -355,7 +355,7 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T } updateCursorWithReply(self, res); expectedBegin = res.end; - TraceEvent(SevDebug, "SPC_GetMoreB", self->randomID) + DisabledTraceEvent(SevDebug, "SPC_GetMoreB", self->randomID) .detail("Has", self->hasMessage()) .detail("End", res.end) .detail("Popped", res.popped.present() ? res.popped.get() : 0); @@ -367,7 +367,7 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T } } } catch (Error& e) { - TraceEvent(SevDebug, "SPC_GetMoreB_Error").error(e, true); + TraceEvent(SevDebug, "SPC_GetMoreB_Error", self->randomID).error(e, true); if (e.code() == error_code_connection_failed || e.code() == error_code_operation_obsolete) { // NOTE: delay in order to avoid the endless retry loop block other tasks self->peekReplyStream.reset(); @@ -424,7 +424,6 @@ Future ILogSystem::ServerPeekCursor::getMore(TaskPriority taskID) { if (hasMessage() && !parallelGetMore) return Void(); if (!more.isValid() || more.isReady()) { - // TODO: add tagLocalityRemoteLog when log router support streaming peek if (usePeekStream && (tag.locality >= 0 || tag.locality == tagLocalityLogRouter || tag.locality == tagLocalityRemoteLog)) { more = serverPeekStreamGetMore(this, taskID); } else if (parallelGetMore || onlySpilled || futureResults.size()) { diff --git a/fdbserver/OldTLogServer_4_6.actor.cpp b/fdbserver/OldTLogServer_4_6.actor.cpp index f50744cc1a..7cb4c565bf 100644 --- a/fdbserver/OldTLogServer_4_6.actor.cpp +++ b/fdbserver/OldTLogServer_4_6.actor.cpp @@ -1162,7 +1162,9 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref } } catch (Error& e) { self->activePeekStreams--; - TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); + TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId) + .detail("PeerAddr", req.reply.getEndpoint().getPrimaryAddress()) + .error(e, true); if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { req.reply.sendError(e); diff --git a/fdbserver/OldTLogServer_6_0.actor.cpp b/fdbserver/OldTLogServer_6_0.actor.cpp index 5d77b4e12f..cc561d1f3d 100644 --- a/fdbserver/OldTLogServer_6_0.actor.cpp +++ b/fdbserver/OldTLogServer_6_0.actor.cpp @@ -1474,7 +1474,9 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref } } catch (Error& e) { self->activePeekStreams--; - TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); + TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId) + .detail("PeerAddr", req.reply.getEndpoint().getPrimaryAddress()) + .error(e, true); if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { req.reply.sendError(e); diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index 687ea0e638..284afb1da3 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -1903,7 +1903,9 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref } } catch (Error& e) { self->activePeekStreams--; - TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); + TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId) + .detail("PeerAddr", req.reply.getEndpoint().getPrimaryAddress()) + .error(e, true); if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { req.reply.sendError(e); diff --git a/fdbserver/TLogInterface.h b/fdbserver/TLogInterface.h index 322c7ddb35..1ca049927d 100644 --- a/fdbserver/TLogInterface.h +++ b/fdbserver/TLogInterface.h @@ -83,7 +83,7 @@ struct TLogInterface { streams.push_back(disablePopRequest.getReceiver()); streams.push_back(enablePopRequest.getReceiver()); streams.push_back(snapRequest.getReceiver()); - streams.push_back(peekStreamMessages.getReceiver(TaskPriority::TLogPeek)); + streams.push_back(peekStreamMessages.getReceiver(TaskPriority::TLogPeek)); FlowTransport::transport().addEndpoints(streams); } @@ -235,7 +235,7 @@ struct TLogPeekStreamRequest { Arena arena; Version begin; Tag tag; - bool returnIfBlocked; + bool returnIfBlocked; int limitBytes; ReplyPromiseStream reply; diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index f7853e23b2..6608468860 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -1953,7 +1953,9 @@ ACTOR Future tLogPeekStream(TLogData* self, TLogPeekStreamRequest req, Ref } } catch (Error& e) { self->activePeekStreams--; - TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId).error(e, true); + TraceEvent(SevDebug, "TLogPeekStreamEnd", logData->logId) + .detail("PeerAddr", req.reply.getEndpoint().getPrimaryAddress()) + .error(e, true); if (e.code() == error_code_end_of_stream || e.code() == error_code_operation_obsolete) { req.reply.sendError(e); diff --git a/flow/Trace.h b/flow/Trace.h index 467422b9cc..aeaabb4373 100644 --- a/flow/Trace.h +++ b/flow/Trace.h @@ -599,4 +599,5 @@ extern TraceBatch g_traceBatch; #define DUMPTOKEN(name) \ TraceEvent("DumpToken", recruited.id()).detail("Name", #name).detail("Token", name.getEndpoint().token) +#define DisabledTraceEvent(...) false && TraceEvent() #endif From be796a62a0498e1a936595e3d4b56e7449c1d954 Mon Sep 17 00:00:00 2001 From: Chaoguang Lin Date: Wed, 11 Aug 2021 18:39:47 -0700 Subject: [PATCH 30/63] Update comments --- fdbcli/ThrottleCommand.actor.cpp | 1 - fdbclient/IClientApi.h | 2 ++ fdbclient/NativeAPI.actor.h | 4 ++-- fdbclient/ReadYourWrites.h | 1 + 4 files changed, 5 insertions(+), 3 deletions(-) diff --git a/fdbcli/ThrottleCommand.actor.cpp b/fdbcli/ThrottleCommand.actor.cpp index d09cf263e6..a9db5e43d6 100644 --- a/fdbcli/ThrottleCommand.actor.cpp +++ b/fdbcli/ThrottleCommand.actor.cpp @@ -27,7 +27,6 @@ #include "fdbclient/CommitTransaction.h" #include "flow/Arena.h" -#include "flow/FastRef.h" #include "flow/ThreadHelper.actor.h" #include "flow/genericactors.actor.h" #include "flow/actorcompiler.h" // This must be the last #include. diff --git a/fdbclient/IClientApi.h b/fdbclient/IClientApi.h index cf04eeedbf..b59c80c4ed 100644 --- a/fdbclient/IClientApi.h +++ b/fdbclient/IClientApi.h @@ -89,6 +89,7 @@ public: virtual void addref() = 0; virtual void delref() = 0; + // used in template functions as returned Future type template using FutureT = ThreadFuture; }; @@ -118,6 +119,7 @@ public: // Management API, create snapshot virtual ThreadFuture createSnapshot(const StringRef& uid, const StringRef& snapshot_command) = 0; + // used in templated functions as Transaction type that can be created using TransactionT = ITransaction; }; diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h index 02139aa73b..6afef95873 100644 --- a/fdbclient/NativeAPI.actor.h +++ b/fdbclient/NativeAPI.actor.h @@ -76,8 +76,6 @@ struct NetworkOptions { NetworkOptions(); }; -class ReadYourWritesTransaction; // workaround cyclic dependency - class Database final : public ThreadSafeReferenceCounted { public: enum { API_VERSION_LATEST = -1 }; @@ -111,6 +109,7 @@ public: const UniqueOrderedOptionList& getTransactionDefaults() const; + // used in template functions to create a transaction using TransactionT = ReadYourWritesTransaction; Reference createTransaction(); @@ -174,6 +173,7 @@ private: void clear(); }; +class ReadYourWritesTransaction; // workaround cyclic dependency struct TransactionInfo { Optional debugID; TaskPriority taskID; diff --git a/fdbclient/ReadYourWrites.h b/fdbclient/ReadYourWrites.h index e24b02b13a..781a925b54 100644 --- a/fdbclient/ReadYourWrites.h +++ b/fdbclient/ReadYourWrites.h @@ -175,6 +175,7 @@ public: void setSpecialKeySpaceErrorMsg(const std::string& msg) { specialKeySpaceErrorMsg = msg; } Transaction& getTransaction() { return tr; } + // used in template functions as returned Future type template using FutureT = Future; private: From 6ef49fe91ec2f6013060ae5754c825407f4c60c7 Mon Sep 17 00:00:00 2001 From: Chaoguang Lin Date: Wed, 11 Aug 2021 23:19:39 -0700 Subject: [PATCH 31/63] Update comments, remove headers --- fdbclient/IClientApi.h | 2 +- fdbclient/TagThrottle.actor.cpp | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/fdbclient/IClientApi.h b/fdbclient/IClientApi.h index b59c80c4ed..cd684c2369 100644 --- a/fdbclient/IClientApi.h +++ b/fdbclient/IClientApi.h @@ -119,7 +119,7 @@ public: // Management API, create snapshot virtual ThreadFuture createSnapshot(const StringRef& uid, const StringRef& snapshot_command) = 0; - // used in templated functions as Transaction type that can be created + // used in template functions as the Transaction type that can be created through createTransaction() using TransactionT = ITransaction; }; diff --git a/fdbclient/TagThrottle.actor.cpp b/fdbclient/TagThrottle.actor.cpp index c874c0d7c3..af017b29b3 100644 --- a/fdbclient/TagThrottle.actor.cpp +++ b/fdbclient/TagThrottle.actor.cpp @@ -22,8 +22,6 @@ #include "fdbclient/CommitProxyInterface.h" #include "fdbclient/DatabaseContext.h" -#include "flow/FastRef.h" -#include "flow/ThreadHelper.actor.h" #include "flow/actorcompiler.h" // has to be last include void TagSet::addTag(TransactionTagRef tag) { From df7a801945f0e607af784205c57eb8e29bb35c47 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Thu, 12 Aug 2021 14:10:34 -0700 Subject: [PATCH 32/63] remove FIXME --- fdbclient/ServerKnobs.cpp | 2 +- fdbserver/LogRouter.actor.cpp | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 1e90c5a38b..0873b072dc 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -64,7 +64,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( TLOG_MESSAGE_BLOCK_BYTES, 10e6 ); init( TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR, double(TLOG_MESSAGE_BLOCK_BYTES) / (TLOG_MESSAGE_BLOCK_BYTES - MAX_MESSAGE_SIZE) ); //1.0121466709838096006362758832473 init( PEEK_TRACKER_EXPIRATION_TIME, 600 ); if( randomize && BUGGIFY ) PEEK_TRACKER_EXPIRATION_TIME = deterministicRandom()->coinflip() ? 0.1 : 120; - init( PEEK_USING_STREAMING, true ); + init( PEEK_USING_STREAMING, true ); init( PARALLEL_GET_MORE_REQUESTS, 32 ); if( randomize && BUGGIFY ) PARALLEL_GET_MORE_REQUESTS = 2; init( MULTI_CURSOR_PRE_FETCH_LIMIT, 10 ); init( MAX_QUEUE_COMMIT_BYTES, 15e6 ); if( randomize && BUGGIFY ) MAX_QUEUE_COMMIT_BYTES = 5000; diff --git a/fdbserver/LogRouter.actor.cpp b/fdbserver/LogRouter.actor.cpp index 22348fe8b3..7e3316b28d 100644 --- a/fdbserver/LogRouter.actor.cpp +++ b/fdbserver/LogRouter.actor.cpp @@ -694,11 +694,9 @@ ACTOR Future logRouterCore(TLogInterface interf, req.reply, &logRouterData, req.begin, req.tag, req.returnIfBlocked, req.onlySpilled, req.sequence)); } when(TLogPeekStreamRequest req = waitNext(interf.peekStreamMessages.getFuture())) { - // FIXME: currently LogRouter doesn't support streaming peek request TraceEvent(SevDebug, "LogRouterPeekStream", logRouterData.dbgid) .detail("Token", interf.peekStreamMessages.getEndpoint().token); addActor.send(logRouterPeekStream(&logRouterData, req)); - // req.reply.sendError(operation_failed()); } when(TLogPopRequest req = waitNext(interf.popMessages.getFuture())) { // Request from remote tLog to pop data from LR From c45daf6f512ad663927536de4ce9db1729c928ce Mon Sep 17 00:00:00 2001 From: Neethu Haneesha Bingi Date: Fri, 13 Aug 2021 01:14:52 -0700 Subject: [PATCH 33/63] Disabling option for removing eagerReads for ClearRange mutations. --- fdbclient/ServerKnobs.cpp | 1 + fdbclient/ServerKnobs.h | 1 + fdbserver/storageserver.actor.cpp | 29 +++++++++++++++++------------ 3 files changed, 19 insertions(+), 12 deletions(-) diff --git a/fdbclient/ServerKnobs.cpp b/fdbclient/ServerKnobs.cpp index 3437186209..2ed56c64e5 100644 --- a/fdbclient/ServerKnobs.cpp +++ b/fdbclient/ServerKnobs.cpp @@ -627,6 +627,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi init( FETCH_KEYS_TOO_LONG_TIME_CRITERIA, 300.0 ); init( MAX_STORAGE_COMMIT_TIME, 120.0 ); //The max fsync stall time on the storage server and tlog before marking a disk as failed init( RANGESTREAM_LIMIT_BYTES, 2e6 ); if( randomize && BUGGIFY ) RANGESTREAM_LIMIT_BYTES = 1; + init( ENABLE_CLEAR_RANGE_EAGER_READS, true ); //Wait Failure init( MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS, 250 ); if( randomize && BUGGIFY ) MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS = 2; diff --git a/fdbclient/ServerKnobs.h b/fdbclient/ServerKnobs.h index 82dbd227b0..d3dd077a4c 100644 --- a/fdbclient/ServerKnobs.h +++ b/fdbclient/ServerKnobs.h @@ -563,6 +563,7 @@ public: double FETCH_KEYS_TOO_LONG_TIME_CRITERIA; double MAX_STORAGE_COMMIT_TIME; int64_t RANGESTREAM_LIMIT_BYTES; + bool ENABLE_CLEAR_RANGE_EAGER_READS; // Wait Failure int MAX_OUTSTANDING_WAIT_FAILURE_REQUESTS; diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 15a06b5ce5..76170d79c9 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -241,10 +241,12 @@ struct UpdateEagerReadInfo { void addMutation(MutationRef const& m) { // SOMEDAY: Theoretically we can avoid a read if there is an earlier overlapping ClearRange - if (m.type == MutationRef::ClearRange && !m.param2.startsWith(systemKeys.end)) + if (m.type == MutationRef::ClearRange && !m.param2.startsWith(systemKeys.end) && + SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS) keyBegin.push_back(m.param2); else if (m.type == MutationRef::CompareAndClear) { - keyBegin.push_back(keyAfter(m.param1, arena)); + if (SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS) + keyBegin.push_back(keyAfter(m.param1, arena)); if (keys.size() > 0 && keys.back().first == m.param1) { // Don't issue a second read, if the last read was equal to the current key. // CompareAndClear is likely to be used after another atomic operation on same key. @@ -260,8 +262,10 @@ struct UpdateEagerReadInfo { } void finishKeyBegin() { - std::sort(keyBegin.begin(), keyBegin.end()); - keyBegin.resize(std::unique(keyBegin.begin(), keyBegin.end()) - keyBegin.begin()); + if (SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS) { + std::sort(keyBegin.begin(), keyBegin.end()); + keyBegin.resize(std::unique(keyBegin.begin(), keyBegin.end()) - keyBegin.begin()); + } std::sort(keys.begin(), keys.end(), [](const std::pair& lhs, const std::pair& rhs) { return (lhs.first < rhs.first) || (lhs.first == rhs.first && lhs.second > rhs.second); }); @@ -2377,21 +2381,22 @@ void getQueuingMetrics(StorageServer* self, StorageQueuingMetricsRequest const& ACTOR Future doEagerReads(StorageServer* data, UpdateEagerReadInfo* eager) { eager->finishKeyBegin(); - vector> keyEnd(eager->keyBegin.size()); - for (int i = 0; i < keyEnd.size(); i++) - keyEnd[i] = data->storage.readNextKeyInclusive(eager->keyBegin[i]); + if (SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS) { + vector> keyEnd(eager->keyBegin.size()); + for (int i = 0; i < keyEnd.size(); i++) + keyEnd[i] = data->storage.readNextKeyInclusive(eager->keyBegin[i]); - state Future> futureKeyEnds = getAll(keyEnd); + state Future> futureKeyEnds = getAll(keyEnd); + state vector keyEndVal = wait(futureKeyEnds); + eager->keyEnd = keyEndVal; + } vector>> value(eager->keys.size()); for (int i = 0; i < value.size(); i++) value[i] = data->storage.readValuePrefix(eager->keys[i].first, eager->keys[i].second); state Future>> futureValues = getAll(value); - state vector keyEndVal = wait(futureKeyEnds); vector> optionalValues = wait(futureValues); - - eager->keyEnd = keyEndVal; eager->value = optionalValues; return Void(); @@ -2500,7 +2505,7 @@ bool expandMutation(MutationRef& m, i = d.lastLessOrEqual(m.param2); if (i && i->isClearTo() && i->getEndKey() >= m.param2) { m.param2 = i->getEndKey(); - } else { + } else if (SERVER_KNOBS->ENABLE_CLEAR_RANGE_EAGER_READS) { // Expand to the next set or clear (from storage or latestVersion), and if it // is a clear, engulf it as well i = d.lower_bound(m.param2); From c6ed0f94bf1f08f0b97b80a3198d4053ff2cada9 Mon Sep 17 00:00:00 2001 From: Eric Dong Date: Fri, 13 Aug 2021 13:02:49 -0700 Subject: [PATCH 34/63] Fix the wording of a CMake message for SSL on Windows --- cmake/FDBComponents.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/FDBComponents.cmake b/cmake/FDBComponents.cmake index 2aac3985eb..e05fb1f5df 100644 --- a/cmake/FDBComponents.cmake +++ b/cmake/FDBComponents.cmake @@ -42,7 +42,7 @@ else() set(WITH_TLS OFF) endif() if(WIN32) - message(STATUS "TLS is temporarilty disabled on macOS while libressl -> openssl transition happens") + message(STATUS "TLS is temporarilty disabled on Windows while libressl -> openssl transition happens") set(WITH_TLS OFF) endif() endif() From ceb83f7f5e3189df44129139bfe7a930ff7c6bc4 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sat, 14 Aug 2021 16:51:39 -0700 Subject: [PATCH 35/63] Make ccInterface a const reference in workerServer --- fdbserver/ClusterController.actor.cpp | 8 ++++---- fdbserver/WorkerInterface.actor.h | 2 +- fdbserver/masterserver.actor.cpp | 2 +- fdbserver/worker.actor.cpp | 10 +++++----- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 7ddae0a5ba..d239b7cbab 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -4858,18 +4858,18 @@ ACTOR Future clusterControllerCore(ClusterControllerFullInterface interf, ++self.getWorkersRequests; vector workers; - for (auto& it : self.id_worker) { + for (auto const& [id, worker] : self.id_worker) { if ((req.flags & GetWorkersRequest::NON_EXCLUDED_PROCESSES_ONLY) && - self.db.config.isExcludedServer(it.second.details.interf.addresses())) { + self.db.config.isExcludedServer(worker.details.interf.addresses())) { continue; } if ((req.flags & GetWorkersRequest::TESTER_CLASS_ONLY) && - it.second.details.processClass.classType() != ProcessClass::TesterClass) { + worker.details.processClass.classType() != ProcessClass::TesterClass) { continue; } - workers.push_back(it.second.details); + workers.push_back(worker.details); } req.reply.send(workers); diff --git a/fdbserver/WorkerInterface.actor.h b/fdbserver/WorkerInterface.actor.h index 091d6fcee7..b1ee9e1728 100644 --- a/fdbserver/WorkerInterface.actor.h +++ b/fdbserver/WorkerInterface.actor.h @@ -925,7 +925,7 @@ ACTOR Future storageServer( connFile); // changes pssi->id() to be the recovered ID); // changes pssi->id() to be the recovered ID ACTOR Future masterServer(MasterInterface mi, Reference const> db, - Reference>> ccInterface, + Reference> const> ccInterface, ServerCoordinators serverCoordinators, LifetimeToken lifetime, bool forceRecovery); diff --git a/fdbserver/masterserver.actor.cpp b/fdbserver/masterserver.actor.cpp index 7a04d53aad..49476c493f 100644 --- a/fdbserver/masterserver.actor.cpp +++ b/fdbserver/masterserver.actor.cpp @@ -1981,7 +1981,7 @@ ACTOR Future masterCore(Reference self) { ACTOR Future masterServer(MasterInterface mi, Reference const> db, - Reference>> ccInterface, + Reference> const> ccInterface, ServerCoordinators coordinators, LifetimeToken lifetime, bool forceRecovery) { diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index cd88f39b3d..f35cdd1aa6 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -604,7 +604,7 @@ ACTOR Future registrationClient(Reference> dbInfo) { +bool addressInDbAndPrimaryDc(const NetworkAddress& address, Reference const> dbInfo) { const auto& dbi = dbInfo->get(); if (dbi.master.addresses().contains(address)) { @@ -661,7 +661,7 @@ bool addressInDbAndPrimaryDc(const NetworkAddress& address, Reference> dbInfo) { +bool addressesInDbAndPrimaryDc(const NetworkAddressList& addresses, Reference const> dbInfo) { return addressInDbAndPrimaryDc(addresses.address, dbInfo) || (addresses.secondaryAddress.present() && addressInDbAndPrimaryDc(addresses.secondaryAddress.get(), dbInfo)); } @@ -723,10 +723,10 @@ TEST_CASE("/fdbserver/worker/addressInDbAndPrimaryDc") { } // namespace // The actor that actively monitors the health of local and peer servers, and reports anomaly to the cluster controller. -ACTOR Future healthMonitor(Reference>> ccInterface, +ACTOR Future healthMonitor(Reference> const> ccInterface, WorkerInterface interf, LocalityData locality, - Reference> dbInfo) { + Reference const> dbInfo) { loop { Future nextHealthCheckDelay = Never(); if (dbInfo->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS && @@ -1212,7 +1212,7 @@ struct SharedLogsValue { }; ACTOR Future workerServer(Reference connFile, - Reference>> ccInterface, + Reference> const> ccInterface, LocalityData locality, Reference> asyncPriorityInfo, ProcessClass initialClass, From 52a64eb04be8c77c1de2ee943592de09d4259d80 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sat, 14 Aug 2021 17:41:57 -0700 Subject: [PATCH 36/63] Improve const-correctness for AsyncVar references --- fdbserver/TesterInterface.actor.h | 2 +- fdbserver/WorkerInterface.actor.h | 5 +++-- fdbserver/tester.actor.cpp | 8 ++++---- fdbserver/worker.actor.cpp | 18 +++++++++--------- fdbserver/workloads/ReadWrite.actor.cpp | 2 +- fdbserver/workloads/workloads.actor.h | 2 +- 6 files changed, 19 insertions(+), 18 deletions(-) diff --git a/fdbserver/TesterInterface.actor.h b/fdbserver/TesterInterface.actor.h index 8320cc566b..3874ff134d 100644 --- a/fdbserver/TesterInterface.actor.h +++ b/fdbserver/TesterInterface.actor.h @@ -114,7 +114,7 @@ struct TesterInterface { ACTOR Future testerServerCore(TesterInterface interf, Reference ccf, - Reference> serverDBInfo, + Reference const> serverDBInfo, LocalityData locality); enum test_location_t { TEST_HERE, TEST_ON_SERVERS, TEST_ON_TESTERS }; diff --git a/fdbserver/WorkerInterface.actor.h b/fdbserver/WorkerInterface.actor.h index b1ee9e1728..7227172b1f 100644 --- a/fdbserver/WorkerInterface.actor.h +++ b/fdbserver/WorkerInterface.actor.h @@ -880,8 +880,9 @@ class Database openDBOnServer(Reference const> const& db, TaskPriority taskID = TaskPriority::DefaultEndpoint, LockAware = LockAware::False, EnableLocalityLoadBalance = EnableLocalityLoadBalance::True); -ACTOR Future extractClusterInterface(Reference>> a, - Reference>> b); +ACTOR Future extractClusterInterface( + Reference> const> in, + Reference>> out); ACTOR Future fdbd(Reference ccf, LocalityData localities, diff --git a/fdbserver/tester.actor.cpp b/fdbserver/tester.actor.cpp index 33cce419de..a4f382cb18 100644 --- a/fdbserver/tester.actor.cpp +++ b/fdbserver/tester.actor.cpp @@ -315,7 +315,7 @@ struct CompoundWorkload : TestWorkload { TestWorkload* getWorkloadIface(WorkloadRequest work, VectorRef options, - Reference> dbInfo) { + Reference const> dbInfo) { Value testName = getOption(options, LiteralStringRef("testName"), LiteralStringRef("no-test-specified")); WorkloadContext wcx; wcx.clientId = work.clientId; @@ -350,7 +350,7 @@ TestWorkload* getWorkloadIface(WorkloadRequest work, return workload; } -TestWorkload* getWorkloadIface(WorkloadRequest work, Reference> dbInfo) { +TestWorkload* getWorkloadIface(WorkloadRequest work, Reference const> dbInfo) { if (work.options.size() < 1) { TraceEvent(SevError, "TestCreationError").detail("Reason", "No options provided"); fprintf(stderr, "ERROR: No options were provided for workload.\n"); @@ -602,7 +602,7 @@ ACTOR Future runWorkloadAsync(Database cx, ACTOR Future testerServerWorkload(WorkloadRequest work, Reference ccf, - Reference> dbInfo, + Reference const> dbInfo, LocalityData locality) { state WorkloadInterface workIface; state bool replied = false; @@ -661,7 +661,7 @@ ACTOR Future testerServerWorkload(WorkloadRequest work, ACTOR Future testerServerCore(TesterInterface interf, Reference ccf, - Reference> dbInfo, + Reference const> dbInfo, LocalityData locality) { state PromiseStream> addWorkload; state Future workerFatalError = actorCollection(addWorkload.getFuture()); diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index f35cdd1aa6..1933858820 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -959,7 +959,7 @@ ACTOR Future storageServerRollbackRebooter(std::set> db, + Reference const> db, std::string folder, ActorCollection* filesClosed, int64_t memoryLimit, @@ -1006,7 +1006,7 @@ ACTOR Future storageServerRollbackRebooter(std::set storageCacheRollbackRebooter(Future prevStorageCache, UID id, LocalityData locality, - Reference> db) { + Reference const> db) { loop { ErrorOr e = wait(errorOr(prevStorageCache)); if (!e.isError()) { @@ -2044,14 +2044,14 @@ ACTOR Future workerServer(Reference connFile, } } -ACTOR Future extractClusterInterface(Reference>> a, - Reference>> b) { +ACTOR Future extractClusterInterface(Reference> const> in, + Reference>> out) { loop { - if (a->get().present()) - b->set(a->get().get().clientInterface); + if (in->get().present()) + out->set(in->get().get().clientInterface); else - b->set(Optional()); - wait(a->onChange()); + out->set(Optional()); + wait(in->onChange()); } } @@ -2086,7 +2086,7 @@ ACTOR Future printTimeout() { return Void(); } -ACTOR Future printOnFirstConnected(Reference>> ci) { +ACTOR Future printOnFirstConnected(Reference> const> ci) { state Future timeoutFuture = printTimeout(); loop { choose { diff --git a/fdbserver/workloads/ReadWrite.actor.cpp b/fdbserver/workloads/ReadWrite.actor.cpp index c1ef14fc4c..edd2c3cfec 100644 --- a/fdbserver/workloads/ReadWrite.actor.cpp +++ b/fdbserver/workloads/ReadWrite.actor.cpp @@ -224,7 +224,7 @@ struct ReadWriteWorkload : KVWorkload { Future setup(Database const& cx) override { return _setup(cx, this); } Future start(Database const& cx) override { return _start(cx, this); } - ACTOR static Future traceDumpWorkers(Reference> db) { + ACTOR static Future traceDumpWorkers(Reference const> db) { try { loop { choose { diff --git a/fdbserver/workloads/workloads.actor.h b/fdbserver/workloads/workloads.actor.h index 702a408968..2ad0ba0134 100644 --- a/fdbserver/workloads/workloads.actor.h +++ b/fdbserver/workloads/workloads.actor.h @@ -49,7 +49,7 @@ struct WorkloadContext { Standalone> options; int clientId, clientCount; int64_t sharedRandomNumber; - Reference> dbInfo; + Reference const> dbInfo; WorkloadContext(); WorkloadContext(const WorkloadContext&); From 925c4a935de76800356e13393068b1cd4ad44bae Mon Sep 17 00:00:00 2001 From: hao fu Date: Mon, 16 Aug 2021 08:29:11 -0700 Subject: [PATCH 37/63] Log size of locationCache in client We observed that multi-threaded client has more memory usage, and we would like to understand it better. One suspect is that this shard-to-SS cache is playing a role, logging the size of it to get better vision. --- fdbclient/NativeAPI.actor.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index ebca9a7976..e477b54b5f 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -373,7 +373,8 @@ void traceTSSErrors(const char* name, UID tssId, const std::unordered_map databaseLogger(DatabaseContext* cx) { state double lastLogged = 0; loop { - wait(delay(CLIENT_KNOBS->SYSTEM_MONITOR_INTERVAL, TaskPriority::FlushTrace)); + wait(delay(CLIENT_KNOBS->SYSTEM_MONITOR_INTERVAL, TaskPriority::FlushTrace)); + TraceEvent ev("TransactionMetrics", cx->dbId); ev.detail("Elapsed", (lastLogged == 0) ? 0 : now() - lastLogged) @@ -384,6 +385,7 @@ ACTOR Future databaseLogger(DatabaseContext* cx) { cx->cc.logToTraceEvent(ev); + ev.detail("LocationCacheEntryCount", cx->locationCache.size()); ev.detail("MeanLatency", cx->latencies.mean()) .detail("MedianLatency", cx->latencies.median()) .detail("Latency90", cx->latencies.percentile(0.90)) From 04d33d3cba0a2b8811ff2ff629ee30b164558136 Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Mon, 16 Aug 2021 14:30:05 -0700 Subject: [PATCH 38/63] Remove anonymous namespace in PaxosConfigTransaction.actor.cpp --- fdbclient/PaxosConfigTransaction.actor.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/fdbclient/PaxosConfigTransaction.actor.cpp b/fdbclient/PaxosConfigTransaction.actor.cpp index ba47b949d1..a41e755bfd 100644 --- a/fdbclient/PaxosConfigTransaction.actor.cpp +++ b/fdbclient/PaxosConfigTransaction.actor.cpp @@ -22,8 +22,6 @@ #include "fdbclient/PaxosConfigTransaction.h" #include "flow/actorcompiler.h" // must be last include -namespace { - // TODO: Some replicas may reply after quorum has already been achieved, and we may want to add them to the readReplicas // list class GetGenerationQuorum { @@ -72,8 +70,6 @@ public: Optional getLastSeenLiveVersion() const { return lastSeenLiveVersion; } }; -} // namespace - class PaxosConfigTransactionImpl { ConfigTransactionCommitRequest toCommit; Future getGenerationFuture; From d12bda94ae3d5593c21a49a01c95e2f854cc9b0d Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Mon, 16 Aug 2021 16:33:20 -0700 Subject: [PATCH 39/63] disable trace log --- fdbserver/LogSystemPeekCursor.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/LogSystemPeekCursor.actor.cpp b/fdbserver/LogSystemPeekCursor.actor.cpp index 4ab5aaf56f..429e803c02 100644 --- a/fdbserver/LogSystemPeekCursor.actor.cpp +++ b/fdbserver/LogSystemPeekCursor.actor.cpp @@ -367,7 +367,7 @@ ACTOR Future serverPeekStreamGetMore(ILogSystem::ServerPeekCursor* self, T } } } catch (Error& e) { - TraceEvent(SevDebug, "SPC_GetMoreB_Error", self->randomID).error(e, true); + DisabledTraceEvent(SevDebug, "SPC_GetMoreB_Error", self->randomID).error(e, true); if (e.code() == error_code_connection_failed || e.code() == error_code_operation_obsolete) { // NOTE: delay in order to avoid the endless retry loop block other tasks self->peekReplyStream.reset(); From cc18cc742c268600bf8e78e2ee61b981f3bc9ae8 Mon Sep 17 00:00:00 2001 From: Chaoguang Lin Date: Tue, 17 Aug 2021 10:14:39 -0700 Subject: [PATCH 40/63] Add fdbcli external client tests --- bindings/python/CMakeLists.txt | 22 +++++++++++++++++++++- bindings/python/tests/fdbcli_tests.py | 11 +++++++++-- 2 files changed, 30 insertions(+), 3 deletions(-) diff --git a/bindings/python/CMakeLists.txt b/bindings/python/CMakeLists.txt index 2fe304a0dc..802514c485 100644 --- a/bindings/python/CMakeLists.txt +++ b/bindings/python/CMakeLists.txt @@ -77,7 +77,7 @@ add_dependencies(packages python_package) if (NOT WIN32 AND NOT OPEN_FOR_IDE) add_fdbclient_test( - NAME fdbcli_tests + NAME single_process_fdbcli_tests COMMAND ${CMAKE_SOURCE_DIR}/bindings/python/tests/fdbcli_tests.py ${CMAKE_BINARY_DIR}/bin/fdbcli @CLUSTER_FILE@ @@ -92,4 +92,24 @@ if (NOT WIN32 AND NOT OPEN_FOR_IDE) @CLUSTER_FILE@ 5 ) + if (TARGET external_client) # external_client copies fdb_c to bindings/c/libfdb_c.so + add_fdbclient_test( + NAME single_process_external_client_fdbcli_tests + COMMAND ${CMAKE_SOURCE_DIR}/bindings/python/tests/fdbcli_tests.py + ${CMAKE_BINARY_DIR}/bin/fdbcli + @CLUSTER_FILE@ + 1 + ${CMAKE_BINARY_DIR}/bindings/c/libfdb_c.so + ) + add_fdbclient_test( + NAME multi_process_external_client_fdbcli_tests + PROCESS_NUMBER 5 + TEST_TIMEOUT 120 # The test can take near to 1 minutes sometime, set timeout to 2 minutes to be safe + COMMAND ${CMAKE_SOURCE_DIR}/bindings/python/tests/fdbcli_tests.py + ${CMAKE_BINARY_DIR}/bin/fdbcli + @CLUSTER_FILE@ + 5 + ${CMAKE_BINARY_DIR}/bindings/c/libfdb_c.so + ) + endif() endif() diff --git a/bindings/python/tests/fdbcli_tests.py b/bindings/python/tests/fdbcli_tests.py index 0529443923..25511b7b33 100755 --- a/bindings/python/tests/fdbcli_tests.py +++ b/bindings/python/tests/fdbcli_tests.py @@ -1,6 +1,7 @@ #!/usr/bin/env python3 import sys +import os import subprocess import logging import functools @@ -450,8 +451,14 @@ def wait_for_database_available(logger): time.sleep(1) if __name__ == '__main__': - # fdbcli_tests.py - assert len(sys.argv) == 4, "Please pass arguments: " + # fdbcli_tests.py [external_client_library_path] + assert len(sys.argv) == 4 or len(sys.argv) == 5, "Please pass arguments: [external_client_library_path]" + # set external client library + if len(sys.argv) == 5: + external_client_library_path = sys.argv[4] + # disable local client and use the external client library + os.environ['FDB_NETWORK_OPTION_DISABLE_LOCAL_CLIENT'] = '' + os.environ['FDB_NETWORK_OPTION_EXTERNAL_CLIENT_LIBRARY'] = external_client_library_path # shell command template command_template = [sys.argv[1], '-C', sys.argv[2], '--exec'] # tests for fdbcli commands From 775ac3e27c5734d55a663c06201a50004b6d15ff Mon Sep 17 00:00:00 2001 From: Chaoguang Lin Date: Tue, 17 Aug 2021 10:15:35 -0700 Subject: [PATCH 41/63] Format fdbcli_tests.py file --- bindings/python/tests/fdbcli_tests.py | 71 +++++++++++++++++---------- 1 file changed, 46 insertions(+), 25 deletions(-) diff --git a/bindings/python/tests/fdbcli_tests.py b/bindings/python/tests/fdbcli_tests.py index 25511b7b33..11f7c191eb 100755 --- a/bindings/python/tests/fdbcli_tests.py +++ b/bindings/python/tests/fdbcli_tests.py @@ -9,6 +9,7 @@ import json import time import random + def enable_logging(level=logging.ERROR): """Enable logging in the function with the specified logging level @@ -17,7 +18,7 @@ def enable_logging(level=logging.ERROR): """ def func_decorator(func): @functools.wraps(func) - def wrapper(*args,**kwargs): + def wrapper(*args, **kwargs): # initialize logger logger = logging.getLogger(func.__name__) logger.setLevel(level) @@ -33,6 +34,7 @@ def enable_logging(level=logging.ERROR): return wrapper return func_decorator + def run_fdbcli_command(*args): """run the fdbcli statement: fdbcli --exec ' ... '. @@ -42,6 +44,7 @@ def run_fdbcli_command(*args): commands = command_template + ["{}".format(' '.join(args))] return subprocess.run(commands, stdout=subprocess.PIPE).stdout.decode('utf-8').strip() + def run_fdbcli_command_and_get_error(*args): """run the fdbcli statement: fdbcli --exec ' ... '. @@ -51,6 +54,7 @@ def run_fdbcli_command_and_get_error(*args): commands = command_template + ["{}".format(' '.join(args))] return subprocess.run(commands, stdout=subprocess.PIPE, stderr=subprocess.PIPE).stderr.decode('utf-8').strip() + @enable_logging() def advanceversion(logger): # get current read version @@ -73,6 +77,7 @@ def advanceversion(logger): logger.debug("Read version: {}".format(version4)) assert version4 >= version3 + @enable_logging() def maintenance(logger): # expected fdbcli output when running 'maintenance' while there's no ongoing maintenance @@ -95,6 +100,7 @@ def maintenance(logger): output3 = run_fdbcli_command('maintenance') assert output3 == no_maintenance_output + @enable_logging() def setclass(logger): output1 = run_fdbcli_command('setclass') @@ -109,11 +115,11 @@ def setclass(logger): # check class source assert 'command_line' in class_type_line_1 # set class to a random valid type - class_types = ['storage', 'storage', 'transaction', 'resolution', - 'commit_proxy', 'grv_proxy', 'master', 'stateless', 'log', - 'router', 'cluster_controller', 'fast_restore', 'data_distributor', - 'coordinator', 'ratekeeper', 'storage_cache', 'backup' - ] + class_types = ['storage', 'storage', 'transaction', 'resolution', + 'commit_proxy', 'grv_proxy', 'master', 'stateless', 'log', + 'router', 'cluster_controller', 'fast_restore', 'data_distributor', + 'coordinator', 'ratekeeper', 'storage_cache', 'backup' + ] random_class_type = random.choice(class_types) logger.debug("Change to type: {}".format(random_class_type)) run_fdbcli_command('setclass', network_address, random_class_type) @@ -135,6 +141,7 @@ def setclass(logger): logger.debug(class_type_line_3) assert class_type_line_3 == class_type_line_1 + @enable_logging() def lockAndUnlock(logger): # lock an unlocked database, should be successful @@ -149,7 +156,7 @@ def lockAndUnlock(logger): output2 = run_fdbcli_command_and_get_error("lock") assert output2 == 'ERROR: Database is locked (1038)' # unlock the database - process = subprocess.Popen(command_template + ['unlock ' + lock_uid], stdin = subprocess.PIPE, stdout = subprocess.PIPE) + process = subprocess.Popen(command_template + ['unlock ' + lock_uid], stdin=subprocess.PIPE, stdout=subprocess.PIPE) line1 = process.stdout.readline() # The randome passphrease we need to confirm to proceed the unlocking line2 = process.stdout.readline() @@ -160,6 +167,7 @@ def lockAndUnlock(logger): assert output3.decode('utf-8').strip() == 'Database unlocked.' assert not get_value_from_status_json(True, 'cluster', 'database_lock_state', 'locked') + @enable_logging() def kill(logger): output1 = run_fdbcli_command('kill') @@ -169,11 +177,11 @@ def kill(logger): address = lines[1] logger.debug("Address: {}".format(address)) old_generation = get_value_from_status_json(False, 'cluster', 'generation') - # This is currently an issue with fdbcli, - # where you need to first run 'kill' to initialize processes' list + # This is currently an issue with fdbcli, + # where you need to first run 'kill' to initialize processes' list # and then specify the certain process to kill - process = subprocess.Popen(command_template[:-1], stdin = subprocess.PIPE, stdout = subprocess.PIPE) - # + process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE) + # output2, err = process.communicate(input='kill; kill {}\n'.format(address).encode()) logger.debug(output2) # wait for a second for the cluster recovery @@ -182,6 +190,7 @@ def kill(logger): logger.debug("Old: {}, New: {}".format(old_generation, new_generation)) assert new_generation > old_generation + @enable_logging() def suspend(logger): output1 = run_fdbcli_command('suspend') @@ -201,7 +210,7 @@ def suspend(logger): assert len(pinfo) == 1 pid = pinfo[0].split(' ')[0] logger.debug("Pid: {}".format(pid)) - process = subprocess.Popen(command_template[:-1], stdin = subprocess.PIPE, stdout = subprocess.PIPE) + process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE) # suspend the process for enough long time output2, err = process.communicate(input='suspend; suspend 3600 {}\n'.format(address).encode()) # the cluster should be unavailable after the only process being suspended @@ -214,7 +223,7 @@ def suspend(logger): kill_output = subprocess.check_output(['kill', pid]).decode().strip() logger.debug("Kill result: {}".format(kill_output)) # The process should come back after a few time - duration = 0 # seconds we already wait + duration = 0 # seconds we already wait while not get_value_from_status_json(False, 'client', 'database_status', 'available') and duration < 60: logger.debug("Sleep for 1 second to wait cluster recovery") time.sleep(1) @@ -222,6 +231,7 @@ def suspend(logger): # at most after 60 seconds, the cluster should be available assert get_value_from_status_json(False, 'client', 'database_status', 'available') + def get_value_from_status_json(retry, *args): while True: result = json.loads(run_fdbcli_command('status', 'json')) @@ -230,9 +240,10 @@ def get_value_from_status_json(retry, *args): for arg in args: assert arg in result result = result[arg] - + return result + @enable_logging() def consistencycheck(logger): consistency_check_on_output = 'ConsistencyCheck is on' @@ -246,6 +257,7 @@ def consistencycheck(logger): output3 = run_fdbcli_command('consistencycheck') assert output3 == consistency_check_on_output + @enable_logging() def cache_range(logger): # this command is currently experimental @@ -253,6 +265,7 @@ def cache_range(logger): run_fdbcli_command('cache_range', 'set', 'a', 'b') run_fdbcli_command('cache_range', 'clear', 'a', 'b') + @enable_logging() def datadistribution(logger): output1 = run_fdbcli_command('datadistribution', 'off') @@ -272,6 +285,7 @@ def datadistribution(logger): assert output6 == 'Data distribution is enabled for rebalance.' time.sleep(1) + @enable_logging() def transaction(logger): """This test will cover the transaction related fdbcli commands. @@ -281,7 +295,7 @@ def transaction(logger): """ err1 = run_fdbcli_command_and_get_error('set', 'key', 'value') assert err1 == 'ERROR: writemode must be enabled to set or clear keys in the database.' - process = subprocess.Popen(command_template[:-1], stdin = subprocess.PIPE, stdout = subprocess.PIPE) + process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE) transaction_flow = ['writemode on', 'begin', 'getversion', 'set key value', 'get key', 'commit'] output1, _ = process.communicate(input='\n'.join(transaction_flow).encode()) # split the output into lines @@ -300,13 +314,13 @@ def transaction(logger): output2 = run_fdbcli_command('get', 'key') assert output2 == "`key' is `value'" # test rollback and read-your-write behavior - process = subprocess.Popen(command_template[:-1], stdin = subprocess.PIPE, stdout = subprocess.PIPE) + process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE) transaction_flow = [ - 'writemode on', 'begin', 'getrange a z', + 'writemode on', 'begin', 'getrange a z', 'clear key', 'get key', # 'option on READ_YOUR_WRITES_DISABLE', 'get key', 'rollback' - ] + ] output3, _ = process.communicate(input='\n'.join(transaction_flow).encode()) lines = list(filter(len, output3.decode().split('\n')))[-5:] # lines[0] == "Transaction started" and lines[1] == 'Range limited to 25 keys' @@ -317,13 +331,13 @@ def transaction(logger): output4 = run_fdbcli_command('get', 'key') assert output4 == "`key' is `value'" # test read_your_write_disable option and clear the inserted key - process = subprocess.Popen(command_template[:-1], stdin = subprocess.PIPE, stdout = subprocess.PIPE) + process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE) transaction_flow = [ 'writemode on', 'begin', 'option on READ_YOUR_WRITES_DISABLE', 'clear key', 'get key', 'commit' - ] + ] output6, _ = process.communicate(input='\n'.join(transaction_flow).encode()) lines = list(filter(len, output6.decode().split('\n')))[-4:] assert lines[1] == 'Option enabled for current transaction' @@ -333,6 +347,7 @@ def transaction(logger): output7 = run_fdbcli_command('get', 'key') assert output7 == "`key': not found" + def get_fdb_process_addresses(logger): # get all processes' network addresses output = run_fdbcli_command('kill') @@ -342,6 +357,7 @@ def get_fdb_process_addresses(logger): assert len(addresses) == process_number return addresses + @enable_logging(logging.DEBUG) def coordinators(logger): # we should only have one coordinator for now @@ -369,6 +385,7 @@ def coordinators(logger): assert len(get_value_from_status_json(True, 'client', 'coordinators', 'coordinators')) == 1 wait_for_database_available(logger) + @enable_logging(logging.DEBUG) def exclude(logger): # get all processes' network addresses @@ -381,7 +398,7 @@ def exclude(logger): # randomly pick one and exclude the process excluded_address = random.choice(addresses) # If we see "not enough space" error, use FORCE option to proceed - # this should be a safe operation as we do not need any storage space for the test + # this should be a safe operation as we do not need any storage space for the test force = False # sometimes we need to retry the exclude while True: @@ -418,6 +435,8 @@ def exclude(logger): wait_for_database_available(logger) # read the system key 'k', need to enable the option first + + def read_system_key(k): output = run_fdbcli_command('option', 'on', 'READ_SYSTEM_KEYS;', 'get', k) if 'is' not in output: @@ -426,6 +445,7 @@ def read_system_key(k): _, value = output.split(' is ') return value + @enable_logging() def throttle(logger): # no throttled tags at the beginning @@ -443,6 +463,7 @@ def throttle(logger): assert enable_flag == "`0'" # TODO : test manual throttling, not easy to do now + def wait_for_database_available(logger): # sometimes the change takes some time to have effect and the database can be unavailable at that time # this is to wait until the database is available again @@ -450,9 +471,11 @@ def wait_for_database_available(logger): logger.debug("Database unavailable for now, wait for one second") time.sleep(1) + if __name__ == '__main__': # fdbcli_tests.py [external_client_library_path] - assert len(sys.argv) == 4 or len(sys.argv) == 5, "Please pass arguments: [external_client_library_path]" + assert len(sys.argv) == 4 or len( + sys.argv) == 5, "Please pass arguments: [external_client_library_path]" # set external client library if len(sys.argv) == 5: external_client_library_path = sys.argv[4] @@ -466,7 +489,7 @@ if __name__ == '__main__': process_number = int(sys.argv[3]) if process_number == 1: # TODO: disable for now, the change can cause the database unavailable - #advanceversion() + # advanceversion() cache_range() consistencycheck() datadistribution() @@ -481,5 +504,3 @@ if __name__ == '__main__': assert process_number > 1, "Process number should be positive" coordinators() exclude() - - From 179c313e08ea26d28bf16b4bc360ebc76a3c635c Mon Sep 17 00:00:00 2001 From: Chaoguang Lin Date: Tue, 17 Aug 2021 20:02:25 +0000 Subject: [PATCH 42/63] Use DatabaseContext instead of Database in template functions --- fdbclient/DatabaseContext.h | 4 ++++ fdbclient/NativeAPI.actor.cpp | 4 ++++ fdbclient/TagThrottle.actor.h | 3 +++ fdbserver/Ratekeeper.actor.cpp | 11 ++++++----- fdbserver/workloads/TagThrottleApi.actor.cpp | 12 ++++++------ fdbserver/workloads/WriteTagThrottling.actor.cpp | 4 ++-- 6 files changed, 25 insertions(+), 13 deletions(-) diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index 8d2cbf21d8..5ee6d9a7a0 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -437,6 +437,10 @@ public: // Requests to the storage server will no longer be duplicated to its pair TSS. void removeTssMapping(StorageServerInterface const& ssi); + // Used for template code + using TransactionT = ReadYourWritesTransaction; + Reference createTransaction(); + private: std::unordered_map> watchMap; }; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index c2c8ecd6d5..a1b808293a 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -6534,3 +6534,7 @@ ACTOR Future setPerpetualStorageWiggle(Database cx, bool enable, LockAware } return Void(); } + +Reference DatabaseContext::createTransaction() { + return makeReference(Database(Reference::addRef(this))); +} diff --git a/fdbclient/TagThrottle.actor.h b/fdbclient/TagThrottle.actor.h index 9a471a266a..522d37f7b0 100644 --- a/fdbclient/TagThrottle.actor.h +++ b/fdbclient/TagThrottle.actor.h @@ -244,6 +244,9 @@ extern const KeyRef tagThrottleCountKey; namespace ThrottleApi { +// The template functions can be called with Native API like DatabaseContext, Transaction/ReadYourWritesTransaction +// or using IClientAPI like IDatabase, ITransaction + ACTOR template Future getValidAutoEnabled(Reference tr) { state bool result; diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index 847862803c..a0de0c8c95 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -23,6 +23,7 @@ #include "fdbrpc/FailureMonitor.h" #include "fdbrpc/Smoother.h" #include "fdbrpc/simulator.h" +#include "fdbclient/DatabaseContext.h" #include "fdbclient/ReadYourWrites.h" #include "fdbclient/TagThrottle.actor.h" #include "fdbserver/Knobs.h" @@ -527,7 +528,7 @@ struct RatekeeperLimits { context(context) {} }; -struct GrvProxyInfo { +struct GRVProxyInfo { int64_t totalTransactions; int64_t batchTransactions; uint64_t lastThrottledTagChangeId; @@ -535,7 +536,7 @@ struct GrvProxyInfo { double lastUpdateTime; double lastTagPushTime; - GrvProxyInfo() + GRVProxyInfo() : totalTransactions(0), batchTransactions(0), lastThrottledTagChangeId(0), lastUpdateTime(0), lastTagPushTime(0) { } }; @@ -547,7 +548,7 @@ struct RatekeeperData { Map storageQueueInfo; Map tlogQueueInfo; - std::map grvProxyInfo; + std::map grvProxyInfo; Smoother smoothReleasedTransactions, smoothBatchReleasedTransactions, smoothTotalDurableBytes; HealthMetrics healthMetrics; DatabaseConfiguration configuration; @@ -597,7 +598,7 @@ struct RatekeeperData { autoThrottlingEnabled(false) { expiredTagThrottleCleanup = recurring( [this]() { - Reference db = makeReference(this->db); + Reference db = Reference::addRef(this->db.getPtr()); ThrottleApi::expire(db); }, SERVER_KNOBS->TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL); @@ -946,7 +947,7 @@ void tryAutoThrottleTag(RatekeeperData* self, TagSet tags; tags.addTag(tag); - Reference db = makeReference(self->db); + Reference db = Reference::addRef(self->db.getPtr()); self->addActor.send(ThrottleApi::throttleTags(db, tags, clientRate.get(), diff --git a/fdbserver/workloads/TagThrottleApi.actor.cpp b/fdbserver/workloads/TagThrottleApi.actor.cpp index 2a4094989a..97d0207a89 100644 --- a/fdbserver/workloads/TagThrottleApi.actor.cpp +++ b/fdbserver/workloads/TagThrottleApi.actor.cpp @@ -78,7 +78,7 @@ struct TagThrottleApiWorkload : TestWorkload { state TransactionPriority priority = deterministicRandom()->randomChoice(allTransactionPriorities); state double rate = deterministicRandom()->random01() * 20; state double duration = 1 + deterministicRandom()->random01() * 19; - state Reference db = makeReference(cx); + state Reference db = Reference::addRef(cx.getPtr()); TagSet tagSet; tagSet.addTag(tag); @@ -138,7 +138,7 @@ struct TagThrottleApiWorkload : TestWorkload { } } - state Reference db = makeReference(cx); + state Reference db = Reference::addRef(cx.getPtr()); bool removed = wait(ThrottleApi::unthrottleTags(db, tagSet, throttleType, priority)); if (removed) { ASSERT(erased || !throttleType.present() || throttleType.get() == TagThrottleType::AUTO); @@ -154,7 +154,7 @@ struct TagThrottleApiWorkload : TestWorkload { Database cx, std::map, TagThrottleInfo> const* manuallyThrottledTags) { - state Reference db = makeReference(cx); + state Reference db = Reference::addRef(cx.getPtr()); std::vector tags = wait(ThrottleApi::getThrottledTags(db, CLIENT_KNOBS->TOO_MANY)); int manualThrottledTags = 0; @@ -188,7 +188,7 @@ struct TagThrottleApiWorkload : TestWorkload { } ACTOR Future getRecommendedTags(TagThrottleApiWorkload* self, Database cx) { - state Reference db = makeReference(cx); + state Reference db = Reference::addRef(cx.getPtr()); std::vector tags = wait(ThrottleApi::getRecommendedTags(db, CLIENT_KNOBS->TOO_MANY)); for (auto& tag : tags) { @@ -205,7 +205,7 @@ struct TagThrottleApiWorkload : TestWorkload { deterministicRandom()->coinflip() ? Optional() : deterministicRandom()->randomChoice(allTransactionPriorities); - state Reference db = makeReference(cx); + state Reference db = Reference::addRef(cx.getPtr()); bool unthrottled = wait(ThrottleApi::unthrottleAll(db, throttleType, priority)); if (!throttleType.present() || throttleType.get() == TagThrottleType::MANUAL) { bool unthrottleExpected = false; @@ -233,7 +233,7 @@ struct TagThrottleApiWorkload : TestWorkload { } ACTOR Future enableAutoThrottling(TagThrottleApiWorkload* self, Database cx) { - state Reference db = makeReference(cx); + state Reference db = Reference::addRef(cx.getPtr()); if (deterministicRandom()->coinflip()) { wait(ThrottleApi::enableAuto(db, true)); self->autoThrottleEnabled = true; diff --git a/fdbserver/workloads/WriteTagThrottling.actor.cpp b/fdbserver/workloads/WriteTagThrottling.actor.cpp index a6ff0f5d7a..a9170f122c 100644 --- a/fdbserver/workloads/WriteTagThrottling.actor.cpp +++ b/fdbserver/workloads/WriteTagThrottling.actor.cpp @@ -94,7 +94,7 @@ struct WriteTagThrottlingWorkload : KVWorkload { std::string description() const override { return WriteTagThrottlingWorkload::NAME; } ACTOR static Future _setup(Database cx, WriteTagThrottlingWorkload* self) { - state Reference db = makeReference(cx); + state Reference db = Reference::addRef(cx.getPtr()); ASSERT(CLIENT_KNOBS->MAX_TAGS_PER_TRANSACTION >= MIN_TAGS_PER_TRANSACTION && CLIENT_KNOBS->MAX_TRANSACTION_TAG_LENGTH >= MIN_TRANSACTION_TAG_LENGTH); if (self->populateData) { @@ -307,7 +307,7 @@ struct WriteTagThrottlingWorkload : KVWorkload { } ACTOR static Future throttledTagUpdater(Database cx, WriteTagThrottlingWorkload* self) { state std::vector tags; - state Reference db = makeReference(cx); + state Reference db = Reference::addRef(cx.getPtr()); loop { wait(delay(1.0)); wait(store(tags, ThrottleApi::getThrottledTags(db, CLIENT_KNOBS->TOO_MANY, true))); From 53e83cf653249d984aaeef24090be34ffdf7095c Mon Sep 17 00:00:00 2001 From: Chaoguang Lin Date: Tue, 17 Aug 2021 20:10:43 +0000 Subject: [PATCH 43/63] Remove change to Database --- fdbclient/DatabaseContext.h | 2 +- fdbclient/NativeAPI.actor.cpp | 4 ---- fdbclient/NativeAPI.actor.h | 5 +---- 3 files changed, 2 insertions(+), 9 deletions(-) diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index 5ee6d9a7a0..a2754965f1 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -437,7 +437,7 @@ public: // Requests to the storage server will no longer be duplicated to its pair TSS. void removeTssMapping(StorageServerInterface const& ssi); - // Used for template code + // used in template functions to create a transaction using TransactionT = ReadYourWritesTransaction; Reference createTransaction(); diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index a1b808293a..4efac55415 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -1773,10 +1773,6 @@ Database Database::createDatabase(std::string connFileName, return Database::createDatabase(rccf, apiVersion, internal, clientLocality); } -Reference Database::createTransaction() { - return makeReference(*this); -} - Reference DatabaseContext::getWatchMetadata(KeyRef key) const { const auto it = watchMap.find(key); if (it == watchMap.end()) diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h index 02139aa73b..e920f54155 100644 --- a/fdbclient/NativeAPI.actor.h +++ b/fdbclient/NativeAPI.actor.h @@ -78,7 +78,7 @@ struct NetworkOptions { class ReadYourWritesTransaction; // workaround cyclic dependency -class Database final : public ThreadSafeReferenceCounted { +class Database { public: enum { API_VERSION_LATEST = -1 }; @@ -111,9 +111,6 @@ public: const UniqueOrderedOptionList& getTransactionDefaults() const; - using TransactionT = ReadYourWritesTransaction; - Reference createTransaction(); - private: Reference db; }; From 7ac1d17bcb5b6c366666b6473bf4299b7f14b5ab Mon Sep 17 00:00:00 2001 From: Chaoguang Lin Date: Tue, 17 Aug 2021 21:05:57 +0000 Subject: [PATCH 44/63] Add a local namespace in Ratekeeper.actor.cpp to differenitate name GrvProxyInfo --- fdbserver/Ratekeeper.actor.cpp | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index a0de0c8c95..39a9e9aa35 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -528,7 +528,10 @@ struct RatekeeperLimits { context(context) {} }; -struct GRVProxyInfo { +namespace RatekeeperActorCpp { + +// Differentiate from GrvProxyInfo in DatabaseContext.h +struct GrvProxyInfo { int64_t totalTransactions; int64_t batchTransactions; uint64_t lastThrottledTagChangeId; @@ -536,11 +539,13 @@ struct GRVProxyInfo { double lastUpdateTime; double lastTagPushTime; - GRVProxyInfo() + GrvProxyInfo() : totalTransactions(0), batchTransactions(0), lastThrottledTagChangeId(0), lastUpdateTime(0), lastTagPushTime(0) { } }; +} + struct RatekeeperData { UID id; Database db; @@ -548,7 +553,7 @@ struct RatekeeperData { Map storageQueueInfo; Map tlogQueueInfo; - std::map grvProxyInfo; + std::map grvProxyInfo; Smoother smoothReleasedTransactions, smoothBatchReleasedTransactions, smoothTotalDurableBytes; HealthMetrics healthMetrics; DatabaseConfiguration configuration; From 2c101182296ca19450a0c439f664e53c5e79668f Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 17 Aug 2021 17:26:51 -0500 Subject: [PATCH 45/63] Force kill in killDatacenter didn't actually force kill always --- fdbrpc/sim2.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbrpc/sim2.actor.cpp b/fdbrpc/sim2.actor.cpp index 6a946d4981..17f97f4dbb 100644 --- a/fdbrpc/sim2.actor.cpp +++ b/fdbrpc/sim2.actor.cpp @@ -1900,7 +1900,7 @@ public: KillType ktResult, ktMin = kt; for (auto& datacenterMachine : datacenterMachines) { - if (deterministicRandom()->random01() < 0.99) { + if (deterministicRandom()->random01() < 0.99 || forceKill) { killMachine(datacenterMachine.first, kt, true, &ktResult); if (ktResult != kt) { TraceEvent(SevWarn, "KillDCFail") From 6aabd9a03ec30669c4d19410e6f0eb7793b750bf Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Tue, 17 Aug 2021 18:18:28 -0500 Subject: [PATCH 46/63] Adding FIXME for simulation issue --- fdbserver/workloads/KillRegion.actor.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/fdbserver/workloads/KillRegion.actor.cpp b/fdbserver/workloads/KillRegion.actor.cpp index 8da635e4f2..a8c823edea 100644 --- a/fdbserver/workloads/KillRegion.actor.cpp +++ b/fdbserver/workloads/KillRegion.actor.cpp @@ -84,6 +84,8 @@ struct KillRegionWorkload : TestWorkload { TraceEvent("ForceRecovery_Wait").log(); wait(delay(deterministicRandom()->random01() * self->testDuration)); + // FIXME: killDataCenter breaks simulation if forceKill=false, since some processes can survive and + // partially complete a recovery g_simulator.killDataCenter(LiteralStringRef("0"), deterministicRandom()->random01() < 0.5 ? ISimulator::KillInstantly : ISimulator::RebootAndDelete, From 977f8f494e9d3e05a9277e23569fa8f37b7dd401 Mon Sep 17 00:00:00 2001 From: Yao Xiao Date: Wed, 18 Aug 2021 01:26:55 -0700 Subject: [PATCH 47/63] Add version and timestamp to TimeKeeperCommit event. --- fdbserver/ClusterController.actor.cpp | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index c864a6bfc3..8b12eb518b 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -3938,15 +3938,7 @@ ACTOR Future timeKeeper(ClusterControllerData* self) { state Reference tr = makeReference(self->cx); loop { try { - if (!g_network->isSimulated()) { - // This is done to provide an arbitrary logged transaction every ~10s. - // FIXME: replace or augment this with logging on the proxy which tracks - // how long it is taking to hear responses from each other component. - - UID debugID = deterministicRandom()->randomUniqueID(); - TraceEvent("TimeKeeperCommit", debugID).log(); - tr->debugTransaction(debugID); - } + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::LOCK_AWARE); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); @@ -3959,7 +3951,15 @@ ACTOR Future timeKeeper(ClusterControllerData* self) { Version v = tr->getReadVersion().get(); int64_t currentTime = (int64_t)now(); versionMap.set(tr, currentTime, v); + if (!g_network->isSimulated()) { + // This is done to provide an arbitrary logged transaction every ~10s. + // FIXME: replace or augment this with logging on the proxy which tracks + // how long it is taking to hear responses from each other component. + UID debugID = deterministicRandom()->randomUniqueID(); + TraceEvent("TimeKeeperCommit", debugID).detail("Version", v).detail("Timestamp", currentTime).log(); + tr->debugTransaction(debugID); + } int64_t ttl = currentTime - SERVER_KNOBS->TIME_KEEPER_DELAY * SERVER_KNOBS->TIME_KEEPER_MAX_ENTRIES; if (ttl > 0) { versionMap.erase(tr, 0, ttl); From bce42671ffd4f248d5ecb1b72fe10e4586454c22 Mon Sep 17 00:00:00 2001 From: Mohamed Oulmahdi Date: Wed, 18 Aug 2021 16:05:12 +0200 Subject: [PATCH 48/63] Add IThreadPool test to cmake --- tests/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index 1a10052692..9b0aa0ce55 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -109,6 +109,7 @@ if(WITH_PYTHON) add_fdb_test(TEST_FILES pt.TXT IGNORE) add_fdb_test(TEST_FILES randomSelector.txt IGNORE) add_fdb_test(TEST_FILES selectorCorrectness.txt IGNORE) + add_fdb_test(TEST_FILES IThreadPool.txt IGNORE) add_fdb_test(TEST_FILES fast/AtomicBackupCorrectness.toml) add_fdb_test(TEST_FILES fast/AtomicBackupToDBCorrectness.toml) add_fdb_test(TEST_FILES fast/AtomicOps.toml) From 164ce4b33431cd059ef827dc747145d0a2127ac8 Mon Sep 17 00:00:00 2001 From: Mohamed Oulmahdi Date: Wed, 18 Aug 2021 15:46:20 +0200 Subject: [PATCH 49/63] Add Windows specific code for system time --- fdbclient/SpecialKeySpace.actor.cpp | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/fdbclient/SpecialKeySpace.actor.cpp b/fdbclient/SpecialKeySpace.actor.cpp index dd206411cc..793e694f84 100644 --- a/fdbclient/SpecialKeySpace.actor.cpp +++ b/fdbclient/SpecialKeySpace.actor.cpp @@ -1964,16 +1964,29 @@ void parse(StringRef& val, WaitState& w) { void parse(StringRef& val, time_t& t) { struct tm tm = { 0 }; +#ifdef _WIN32 + std::istringstream s(val.toString()); + s.imbue(std::locale(setlocale(LC_TIME, nullptr))); + s >> std::get_time(&tm, "%FT%T%z"); + if (s.fail()) { + throw std::invalid_argument("failed to parse ISO 8601 datetime"); + } + long timezone; + if (_get_timezone(&timezone) != 0) { + throw std::runtime_error("failed to convert ISO 8601 datetime"); + } + timezone = -timezone; +#else if (strptime(val.toString().c_str(), "%FT%T%z", &tm) == nullptr) { throw std::invalid_argument("failed to parse ISO 8601 datetime"); } - long timezone = tm.tm_gmtoff; t = timegm(&tm); if (t == -1) { throw std::runtime_error("failed to convert ISO 8601 datetime"); } t -= timezone; +#endif } void parse(StringRef& val, NetworkAddress& a) { From 3a9e7150f79722f780428daa123e963c0995257a Mon Sep 17 00:00:00 2001 From: Mohamed Oulmahdi Date: Wed, 18 Aug 2021 15:47:17 +0200 Subject: [PATCH 50/63] Add SIGUSR1 and SIGUSR2 macros for Windows --- fdbserver/SigStack.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/fdbserver/SigStack.cpp b/fdbserver/SigStack.cpp index 6ad74ddb63..f00ec5eebc 100644 --- a/fdbserver/SigStack.cpp +++ b/fdbserver/SigStack.cpp @@ -19,6 +19,11 @@ extern "C" void stackSignalHandler(int sig) { } } +#ifdef _WIN32 +#define SIGUSR1 10 +#define SIGUSR2 12 +#endif + void setupStackSignal() { std::signal(SIGUSR1, &stackSignalHandler); } From deaf302987ce4f329554a07b6af96fd07b59f4d7 Mon Sep 17 00:00:00 2001 From: Mohamed Oulmahdi Date: Wed, 18 Aug 2021 15:48:27 +0200 Subject: [PATCH 51/63] Remove deprecated Action parameter for Windows Installer --- packaging/msi/FDBInstaller.wxs.cmake | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/packaging/msi/FDBInstaller.wxs.cmake b/packaging/msi/FDBInstaller.wxs.cmake index 87b0bc40a8..90ea02a233 100644 --- a/packaging/msi/FDBInstaller.wxs.cmake +++ b/packaging/msi/FDBInstaller.wxs.cmake @@ -150,13 +150,13 @@ - + - + From 4b8dadfdfb70f2ec1de34aeba9533c63c9d56dbc Mon Sep 17 00:00:00 2001 From: Yao Xiao Date: Wed, 18 Aug 2021 11:46:57 -0700 Subject: [PATCH 52/63] Use Time instead of Timestamp. --- fdbserver/ClusterController.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 8b12eb518b..75305b8c61 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -3957,7 +3957,7 @@ ACTOR Future timeKeeper(ClusterControllerData* self) { // how long it is taking to hear responses from each other component. UID debugID = deterministicRandom()->randomUniqueID(); - TraceEvent("TimeKeeperCommit", debugID).detail("Version", v).detail("Timestamp", currentTime).log(); + TraceEvent("TimeKeeperCommit", debugID).detail("Version", v).log(); tr->debugTransaction(debugID); } int64_t ttl = currentTime - SERVER_KNOBS->TIME_KEEPER_DELAY * SERVER_KNOBS->TIME_KEEPER_MAX_ENTRIES; From 074bd174b6ef285800592bdb4aa801d171e72bbb Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Wed, 18 Aug 2021 12:42:17 -0500 Subject: [PATCH 53/63] Fix worker segfault by reference counting LocalConfiguration --- fdbserver/ConfigDatabaseUnitTests.actor.cpp | 29 ++++++++++++--------- fdbserver/LocalConfiguration.actor.cpp | 4 --- fdbserver/LocalConfiguration.h | 4 +-- fdbserver/worker.actor.cpp | 18 +++++++------ 4 files changed, 27 insertions(+), 28 deletions(-) diff --git a/fdbserver/ConfigDatabaseUnitTests.actor.cpp b/fdbserver/ConfigDatabaseUnitTests.actor.cpp index 5e874da7a0..ddefa1d8ac 100644 --- a/fdbserver/ConfigDatabaseUnitTests.actor.cpp +++ b/fdbserver/ConfigDatabaseUnitTests.actor.cpp @@ -123,11 +123,11 @@ public: class ReadFromLocalConfigEnvironment { UID id; std::string dataDir; - LocalConfiguration localConfiguration; + Reference localConfiguration; Reference const> cbi; Future consumer; - ACTOR static Future checkEventually(LocalConfiguration const* localConfiguration, + ACTOR static Future checkEventually(Reference localConfiguration, Optional expected) { state double lastMismatchTime = now(); loop { @@ -145,7 +145,7 @@ class ReadFromLocalConfigEnvironment { } ACTOR static Future setup(ReadFromLocalConfigEnvironment* self) { - wait(self->localConfiguration.initialize()); + wait(self->localConfiguration->initialize()); if (self->cbi) { // LocalConfiguration runs in a loop waiting for messages from the // broadcaster. These unit tests use the same @@ -155,7 +155,7 @@ class ReadFromLocalConfigEnvironment { // prevents two actors trying to listen for the same message on the // same interface, causing lots of issues! self->consumer.cancel(); - self->consumer = self->localConfiguration.consume(self->cbi->get()); + self->consumer = self->localConfiguration->consume(self->cbi->get()); } return Void(); } @@ -164,40 +164,43 @@ public: ReadFromLocalConfigEnvironment(std::string const& dataDir, std::string const& configPath, std::map const& manualKnobOverrides) - : dataDir(dataDir), localConfiguration(dataDir, configPath, manualKnobOverrides, IsTest::True), + : dataDir(dataDir), + localConfiguration(makeReference(dataDir, configPath, manualKnobOverrides, IsTest::True)), consumer(Never()) {} Future setup() { return setup(this); } Future restartLocalConfig(std::string const& newConfigPath) { - localConfiguration = LocalConfiguration(dataDir, newConfigPath, {}, IsTest::True); + std::map manualKnobOverrides = {}; + localConfiguration = + makeReference(dataDir, newConfigPath, manualKnobOverrides, IsTest::True); return setup(); } void connectToBroadcaster(Reference const> const& cbi) { this->cbi = cbi; - consumer = localConfiguration.consume(cbi->get()); + consumer = localConfiguration->consume(cbi->get()); } void checkImmediate(Optional expected) const { if (expected.present()) { - ASSERT_EQ(localConfiguration.getTestKnobs().TEST_LONG, expected.get()); + ASSERT_EQ(localConfiguration->getTestKnobs().TEST_LONG, expected.get()); } else { - ASSERT_EQ(localConfiguration.getTestKnobs().TEST_LONG, 0); + ASSERT_EQ(localConfiguration->getTestKnobs().TEST_LONG, 0); } } Future checkEventually(Optional expected) const { - return checkEventually(&localConfiguration, expected); + return checkEventually(localConfiguration, expected); } - LocalConfiguration& getMutableLocalConfiguration() { return localConfiguration; } + LocalConfiguration& getMutableLocalConfiguration() { return *localConfiguration; } Future getError() const { return consumer; } - Version lastSeenVersion() { return localConfiguration.lastSeenVersion(); } + Version lastSeenVersion() { return localConfiguration->lastSeenVersion(); } - ConfigClassSet configClassSet() { return localConfiguration.configClassSet(); } + ConfigClassSet configClassSet() { return localConfiguration->configClassSet(); } }; class LocalConfigEnvironment { diff --git a/fdbserver/LocalConfiguration.actor.cpp b/fdbserver/LocalConfiguration.actor.cpp index 267d67974e..e9a0cdbfbc 100644 --- a/fdbserver/LocalConfiguration.actor.cpp +++ b/fdbserver/LocalConfiguration.actor.cpp @@ -414,10 +414,6 @@ LocalConfiguration::LocalConfiguration(std::string const& dataFolder, IsTest isTest) : impl(PImpl::create(dataFolder, configPath, manualKnobOverrides, isTest)) {} -LocalConfiguration::LocalConfiguration(LocalConfiguration&&) = default; - -LocalConfiguration& LocalConfiguration::operator=(LocalConfiguration&&) = default; - LocalConfiguration::~LocalConfiguration() = default; Future LocalConfiguration::initialize() { diff --git a/fdbserver/LocalConfiguration.h b/fdbserver/LocalConfiguration.h index 9859d0e6bb..3dcb0a2245 100644 --- a/fdbserver/LocalConfiguration.h +++ b/fdbserver/LocalConfiguration.h @@ -43,7 +43,7 @@ FDB_DECLARE_BOOLEAN_PARAM(IsTest); * - Register with the broadcaster to receive new updates for the relevant configuration classes * - Persist these updates when received, and restart if necessary */ -class LocalConfiguration { +class LocalConfiguration : public ReferenceCounted { PImpl impl; public: @@ -51,8 +51,6 @@ public: std::string const& configPath, std::map const& manualKnobOverrides, IsTest = IsTest::False); - LocalConfiguration(LocalConfiguration&&); - LocalConfiguration& operator=(LocalConfiguration&&); ~LocalConfiguration(); Future initialize(); FlowKnobs const& getFlowKnobs() const; diff --git a/fdbserver/worker.actor.cpp b/fdbserver/worker.actor.cpp index ea4de5e7af..0ed71e75d8 100644 --- a/fdbserver/worker.actor.cpp +++ b/fdbserver/worker.actor.cpp @@ -520,7 +520,7 @@ ACTOR Future registrationClient(Reference const> degraded, Reference connFile, Reference> const> issues, - LocalConfiguration* localConfig) { + Reference localConfig) { // Keeps the cluster controller (as it may be re-elected) informed that this worker exists // The cluster controller uses waitFailureClient to find out if we die, and returns from registrationReply // (requiring us to re-register) The registration request piggybacks optional distributor interface if it exists. @@ -625,7 +625,6 @@ bool addressInDbAndPrimaryDc(const NetworkAddress& address, Reference(Endpoint({ grvProxyAddress }, UID(1, 2))); + grvProxyInterf.getConsistentReadVersion = + RequestStream(Endpoint({ grvProxyAddress }, UID(1, 2))); testDbInfo.client.grvProxies.push_back(grvProxyInterf); ASSERT(addressInDbAndPrimaryDc(grvProxyAddress, makeReference>(testDbInfo))); NetworkAddress commitProxyAddress(IPAddress(0x37373737), 1); CommitProxyInterface commitProxyInterf; - commitProxyInterf.commit = RequestStream(Endpoint({ commitProxyAddress }, UID(1, 2))); + commitProxyInterf.commit = + RequestStream(Endpoint({ commitProxyAddress }, UID(1, 2))); testDbInfo.client.commitProxies.push_back(commitProxyInterf); ASSERT(addressInDbAndPrimaryDc(commitProxyAddress, makeReference>(testDbInfo))); @@ -1226,7 +1227,7 @@ ACTOR Future workerServer(Reference connFile, std::string whitelistBinPaths, Reference> dbInfo, ConfigDBType configDBType, - LocalConfiguration* localConfig) { + Reference localConfig) { state PromiseStream errors; state Reference>> ddInterf( new AsyncVar>()); @@ -2373,14 +2374,15 @@ ACTOR Future fdbd(Reference connFile, ConfigDBType configDBType) { state vector> actors; state Promise recoveredDiskFiles; - state LocalConfiguration localConfig(dataFolder, configPath, manualKnobOverrides); + state Reference localConfig = + makeReference(dataFolder, configPath, manualKnobOverrides); // setupStackSignal(); getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::Worker; // FIXME: Initializing here causes simulation issues, these must be fixed /* if (configDBType != ConfigDBType::DISABLED) { - wait(localConfig.initialize()); + wait(localConfig->initialize()); } */ @@ -2451,7 +2453,7 @@ ACTOR Future fdbd(Reference connFile, whitelistBinPaths, dbInfo, configDBType, - &localConfig), + localConfig), "WorkerServer", UID(), &normalWorkerErrors())); From b8e2575f9261339c2529923eda5fba80ef1d81b5 Mon Sep 17 00:00:00 2001 From: yao-xiao-github <87789492+yao-xiao-github@users.noreply.github.com> Date: Wed, 18 Aug 2021 13:07:56 -0700 Subject: [PATCH 54/63] Update log directory to server log directory. (#5396) --- fdbserver/KeyValueStoreRocksDB.actor.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/fdbserver/KeyValueStoreRocksDB.actor.cpp b/fdbserver/KeyValueStoreRocksDB.actor.cpp index e2f88f86a3..c7753e7583 100644 --- a/fdbserver/KeyValueStoreRocksDB.actor.cpp +++ b/fdbserver/KeyValueStoreRocksDB.actor.cpp @@ -88,6 +88,7 @@ rocksdb::Options getOptions() { } options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(bbOpts)); + options.db_log_dir = SERVER_KNOBS->LOG_DIRECTORY; return options; } From 7f0a09d1e2299fac5653fa610a753e00c45394d1 Mon Sep 17 00:00:00 2001 From: Yao Xiao Date: Wed, 18 Aug 2021 15:22:58 -0700 Subject: [PATCH 55/63] Enable debugTransaction before getReadVersion. --- fdbserver/ClusterController.actor.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 75305b8c61..cfb0b21b0e 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -3938,7 +3938,13 @@ ACTOR Future timeKeeper(ClusterControllerData* self) { state Reference tr = makeReference(self->cx); loop { try { - + state UID debugID = deterministicRandom()->randomUniqueID(); + if (!g_network->isSimulated()) { + // This is done to provide an arbitrary logged transaction every ~10s. + // FIXME: replace or augment this with logging on the proxy which tracks + // how long it is taking to hear responses from each other component. + tr->debugTransaction(debugID); + } tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::LOCK_AWARE); tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE); @@ -3952,13 +3958,7 @@ ACTOR Future timeKeeper(ClusterControllerData* self) { int64_t currentTime = (int64_t)now(); versionMap.set(tr, currentTime, v); if (!g_network->isSimulated()) { - // This is done to provide an arbitrary logged transaction every ~10s. - // FIXME: replace or augment this with logging on the proxy which tracks - // how long it is taking to hear responses from each other component. - - UID debugID = deterministicRandom()->randomUniqueID(); TraceEvent("TimeKeeperCommit", debugID).detail("Version", v).log(); - tr->debugTransaction(debugID); } int64_t ttl = currentTime - SERVER_KNOBS->TIME_KEEPER_DELAY * SERVER_KNOBS->TIME_KEEPER_MAX_ENTRIES; if (ttl > 0) { From 0afb63c13f9c157235658c0bee5c45ada8d1d7be Mon Sep 17 00:00:00 2001 From: yao-xiao-github <87789492+yao-xiao-github@users.noreply.github.com> Date: Wed, 18 Aug 2021 15:29:13 -0700 Subject: [PATCH 56/63] Update fdbserver/ClusterController.actor.cpp Co-authored-by: A.J. Beamon --- fdbserver/ClusterController.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index cfb0b21b0e..600ea63113 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -3958,7 +3958,7 @@ ACTOR Future timeKeeper(ClusterControllerData* self) { int64_t currentTime = (int64_t)now(); versionMap.set(tr, currentTime, v); if (!g_network->isSimulated()) { - TraceEvent("TimeKeeperCommit", debugID).detail("Version", v).log(); + TraceEvent("TimeKeeperCommit", debugID).detail("Version", v); } int64_t ttl = currentTime - SERVER_KNOBS->TIME_KEEPER_DELAY * SERVER_KNOBS->TIME_KEEPER_MAX_ENTRIES; if (ttl > 0) { From 7b0756e391fa12671b48155099c754f73f895178 Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Wed, 18 Aug 2021 15:48:02 -0700 Subject: [PATCH 57/63] fix uninitialized int --- fdbserver/OldTLogServer_6_2.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index 284afb1da3..7cce23207d 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -339,7 +339,7 @@ struct TLogData : NonCopyable { int64_t targetVolatileBytes; // The number of bytes of mutations this TLog should hold in memory before spilling. int64_t overheadBytesInput; int64_t overheadBytesDurable; - int activePeekStreams; + int activePeekStreams = 0; WorkerCache tlogCache; FlowLock peekMemoryLimiter; From b2c063dd93624ed0e9c1c1b578871973c836cea4 Mon Sep 17 00:00:00 2001 From: Chaoguang Lin Date: Wed, 18 Aug 2021 18:09:02 -0700 Subject: [PATCH 58/63] Change g_network->runNetwork to API->runNetwork --- fdbcli/fdbcli.actor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index 7aebb1efad..280b420b86 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -3195,10 +3195,10 @@ ACTOR template Future stopNetworkAfter(Future what) { try { T t = wait(what); - g_network->stop(); + API->stopNetwork(); return t; } catch (...) { - g_network->stop(); + API->stopNetwork(); throw; } } @@ -4685,7 +4685,7 @@ int main(int argc, char** argv) { Future cliFuture = runCli(opt); Future timeoutFuture = opt.exit_timeout ? timeExit(opt.exit_timeout) : Never(); auto f = stopNetworkAfter(success(cliFuture) || timeoutFuture); - runNetwork(); + API->runNetwork(); if (cliFuture.isReady()) { return cliFuture.get(); From bfae4bf24ae34897a2609311977952cfa41e8bc6 Mon Sep 17 00:00:00 2001 From: Pierre Zemb Date: Thu, 19 Aug 2021 16:23:26 +0200 Subject: [PATCH 59/63] Fix documentation about the tracing data layout --- documentation/sphinx/source/request-tracing.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/documentation/sphinx/source/request-tracing.rst b/documentation/sphinx/source/request-tracing.rst index 03d2d35c50..335952fb67 100644 --- a/documentation/sphinx/source/request-tracing.rst +++ b/documentation/sphinx/source/request-tracing.rst @@ -63,7 +63,7 @@ Source IP:port 0 string The IP and port of the machine where the s Trace ID 1 uint64 The 64-bit identifier of the trace. All spans in a trace share the same trace ID. Span ID 2 uint64 The 64-bit identifier of the span. All spans have a unique identifier. Start timestamp 3 double The timestamp when the operation represented by the span began. -End timestamp 4 double The timestamp when the operation represented by the span ended. +Duration 4 double The duration in seconds of the operation represented by the span. Operation name 5 string The name of the operation the span represents. Tags 6 map User defined tags, added manually to specify additional information. Parent span IDs 7 vector (Optional) A list of span IDs representing parents of this span. From 68b41392a0166aa3eb42b5a9e19711a2d1a25604 Mon Sep 17 00:00:00 2001 From: Chaoguang Lin Date: Thu, 19 Aug 2021 12:13:26 -0700 Subject: [PATCH 60/63] Change to use ArgumentParser, set env to use external client library in Popen, enable logging in all tests --- bindings/python/CMakeLists.txt | 14 +++--- bindings/python/tests/fdbcli_tests.py | 62 +++++++++++++++++---------- 2 files changed, 45 insertions(+), 31 deletions(-) diff --git a/bindings/python/CMakeLists.txt b/bindings/python/CMakeLists.txt index 802514c485..35e78383a5 100644 --- a/bindings/python/CMakeLists.txt +++ b/bindings/python/CMakeLists.txt @@ -79,16 +79,15 @@ if (NOT WIN32 AND NOT OPEN_FOR_IDE) add_fdbclient_test( NAME single_process_fdbcli_tests COMMAND ${CMAKE_SOURCE_DIR}/bindings/python/tests/fdbcli_tests.py - ${CMAKE_BINARY_DIR}/bin/fdbcli + ${CMAKE_BINARY_DIR} @CLUSTER_FILE@ - 1 ) add_fdbclient_test( NAME multi_process_fdbcli_tests PROCESS_NUMBER 5 TEST_TIMEOUT 120 # The test can take near to 1 minutes sometime, set timeout to 2 minutes to be safe COMMAND ${CMAKE_SOURCE_DIR}/bindings/python/tests/fdbcli_tests.py - ${CMAKE_BINARY_DIR}/bin/fdbcli + ${CMAKE_BINARY_DIR} @CLUSTER_FILE@ 5 ) @@ -96,20 +95,19 @@ if (NOT WIN32 AND NOT OPEN_FOR_IDE) add_fdbclient_test( NAME single_process_external_client_fdbcli_tests COMMAND ${CMAKE_SOURCE_DIR}/bindings/python/tests/fdbcli_tests.py - ${CMAKE_BINARY_DIR}/bin/fdbcli + ${CMAKE_BINARY_DIR} @CLUSTER_FILE@ - 1 - ${CMAKE_BINARY_DIR}/bindings/c/libfdb_c.so + --external-client-library ${CMAKE_BINARY_DIR}/bindings/c/libfdb_c.so ) add_fdbclient_test( NAME multi_process_external_client_fdbcli_tests PROCESS_NUMBER 5 TEST_TIMEOUT 120 # The test can take near to 1 minutes sometime, set timeout to 2 minutes to be safe COMMAND ${CMAKE_SOURCE_DIR}/bindings/python/tests/fdbcli_tests.py - ${CMAKE_BINARY_DIR}/bin/fdbcli + ${CMAKE_BINARY_DIR} @CLUSTER_FILE@ 5 - ${CMAKE_BINARY_DIR}/bindings/c/libfdb_c.so + --external-client-library ${CMAKE_BINARY_DIR}/bindings/c/libfdb_c.so ) endif() endif() diff --git a/bindings/python/tests/fdbcli_tests.py b/bindings/python/tests/fdbcli_tests.py index 11f7c191eb..36f9ec34a9 100755 --- a/bindings/python/tests/fdbcli_tests.py +++ b/bindings/python/tests/fdbcli_tests.py @@ -8,9 +8,10 @@ import functools import json import time import random +from argparse import ArgumentParser, RawDescriptionHelpFormatter -def enable_logging(level=logging.ERROR): +def enable_logging(level=logging.DEBUG): """Enable logging in the function with the specified logging level Args: @@ -42,7 +43,7 @@ def run_fdbcli_command(*args): string: Console output from fdbcli """ commands = command_template + ["{}".format(' '.join(args))] - return subprocess.run(commands, stdout=subprocess.PIPE).stdout.decode('utf-8').strip() + return subprocess.run(commands, stdout=subprocess.PIPE, env=fdbcli_env).stdout.decode('utf-8').strip() def run_fdbcli_command_and_get_error(*args): @@ -52,7 +53,7 @@ def run_fdbcli_command_and_get_error(*args): string: Stderr output from fdbcli """ commands = command_template + ["{}".format(' '.join(args))] - return subprocess.run(commands, stdout=subprocess.PIPE, stderr=subprocess.PIPE).stderr.decode('utf-8').strip() + return subprocess.run(commands, stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=fdbcli_env).stderr.decode('utf-8').strip() @enable_logging() @@ -156,7 +157,7 @@ def lockAndUnlock(logger): output2 = run_fdbcli_command_and_get_error("lock") assert output2 == 'ERROR: Database is locked (1038)' # unlock the database - process = subprocess.Popen(command_template + ['unlock ' + lock_uid], stdin=subprocess.PIPE, stdout=subprocess.PIPE) + process = subprocess.Popen(command_template + ['unlock ' + lock_uid], stdin=subprocess.PIPE, stdout=subprocess.PIPE, env=fdbcli_env) line1 = process.stdout.readline() # The randome passphrease we need to confirm to proceed the unlocking line2 = process.stdout.readline() @@ -180,7 +181,7 @@ def kill(logger): # This is currently an issue with fdbcli, # where you need to first run 'kill' to initialize processes' list # and then specify the certain process to kill - process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE) + process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE, env=fdbcli_env) # output2, err = process.communicate(input='kill; kill {}\n'.format(address).encode()) logger.debug(output2) @@ -210,7 +211,7 @@ def suspend(logger): assert len(pinfo) == 1 pid = pinfo[0].split(' ')[0] logger.debug("Pid: {}".format(pid)) - process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE) + process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE, env=fdbcli_env) # suspend the process for enough long time output2, err = process.communicate(input='suspend; suspend 3600 {}\n'.format(address).encode()) # the cluster should be unavailable after the only process being suspended @@ -295,7 +296,7 @@ def transaction(logger): """ err1 = run_fdbcli_command_and_get_error('set', 'key', 'value') assert err1 == 'ERROR: writemode must be enabled to set or clear keys in the database.' - process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE) + process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE, env=fdbcli_env) transaction_flow = ['writemode on', 'begin', 'getversion', 'set key value', 'get key', 'commit'] output1, _ = process.communicate(input='\n'.join(transaction_flow).encode()) # split the output into lines @@ -314,7 +315,7 @@ def transaction(logger): output2 = run_fdbcli_command('get', 'key') assert output2 == "`key' is `value'" # test rollback and read-your-write behavior - process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE) + process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE, env=fdbcli_env) transaction_flow = [ 'writemode on', 'begin', 'getrange a z', 'clear key', 'get key', @@ -331,7 +332,7 @@ def transaction(logger): output4 = run_fdbcli_command('get', 'key') assert output4 == "`key' is `value'" # test read_your_write_disable option and clear the inserted key - process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE) + process = subprocess.Popen(command_template[:-1], stdin=subprocess.PIPE, stdout=subprocess.PIPE, env=fdbcli_env) transaction_flow = [ 'writemode on', 'begin', 'option on READ_YOUR_WRITES_DISABLE', @@ -354,7 +355,7 @@ def get_fdb_process_addresses(logger): logger.debug(output) # except the first line, each line is one process addresses = output.split('\n')[1:] - assert len(addresses) == process_number + assert len(addresses) == args.process_number return addresses @@ -450,7 +451,9 @@ def read_system_key(k): def throttle(logger): # no throttled tags at the beginning no_throttle_tags_output = 'There are no throttled tags' - assert run_fdbcli_command('throttle', 'list') == no_throttle_tags_output + output = run_fdbcli_command('throttle', 'list') + logger.debug(output) + assert output == no_throttle_tags_output # test 'throttle enable auto' run_fdbcli_command('throttle', 'enable', 'auto') # verify the change is applied by reading the system key @@ -473,21 +476,34 @@ def wait_for_database_available(logger): if __name__ == '__main__': - # fdbcli_tests.py [external_client_library_path] - assert len(sys.argv) == 4 or len( - sys.argv) == 5, "Please pass arguments: [external_client_library_path]" - # set external client library - if len(sys.argv) == 5: - external_client_library_path = sys.argv[4] + parser = ArgumentParser(formatter_class=RawDescriptionHelpFormatter, + description=""" + The test calls fdbcli commands through fdbcli --exec "" interactively using subprocess. + The outputs from fdbcli are returned and compared to predefined results. + Consequently, changing fdbcli outputs or breaking any commands will casue the test to fail. + Commands that are easy to test will run against a single process cluster. + For complex commands like exclude, they will run against a cluster with multiple(current set to 5) processes. + If external_client_library is given, we will disable the local client and use the external client to run fdbcli. + """) + parser.add_argument('build_dir', metavar='BUILD_DIRECTORY', help='FDB build directory') + parser.add_argument('cluster_file', metavar='CLUSTER_FILE', help='FDB cluster file') + parser.add_argument('process_number', nargs='?', metavar='PROCESS_NUMBER', help="Number of fdb processes", type=int, default=1) + parser.add_argument('--external-client-library', '-e', metavar='EXTERNAL_CLIENT_LIBRARY_PATH', help="External client library path") + args = parser.parse_args() + + # keep current environment variables + fdbcli_env = os.environ.copy() + # set external client library if provided + if args.external_client_library: # disable local client and use the external client library - os.environ['FDB_NETWORK_OPTION_DISABLE_LOCAL_CLIENT'] = '' - os.environ['FDB_NETWORK_OPTION_EXTERNAL_CLIENT_LIBRARY'] = external_client_library_path + fdbcli_env['FDB_NETWORK_OPTION_DISABLE_LOCAL_CLIENT'] = '' + fdbcli_env['FDB_NETWORK_OPTION_EXTERNAL_CLIENT_LIBRARY'] = args.external_client_library + # shell command template - command_template = [sys.argv[1], '-C', sys.argv[2], '--exec'] + command_template = [args.build_dir + '/bin/fdbcli', '-C', args.cluster_file, '--exec'] # tests for fdbcli commands # assertions will fail if fdbcli does not work as expected - process_number = int(sys.argv[3]) - if process_number == 1: + if args.process_number == 1: # TODO: disable for now, the change can cause the database unavailable # advanceversion() cache_range() @@ -501,6 +517,6 @@ if __name__ == '__main__': transaction() throttle() else: - assert process_number > 1, "Process number should be positive" + assert args.process_number > 1, "Process number should be positive" coordinators() exclude() From fbed117b219c1130085bdc43e96ef9c4d3184cc8 Mon Sep 17 00:00:00 2001 From: Chaoguang Lin Date: Thu, 19 Aug 2021 19:34:25 +0000 Subject: [PATCH 61/63] Solve comments; Add a method to return DatabaseContext Reference --- fdbclient/NativeAPI.actor.h | 1 + fdbclient/TagThrottle.actor.h | 4 ---- fdbserver/Ratekeeper.actor.cpp | 10 +++------- fdbserver/workloads/TagThrottleApi.actor.cpp | 19 ++++++++----------- .../workloads/WriteTagThrottling.actor.cpp | 5 ++--- 5 files changed, 14 insertions(+), 25 deletions(-) diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h index a695dc4ae8..96bf3aaed9 100644 --- a/fdbclient/NativeAPI.actor.h +++ b/fdbclient/NativeAPI.actor.h @@ -106,6 +106,7 @@ public: inline DatabaseContext* getPtr() const { return db.getPtr(); } inline DatabaseContext* extractPtr() { return db.extractPtr(); } DatabaseContext* operator->() const { return db.getPtr(); } + Reference getReference() const { return db; } const UniqueOrderedOptionList& getTransactionDefaults() const; diff --git a/fdbclient/TagThrottle.actor.h b/fdbclient/TagThrottle.actor.h index 522d37f7b0..4946830ffc 100644 --- a/fdbclient/TagThrottle.actor.h +++ b/fdbclient/TagThrottle.actor.h @@ -35,10 +35,6 @@ #include "fdbclient/CommitTransaction.h" #include "flow/actorcompiler.h" // This must be the last #include. -class Database; - -namespace ThrottleApi {} - typedef StringRef TransactionTagRef; typedef Standalone TransactionTag; diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index 39a9e9aa35..7514869fc1 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -544,7 +544,7 @@ struct GrvProxyInfo { } }; -} +} // namespace RatekeeperActorCpp struct RatekeeperData { UID id; @@ -601,12 +601,8 @@ struct RatekeeperData { SERVER_KNOBS->MAX_TL_SS_VERSION_DIFFERENCE_BATCH, SERVER_KNOBS->TARGET_DURABILITY_LAG_VERSIONS_BATCH), autoThrottlingEnabled(false) { - expiredTagThrottleCleanup = recurring( - [this]() { - Reference db = Reference::addRef(this->db.getPtr()); - ThrottleApi::expire(db); - }, - SERVER_KNOBS->TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL); + expiredTagThrottleCleanup = recurring([this]() { ThrottleApi::expire(this->db.getReference()); }, + SERVER_KNOBS->TAG_THROTTLE_EXPIRED_CLEANUP_INTERVAL); } }; diff --git a/fdbserver/workloads/TagThrottleApi.actor.cpp b/fdbserver/workloads/TagThrottleApi.actor.cpp index 97d0207a89..c77a5ee68c 100644 --- a/fdbserver/workloads/TagThrottleApi.actor.cpp +++ b/fdbserver/workloads/TagThrottleApi.actor.cpp @@ -78,13 +78,12 @@ struct TagThrottleApiWorkload : TestWorkload { state TransactionPriority priority = deterministicRandom()->randomChoice(allTransactionPriorities); state double rate = deterministicRandom()->random01() * 20; state double duration = 1 + deterministicRandom()->random01() * 19; - state Reference db = Reference::addRef(cx.getPtr()); TagSet tagSet; tagSet.addTag(tag); try { - wait(ThrottleApi::throttleTags(db, + wait(ThrottleApi::throttleTags(cx.getReference(), tagSet, rate, duration, @@ -138,8 +137,7 @@ struct TagThrottleApiWorkload : TestWorkload { } } - state Reference db = Reference::addRef(cx.getPtr()); - bool removed = wait(ThrottleApi::unthrottleTags(db, tagSet, throttleType, priority)); + bool removed = wait(ThrottleApi::unthrottleTags(cx.getReference(), tagSet, throttleType, priority)); if (removed) { ASSERT(erased || !throttleType.present() || throttleType.get() == TagThrottleType::AUTO); } else { @@ -154,8 +152,8 @@ struct TagThrottleApiWorkload : TestWorkload { Database cx, std::map, TagThrottleInfo> const* manuallyThrottledTags) { - state Reference db = Reference::addRef(cx.getPtr()); - std::vector tags = wait(ThrottleApi::getThrottledTags(db, CLIENT_KNOBS->TOO_MANY)); + std::vector tags = + wait(ThrottleApi::getThrottledTags(cx.getReference(), CLIENT_KNOBS->TOO_MANY)); int manualThrottledTags = 0; int activeAutoThrottledTags = 0; @@ -188,8 +186,8 @@ struct TagThrottleApiWorkload : TestWorkload { } ACTOR Future getRecommendedTags(TagThrottleApiWorkload* self, Database cx) { - state Reference db = Reference::addRef(cx.getPtr()); - std::vector tags = wait(ThrottleApi::getRecommendedTags(db, CLIENT_KNOBS->TOO_MANY)); + std::vector tags = + wait(ThrottleApi::getRecommendedTags(cx.getReference(), CLIENT_KNOBS->TOO_MANY)); for (auto& tag : tags) { ASSERT(tag.throttleType == TagThrottleType::AUTO); @@ -205,8 +203,7 @@ struct TagThrottleApiWorkload : TestWorkload { deterministicRandom()->coinflip() ? Optional() : deterministicRandom()->randomChoice(allTransactionPriorities); - state Reference db = Reference::addRef(cx.getPtr()); - bool unthrottled = wait(ThrottleApi::unthrottleAll(db, throttleType, priority)); + bool unthrottled = wait(ThrottleApi::unthrottleAll(cx.getReference(), throttleType, priority)); if (!throttleType.present() || throttleType.get() == TagThrottleType::MANUAL) { bool unthrottleExpected = false; bool empty = manuallyThrottledTags->empty(); @@ -233,7 +230,7 @@ struct TagThrottleApiWorkload : TestWorkload { } ACTOR Future enableAutoThrottling(TagThrottleApiWorkload* self, Database cx) { - state Reference db = Reference::addRef(cx.getPtr()); + state Reference db = cx.getReference(); if (deterministicRandom()->coinflip()) { wait(ThrottleApi::enableAuto(db, true)); self->autoThrottleEnabled = true; diff --git a/fdbserver/workloads/WriteTagThrottling.actor.cpp b/fdbserver/workloads/WriteTagThrottling.actor.cpp index a9170f122c..927dbc6f6e 100644 --- a/fdbserver/workloads/WriteTagThrottling.actor.cpp +++ b/fdbserver/workloads/WriteTagThrottling.actor.cpp @@ -94,14 +94,13 @@ struct WriteTagThrottlingWorkload : KVWorkload { std::string description() const override { return WriteTagThrottlingWorkload::NAME; } ACTOR static Future _setup(Database cx, WriteTagThrottlingWorkload* self) { - state Reference db = Reference::addRef(cx.getPtr()); ASSERT(CLIENT_KNOBS->MAX_TAGS_PER_TRANSACTION >= MIN_TAGS_PER_TRANSACTION && CLIENT_KNOBS->MAX_TRANSACTION_TAG_LENGTH >= MIN_TRANSACTION_TAG_LENGTH); if (self->populateData) { wait(bulkSetup(cx, self, self->keyCount, Promise())); } if (self->clientId == 0) { - wait(ThrottleApi::enableAuto(db, true)); + wait(ThrottleApi::enableAuto(cx.getReference(), true)); } return Void(); } @@ -307,7 +306,7 @@ struct WriteTagThrottlingWorkload : KVWorkload { } ACTOR static Future throttledTagUpdater(Database cx, WriteTagThrottlingWorkload* self) { state std::vector tags; - state Reference db = Reference::addRef(cx.getPtr()); + state Reference db = cx.getReference(); loop { wait(delay(1.0)); wait(store(tags, ThrottleApi::getThrottledTags(db, CLIENT_KNOBS->TOO_MANY, true))); From 09342f43eab789c3e9d4f29ba1e1b04bdb5e783f Mon Sep 17 00:00:00 2001 From: Xiaoxi Wang Date: Thu, 19 Aug 2021 13:55:30 -0700 Subject: [PATCH 62/63] fix spelling error --- flow/actorcompiler/Actor checklist.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flow/actorcompiler/Actor checklist.txt b/flow/actorcompiler/Actor checklist.txt index bcb3c28be0..bb413d4d2e 100644 --- a/flow/actorcompiler/Actor checklist.txt +++ b/flow/actorcompiler/Actor checklist.txt @@ -75,10 +75,10 @@ Performance issues: - When waiting for a number of things, wait a little extra time to get the stragglers. (See the SmartQuorum() generic actor) -- If asking another asynch server to do units of work, don't queue up more +- If asking another asynchronous server to do units of work, don't queue up more work than is necessary to keep the server busy. Likewise, if you are - busy, let your own work queue fill up to signal your requestor + busy, let your own work queue fill up to signal your requester that you are blocked. Also do this personally with managers assigning you stuff. -- Pass all variables as "const &" if thier size is greater than 8 bytes. +- Pass all variables as "const &" if their size is greater than 8 bytes. From 17f6f7a2cad6a7c4227fa59016e9599e43509356 Mon Sep 17 00:00:00 2001 From: Josh Slocum Date: Thu, 19 Aug 2021 16:17:55 -0500 Subject: [PATCH 63/63] Using the current max shard size estimate instead of the max possible if unavailable --- fdbserver/DataDistributionTracker.actor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdbserver/DataDistributionTracker.actor.cpp b/fdbserver/DataDistributionTracker.actor.cpp index e27dbf4e56..a50719a118 100644 --- a/fdbserver/DataDistributionTracker.actor.cpp +++ b/fdbserver/DataDistributionTracker.actor.cpp @@ -858,7 +858,7 @@ ACTOR Future fetchShardMetrics(DataDistributionTracker* self, GetMetricsRe when(wait(delay(SERVER_KNOBS->DD_SHARD_METRICS_TIMEOUT, TaskPriority::DataDistribution))) { TEST(true); // DD_SHARD_METRICS_TIMEOUT StorageMetrics largeMetrics; - largeMetrics.bytes = SERVER_KNOBS->MAX_SHARD_BYTES; + largeMetrics.bytes = getMaxShardSize(self->dbSizeEstimate->get()); req.reply.send(largeMetrics); } }