From ccb103caf160f1aac6bc4db387abc2eb986c5ec1 Mon Sep 17 00:00:00 2001 From: Xiaoge Su Date: Sat, 25 Jul 2020 21:44:10 -0700 Subject: [PATCH 1/8] Extract Step 1 and 2 from commitBatch --- fdbserver/MasterProxyServer.actor.cpp | 733 ++++++++++++++++---------- 1 file changed, 453 insertions(+), 280 deletions(-) diff --git a/fdbserver/MasterProxyServer.actor.cpp b/fdbserver/MasterProxyServer.actor.cpp index 55f8ccd8eb..efdc097377 100644 --- a/fdbserver/MasterProxyServer.actor.cpp +++ b/fdbserver/MasterProxyServer.actor.cpp @@ -18,6 +18,9 @@ * limitations under the License. */ +#include +#include + #include "fdbclient/Atomic.h" #include "fdbclient/DatabaseConfiguration.h" #include "fdbclient/FDBTypes.h" @@ -49,8 +52,8 @@ #include "flow/Knobs.h" #include "flow/TDMetric.actor.h" #include "flow/Tracing.h" + #include "flow/actorcompiler.h" // This must be the last #include. -#include ACTOR Future broadcastTxnRequest(TxnStateRequest req, int sendAmount, bool sendReply) { state ReplyPromise reply = req.reply; @@ -168,7 +171,7 @@ struct TransactionRateInfo { Smoother smoothRate; Smoother smoothReleased; - TransactionRateInfo(double rate) : rate(rate), limit(0), budget(0), disabled(true), smoothRate(SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW), + TransactionRateInfo(double rate) : rate(rate), limit(0), budget(0), disabled(true), smoothRate(SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW), smoothReleased(SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW) {} void reset() { @@ -177,7 +180,7 @@ struct TransactionRateInfo { // have started but didn't, and making that our limit. More precisely, we track a smoothed rate limit and release rate, // the difference of which is the rate of additional transactions that we could have released based on that window. // Then we multiply by the window size to get a number of transactions. - // + // // Limit can be negative in the event that we are releasing more transactions than we are allowed (due to the use of // our budget or because of higher priority transactions). double releaseRate = smoothRate.smoothTotal() - smoothReleased.smoothRate(); @@ -191,14 +194,14 @@ struct TransactionRateInfo { void updateBudget(int64_t numStartedAtPriority, bool queueEmptyAtPriority, double elapsed) { // Update the budget to accumulate any extra capacity available or remove any excess that was used. // The actual delta is the portion of the limit we didn't use multiplied by the fraction of the window that elapsed. - // - // We may have exceeded our limit due to the budget or because of higher priority transactions, in which case this - // delta will be negative. The delta can also be negative in the event that our limit was negative, which can happen + // + // We may have exceeded our limit due to the budget or because of higher priority transactions, in which case this + // delta will be negative. The delta can also be negative in the event that our limit was negative, which can happen // if we had already started more transactions in our window than our rate would have allowed. // // This budget has the property that when the budget is required to start transactions (because batches are big), // the sum limit+budget will increase linearly from 0 to the batch size over time and decrease by the batch size - // upon starting a batch. In other words, this works equivalently to a model where we linearly accumulate budget over + // upon starting a batch. In other words, this works equivalently to a model where we linearly accumulate budget over // time in the case that our batches are too big to take advantage of the window based limits. budget = std::max(0.0, budget + elapsed * (limit - numStartedAtPriority) / SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW); @@ -315,7 +318,7 @@ ACTOR Future queueTransactionStartRequests( PromiseStream GRVTimer, double *lastGRVTime, double *GRVBatchTime, FutureStream replyTimes, ProxyStats* stats, TransactionRateInfo* batchRateInfo, - TransactionTagMap* transactionTagCounter) + TransactionTagMap* transactionTagCounter) { loop choose{ when(GetReadVersionRequest req = waitNext(readVersionRequests)) { @@ -477,7 +480,7 @@ struct ProxyCommitData { } return false; } - + void updateLatencyBandConfig(Optional newLatencyBandConfig) { if(newLatencyBandConfig.present() != latencyBandConfig.present() || (newLatencyBandConfig.present() && newLatencyBandConfig.get().grvConfig != latencyBandConfig.get().grvConfig)) @@ -748,7 +751,7 @@ ACTOR Future addBackupMutations(ProxyCommitData* self, std::map addBackupMutations(ProxyCommitData* self, std::mapfirst) ); // We are writing into the configured destination - + auto& tags = self->tagsForKey(backupMutation.param1); toCommit->addTags(tags); toCommit->addTypedMessage(backupMutation); @@ -805,178 +808,329 @@ ACTOR Future releaseResolvingAfter(ProxyCommitData* self, Future rel return Void(); } -// Commit one batch of transactions trs -ACTOR Future commitBatch( - ProxyCommitData* self, - vector* pTrs, - int currentBatchMemBytesCount) -{ - //WARNING: this code is run at a high priority (until the first delay(0)), so it needs to do as little work as possible - state std::vector trs(std::move(*pTrs)); - state int64_t localBatchNumber = ++self->localCommitBatchesStarted; - state LogPushData toCommit(self->logSystem); - state double t1 = now(); - state Optional debugID; - state bool forceRecovery = false; - state int batchOperations = 0; - state Span span("MP:commitBatch"_loc); +namespace CommitBatch { + +struct Context { + ProxyCommitData* const pProxyCommitData; + std::vector trs; + int currentBatchMemBytesCount; + + double startTime; + + Optional debugID; + + bool forceRecovery = false; + + int64_t localBatchNumber; + LogPushData toCommit; + + int batchOperations = 0; + + Span span = Span("MP:commitBatch"_loc); + int64_t batchBytes = 0; - for (int t = 0; t> transactionResolverMap; + std::vector>> txReadConflictRangeIndexMap; + + Future releaseDelay; + Future releaseFuture; + + std::vector resolution; + + Context( + ProxyCommitData*, + const std::vector*, + const int); + + void setupDebugTrack(); + +private: + void evaluateBatchSize(); +}; + +Context::Context( + ProxyCommitData* const pProxyCommitData_, + const std::vector* trs_, + const int currentBatchMemBytesCount) : + + pProxyCommitData(pProxyCommitData_), + trs(std::move(*const_cast*>(trs_))), + currentBatchMemBytesCount(currentBatchMemBytesCount), + + startTime(g_network->now()), + + localBatchNumber(++pProxyCommitData->localCommitBatchesStarted), + toCommit(pProxyCommitData->logSystem) { + + evaluateBatchSize(); + + if (batchOperations != 0) { + latencyBucket = std::min( + SERVER_KNOBS->PROXY_COMPUTE_BUCKETS - 1, + SERVER_KNOBS->PROXY_COMPUTE_BUCKETS * batchBytes / + (batchOperations * ( + CLIENT_KNOBS->VALUE_SIZE_LIMIT + + CLIENT_KNOBS->KEY_SIZE_LIMIT + )) + ); } - state int latencyBucket = batchOperations == 0 ? 0 : std::min(SERVER_KNOBS->PROXY_COMPUTE_BUCKETS-1,SERVER_KNOBS->PROXY_COMPUTE_BUCKETS*batchBytes/(batchOperations*(CLIENT_KNOBS->VALUE_SIZE_LIMIT+CLIENT_KNOBS->KEY_SIZE_LIMIT))); - ASSERT(SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS <= SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT); // since we are using just the former to limit the number of versions actually in flight! + // since we are using just the former to limit the number of versions actually in flight! + ASSERT(SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS <= SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT); +} - // Active load balancing runs at a very high priority (to obtain accurate estimate of memory used by commit batches) so we need to downgrade here - wait(delay(0, TaskPriority::ProxyCommit)); - - self->lastVersionTime = t1; - - ++self->stats.commitBatchIn; - - for (int t = 0; trandomUniqueID(); - g_traceBatch.addAttach("CommitAttachID", trs[t].debugID.get().first(), debugID.get().first()); + } + + g_traceBatch.addAttach( + "CommitAttachID", + tr.debugID.get().first(), + debugID.get().first() + ); } - span.addParent(trs[t].spanContext); + span.addParent(tr.spanContext); } - if(localBatchNumber == 2 && !debugID.present() && self->firstProxy && !g_network->isSimulated()) { - debugID = deterministicRandom()->randomUniqueID(); - TraceEvent("SecondCommitBatch", self->dbgid).detail("DebugID", debugID.get()); + if (debugID.present()) { + g_traceBatch.addEvent( + "CommitDebug", + debugID.get().first(), + "MasterProxyServer.commitBatch.Before" + ); + } +} + +void Context::evaluateBatchSize() { + for (auto& tr: trs) { + auto& mutations = tr.transaction.mutations; + batchOperations += mutations.size(); + batchBytes += mutations.expectedSize(); + } +} + +ACTOR Future preresolutionProcessing(Context* self) { + + state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData; + state std::vector& trs = self->trs; + state const int64_t localBatchNumber = self->localBatchNumber; + state const int latencyBucket = self->latencyBucket; + state const Optional& debugID = self->debugID; + + // Pre-resolution the commits + TEST(pProxyCommitData->latestLocalCommitBatchResolving.get() < localBatchNumber - 1); + wait(pProxyCommitData->latestLocalCommitBatchResolving.whenAtLeast(localBatchNumber - 1)); + self->releaseDelay = delay( + std::min(SERVER_KNOBS->MAX_PROXY_COMPUTE, + self->batchOperations * pProxyCommitData->commitComputePerOperation[latencyBucket]), + TaskPriority::ProxyMasterVersionReply + ); + + if (debugID.present()) { + g_traceBatch.addEvent( + "CommitDebug", debugID.get().first(), + "MasterProxyServer.commitBatch.GettingCommitVersion" + ); } - if (debugID.present()) - g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "MasterProxyServer.commitBatch.Before"); + GetCommitVersionRequest req( + self->span.context, + pProxyCommitData->commitVersionRequestNumber++, + pProxyCommitData->mostRecentProcessedRequestNumber, + pProxyCommitData->dbgid + ); + GetCommitVersionReply versionReply = wait(brokenPromiseToNever( + pProxyCommitData->master.getCommitVersion.getReply( + req, TaskPriority::ProxyMasterVersionReply + ) + )); - /////// Phase 1: Pre-resolution processing (CPU bound except waiting for a version # which is separately pipelined and *should* be available by now (unless empty commit); ordered; currently atomic but could yield) + pProxyCommitData->mostRecentProcessedRequestNumber = versionReply.requestNum; - // Queuing pre-resolution commit processing - TEST(self->latestLocalCommitBatchResolving.get() < localBatchNumber - 1); - wait(self->latestLocalCommitBatchResolving.whenAtLeast(localBatchNumber-1)); - state Future releaseDelay = delay(std::min(SERVER_KNOBS->MAX_PROXY_COMPUTE, batchOperations*self->commitComputePerOperation[latencyBucket]), TaskPriority::ProxyMasterVersionReply); + pProxyCommitData->stats.txnCommitVersionAssigned += trs.size(); + pProxyCommitData->stats.lastCommitVersionAssigned = versionReply.version; - if (debugID.present()) - g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "MasterProxyServer.commitBatch.GettingCommitVersion"); - - GetCommitVersionRequest req(span.context, self->commitVersionRequestNumber++, self->mostRecentProcessedRequestNumber, self->dbgid); - GetCommitVersionReply versionReply = wait( brokenPromiseToNever(self->master.getCommitVersion.getReply(req, TaskPriority::ProxyMasterVersionReply)) ); - self->mostRecentProcessedRequestNumber = versionReply.requestNum; - - self->stats.txnCommitVersionAssigned += trs.size(); - self->stats.lastCommitVersionAssigned = versionReply.version; - - state Version commitVersion = versionReply.version; - state Version prevVersion = versionReply.prevVersion; + self->commitVersion = versionReply.version; + self->prevVersion = versionReply.prevVersion; for(auto it : versionReply.resolverChanges) { - auto rs = self->keyResolvers.modify(it.range); + auto rs = pProxyCommitData->keyResolvers.modify(it.range); for(auto r = rs.begin(); r != rs.end(); ++r) r->value().emplace_back(versionReply.resolverChangesVersion,it.dest); } - //TraceEvent("ProxyGotVer", self->dbgid).detail("Commit", commitVersion).detail("Prev", prevVersion); + //TraceEvent("ProxyGotVer", pProxyContext->dbgid).detail("Commit", commitVersion).detail("Prev", prevVersion); - if (debugID.present()) - g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "MasterProxyServer.commitBatch.GotCommitVersion"); + if (debugID.present()) { + g_traceBatch.addEvent( + "CommitDebug", debugID.get().first(), + "MasterProxyServer.commitBatch.GotCommitVersion" + ); + } - ResolutionRequestBuilder requests( self, commitVersion, prevVersion, self->version, span ); + return Void(); +} + +ACTOR Future getResolution(Context* self) { + ProxyCommitData* pProxyCommitData = self->pProxyCommitData; + std::vector& trs = self->trs; + + ResolutionRequestBuilder requests( + pProxyCommitData, + self->commitVersion, + self->prevVersion, + pProxyCommitData->version, + self->span + ); int conflictRangeCount = 0; - state int64_t maxTransactionBytes = 0; - for (int t = 0; tmaxTransactionBytes = 0; + for (int t = 0; t < trs.size(); t++) { requests.addTransaction(trs[t].transaction, t); - conflictRangeCount += trs[t].transaction.read_conflict_ranges.size() + trs[t].transaction.write_conflict_ranges.size(); + conflictRangeCount += ( + trs[t].transaction.read_conflict_ranges.size() + + trs[t].transaction.write_conflict_ranges.size() + ); //TraceEvent("MPTransactionDump", self->dbgid).detail("Snapshot", trs[t].transaction.read_snapshot); //for(auto& m : trs[t].transaction.mutations) - maxTransactionBytes = std::max(maxTransactionBytes, trs[t].transaction.expectedSize()); + self->maxTransactionBytes = std::max( + self->maxTransactionBytes, trs[t].transaction.expectedSize() + ); // TraceEvent("MPTransactionsDump", self->dbgid).detail("Mutation", m.toString()); } - self->stats.conflictRanges += conflictRangeCount; + pProxyCommitData->stats.conflictRanges += conflictRangeCount; - for (int r = 1; rresolvers.size(); r++) - ASSERT(requests.requests[r].txnStateTransactions.size() == requests.requests[0].txnStateTransactions.size()); + for (int r = 1; r < pProxyCommitData->resolvers.size(); r++) + ASSERT(requests.requests[r].txnStateTransactions.size() == + requests.requests[0].txnStateTransactions.size()); - // Sending these requests is the fuzzy border between phase 1 and phase 2; it could conceivably overlap with resolution processing but is still using CPU - self->stats.txnCommitResolving += trs.size(); - vector< Future > replies; - for (int r = 0; rresolvers.size(); r++) { - requests.requests[r].debugID = debugID; - replies.push_back(brokenPromiseToNever(self->resolvers[r].resolve.getReply(requests.requests[r], TaskPriority::ProxyResolverReply))); + pProxyCommitData->stats.txnCommitResolving += trs.size(); + std::vector> replies; + for (int r = 0; r < pProxyCommitData->resolvers.size(); r++) { + requests.requests[r].debugID = self->debugID; + replies.push_back(brokenPromiseToNever( + pProxyCommitData->resolvers[r].resolve.getReply( + requests.requests[r], TaskPriority::ProxyResolverReply))); } - state vector> transactionResolverMap = std::move( requests.transactionResolverMap ); - state std::vector>> txReadConflictRangeIndexMap = - std::move(requests.txReadConflictRangeIndexMap); // used to report conflicting keys - state Future releaseFuture = releaseResolvingAfter(self, releaseDelay, localBatchNumber); + self->transactionResolverMap.swap(requests.transactionResolverMap); + // Used to report conflicting keys + self->txReadConflictRangeIndexMap.swap(requests.txReadConflictRangeIndexMap); + self->releaseFuture = releaseResolvingAfter( + pProxyCommitData, self->releaseDelay, self->localBatchNumber + ); + + // Wait for the final resolution + std::vector resolutionResp = wait(getAll(replies)); + self->resolution.swap(*const_cast*>(&resolutionResp)); + + if (self->debugID.present()) { + g_traceBatch.addEvent( + "CommitDebug", self->debugID.get().first(), + "MasterProxyServer.commitBatch.AfterResolution" + ); + } + + return Void(); +} + +} // namespace CommitBatch + +// Commit one batch of transactions trs +ACTOR Future commitBatch( + ProxyCommitData* self, + vector* trs, + int currentBatchMemBytesCount) { + //WARNING: this code is run at a high priority (until the first delay(0)), so it needs to do as little work as possible + state CommitBatch::Context context(self, trs, currentBatchMemBytesCount); + + // Active load balancing runs at a very high priority (to obtain accurate estimate of memory used by commit batches) so we need to downgrade here + wait(delay(0, TaskPriority::ProxyCommit)); + + context.pProxyCommitData->lastVersionTime = context.startTime; + ++context.pProxyCommitData->stats.commitBatchIn; + + /////// Phase 1: Pre-resolution processing (CPU bound except waiting for a version # which is separately pipelined and *should* be available by now (unless empty commit); ordered; currently atomic but could yield) + wait(CommitBatch::preresolutionProcessing(&context)); /////// Phase 2: Resolution (waiting on the network; pipelined) - state vector resolution = wait( getAll(replies) ); - - if (debugID.present()) - g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "MasterProxyServer.commitBatch.AfterResolution"); + wait(CommitBatch::getResolution(&context)); ////// Phase 3: Post-resolution processing (CPU bound except for very rare situations; ordered; currently atomic but doesn't need to be) - TEST(self->latestLocalCommitBatchLogging.get() < localBatchNumber - 1); // Queuing post-resolution commit processing - wait(self->latestLocalCommitBatchLogging.whenAtLeast(localBatchNumber-1)); + TEST(context.pProxyCommitData->latestLocalCommitBatchLogging.get() < context.localBatchNumber - 1); // Queuing post-resolution commit processing + wait(context.pProxyCommitData->latestLocalCommitBatchLogging.whenAtLeast(context.localBatchNumber - 1)); wait(yield(TaskPriority::ProxyCommitYield1)); state double computeStart = g_network->timer(); - state double computeDuration = 0; - self->stats.txnCommitResolved += trs.size(); + state double computeDuration = 0; + self->stats.txnCommitResolved += context.trs.size(); - if (debugID.present()) - g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "MasterProxyServer.commitBatch.ProcessingMutations"); + if (context.debugID.present()) { + g_traceBatch.addEvent( + "CommitDebug", context.debugID.get().first(), + "MasterProxyServer.commitBatch.ProcessingMutations" + ); + } state Arena arena; - state bool isMyFirstBatch = !self->version; - state Optional oldCoordinators = self->txnStateStore->readValue(coordinatorsKey).get(); + state bool isMyFirstBatch = !context.pProxyCommitData->version; + state Optional oldCoordinators = context.pProxyCommitData->txnStateStore->readValue(coordinatorsKey).get(); - //TraceEvent("ResolutionResult", self->dbgid).detail("Sequence", sequence).detail("Version", commitVersion).detail("StateMutationProxies", resolution[0].stateMutations.size()).detail("WaitForResolution", now()-t1).detail("R0Committed", resolution[0].committed.size()) + //TraceEvent("ResolutionResult", context.pProxyCommitData->dbgid).detail("Sequence", sequence).detail("Version", commitVersion).detail("StateMutationProxies", resolution[0].stateMutations.size()).detail("WaitForResolution", now()-t1).detail("R0Committed", resolution[0].committed.size()) // .detail("Transactions", trs.size()); - for(int r=1; r, Future> > storeCommits; - for (int versionIndex = 0; versionIndex < resolution[0].stateMutations.size(); versionIndex++) { - // self->logAdapter->setNextVersion( ??? ); << Ideally we would be telling the log adapter that the pushes in this commit will be in the version at which these state mutations were committed by another proxy, but at present we don't have that information here. So the disk queue may be unnecessarily conservative about popping. + for (int versionIndex = 0; versionIndex < context.resolution[0].stateMutations.size(); versionIndex++) { + // context.pProxyCommitData->logAdapter->setNextVersion( ??? ); << Ideally we would be telling the log adapter that the pushes in this commit will be in the version at which these state mutations were committed by another proxy, but at present we don't have that information here. So the disk queue may be unnecessarily conservative about popping. - for (int transactionIndex = 0; transactionIndex < resolution[0].stateMutations[versionIndex].size() && !forceRecovery; transactionIndex++) { + for (int transactionIndex = 0; transactionIndex < context.resolution[0].stateMutations[versionIndex].size() && !context.forceRecovery; transactionIndex++) { bool committed = true; - for (int resolver = 0; resolver < resolution.size(); resolver++) - committed = committed && resolution[resolver].stateMutations[versionIndex][transactionIndex].committed; + for (int resolver = 0; resolver < context.resolution.size(); resolver++) + committed = committed && context.resolution[resolver].stateMutations[versionIndex][transactionIndex].committed; if (committed) - applyMetadataMutations( self->dbgid, arena, resolution[0].stateMutations[versionIndex][transactionIndex].mutations, self->txnStateStore, nullptr, &forceRecovery, self->logSystem, 0, &self->vecBackupKeys, &self->keyInfo, &self->cacheInfo, self->firstProxy ? &self->uid_applyMutationsData : nullptr, self->commit, self->cx, &self->committedVersion, &self->storageCache, &self->tag_popped); + applyMetadataMutations( + context.pProxyCommitData->dbgid, arena, + context.resolution[0].stateMutations[versionIndex][transactionIndex].mutations, + context.pProxyCommitData->txnStateStore, nullptr, &context.forceRecovery, context.pProxyCommitData->logSystem, 0, &context.pProxyCommitData->vecBackupKeys, &context.pProxyCommitData->keyInfo, &context.pProxyCommitData->cacheInfo, context.pProxyCommitData->firstProxy ? &context.pProxyCommitData->uid_applyMutationsData : nullptr, context.pProxyCommitData->commit, context.pProxyCommitData->cx, &context.pProxyCommitData->committedVersion, &context.pProxyCommitData->storageCache, &context.pProxyCommitData->tag_popped); - if( resolution[0].stateMutations[versionIndex][transactionIndex].mutations.size() && firstStateMutations ) { + if( context.resolution[0].stateMutations[versionIndex][transactionIndex].mutations.size() && firstStateMutations ) { ASSERT(committed); firstStateMutations = false; - forceRecovery = false; + context.forceRecovery = false; } - //TraceEvent("MetadataTransaction", self->dbgid).detail("Committed", committed).detail("Mutations", resolution[0].stateMutations[versionIndex][transactionIndex].second.size()).detail("R1Mutations", resolution.back().stateMutations[versionIndex][transactionIndex].second.size()); + //TraceEvent("MetadataTransaction", context.pProxyCommitData->dbgid).detail("Committed", committed).detail("Mutations", resolution[0].stateMutations[versionIndex][transactionIndex].second.size()).detail("R1Mutations", resolution.back().stateMutations[versionIndex][transactionIndex].second.size()); } - //TraceEvent("MetadataBatch", self->dbgid).detail("Transactions", resolution[0].stateMutations[versionIndex].size()); + //TraceEvent("MetadataBatch", context.pProxyCommitData->dbgid).detail("Transactions", resolution[0].stateMutations[versionIndex].size()); // These changes to txnStateStore will be committed by the other proxy, so we simply discard the commit message - auto fcm = self->logAdapter->getCommitMessage(); - storeCommits.emplace_back(fcm, self->txnStateStore->commit()); + auto fcm = context.pProxyCommitData->logAdapter->getCommitMessage(); + storeCommits.emplace_back(fcm, context.pProxyCommitData->txnStateStore->commit()); //discardCommit( dbgid, fcm, txnStateStore->commit() ); if (initialState) { //TraceEvent("ResyncLog", dbgid); initialState = false; - forceRecovery = false; - self->txnStateStore->resyncLog(); + context.forceRecovery = false; + context.pProxyCommitData->txnStateStore->resyncLog(); for (auto &p : storeCommits) { ASSERT(!p.second.isReady()); @@ -988,33 +1142,33 @@ ACTOR Future commitBatch( } // Determine which transactions actually committed (conservatively) by combining results from the resolvers - state vector committed(trs.size()); - ASSERT(transactionResolverMap.size() == committed.size()); + state vector committed(context.trs.size()); + ASSERT(context.transactionResolverMap.size() == committed.size()); // For each commitTransactionRef, it is only sent to resolvers specified in transactionResolverMap // Thus, we use this nextTr to track the correct transaction index on each resolver. - state vector nextTr(resolution.size()); - for (int t = 0; t nextTr(context.resolution.size()); + for (int t = 0; t < context.trs.size(); t++) { uint8_t commit = ConflictBatch::TransactionCommitted; - for (int r : transactionResolverMap[t]) + for (int r : context.transactionResolverMap[t]) { - commit = std::min(resolution[r].committed[nextTr[r]++], commit); + commit = std::min(context.resolution[r].committed[nextTr[r]++], commit); } committed[t] = commit; } - for (int r = 0; rlogAdapter->setNextVersion(commitVersion); + context.pProxyCommitData->logAdapter->setNextVersion(context.commitVersion); - state Optional lockedKey = self->txnStateStore->readValue(databaseLockedKey).get(); + state Optional lockedKey = context.pProxyCommitData->txnStateStore->readValue(databaseLockedKey).get(); state bool locked = lockedKey.present() && lockedKey.get().size(); - state Optional mustContainSystemKey = self->txnStateStore->readValue(mustContainSystemMutationsKey).get(); + state Optional mustContainSystemKey = context.pProxyCommitData->txnStateStore->readValue(mustContainSystemMutationsKey).get(); if(mustContainSystemKey.present() && mustContainSystemKey.get().size()) { - for (int t = 0; t= nonMetadataSystemKeys.end) { foundSystem = true; break; @@ -1027,44 +1181,61 @@ ACTOR Future commitBatch( } } - if(forceRecovery) { + if(context.forceRecovery) { wait( Future(Never()) ); } // This first pass through committed transactions deals with "metadata" effects (modifications of txnStateStore, changes to storage servers' responsibilities) int t; state int commitCount = 0; - for (t = 0; t < trs.size() && !forceRecovery; t++) + for (t = 0; t < context.trs.size() && !context.forceRecovery; t++) { - if (committed[t] == ConflictBatch::TransactionCommitted && (!locked || trs[t].isLockAware())) { + if (committed[t] == ConflictBatch::TransactionCommitted && (!locked || context.trs[t].isLockAware())) { commitCount++; - applyMetadataMutations(self->dbgid, arena, trs[t].transaction.mutations, self->txnStateStore, &toCommit, &forceRecovery, self->logSystem, commitVersion+1, &self->vecBackupKeys, &self->keyInfo, &self->cacheInfo, self->firstProxy ? &self->uid_applyMutationsData : NULL, self->commit, self->cx, &self->committedVersion, &self->storageCache, &self->tag_popped); + applyMetadataMutations( + context.pProxyCommitData->dbgid, arena, + context.trs[t].transaction.mutations, + context.pProxyCommitData->txnStateStore, + &context.toCommit, + &context.forceRecovery, + context.pProxyCommitData->logSystem, + context.commitVersion + 1, + &context.pProxyCommitData->vecBackupKeys, + &context.pProxyCommitData->keyInfo, + &context.pProxyCommitData->cacheInfo, + context.pProxyCommitData->firstProxy ? &context.pProxyCommitData->uid_applyMutationsData : nullptr, + context.pProxyCommitData->commit, + context.pProxyCommitData->cx, + &context.pProxyCommitData->committedVersion, + &context.pProxyCommitData->storageCache, + &context.pProxyCommitData->tag_popped + ); } if(firstStateMutations) { ASSERT(committed[t] == ConflictBatch::TransactionCommitted); firstStateMutations = false; - forceRecovery = false; + context.forceRecovery = false; } } - if (forceRecovery) { - for (; tdbgid).detail("Stage", "AwaitCommit"); + TraceEvent(SevWarn, "RestartingTxnSubsystem", context.pProxyCommitData->dbgid).detail("Stage", "AwaitCommit"); } - lockedKey = self->txnStateStore->readValue(databaseLockedKey).get(); + lockedKey = context.pProxyCommitData->txnStateStore->readValue(databaseLockedKey).get(); state bool lockedAfter = lockedKey.present() && lockedKey.get().size(); - state Optional metadataVersionAfter = self->txnStateStore->readValue(metadataVersionKey).get(); + state Optional metadataVersionAfter = context.pProxyCommitData->txnStateStore->readValue(metadataVersionKey).get(); - auto fcm = self->logAdapter->getCommitMessage(); - storeCommits.emplace_back(fcm, self->txnStateStore->commit()); - self->version = commitVersion; - if (!self->validState.isSet()) self->validState.send(Void()); - ASSERT(commitVersion); + auto fcm = context.pProxyCommitData->logAdapter->getCommitMessage(); + storeCommits.emplace_back(fcm, context.pProxyCommitData->txnStateStore->commit()); + context.pProxyCommitData->version = context.commitVersion; + if (!context.pProxyCommitData->validState.isSet()) context.pProxyCommitData->validState.send(Void()); + ASSERT(context.commitVersion); - if (!isMyFirstBatch && self->txnStateStore->readValue( coordinatorsKey ).get().get() != oldCoordinators.get()) { - wait( brokenPromiseToNever( self->master.changeCoordinators.getReply( ChangeCoordinatorsRequest( self->txnStateStore->readValue( coordinatorsKey ).get().get() ) ) ) ); + if (!isMyFirstBatch && context.pProxyCommitData->txnStateStore->readValue( coordinatorsKey ).get().get() != oldCoordinators.get()) { + wait( brokenPromiseToNever( context.pProxyCommitData->master.changeCoordinators.getReply( ChangeCoordinatorsRequest( context.pProxyCommitData->txnStateStore->readValue( coordinatorsKey ).get().get() ) ) ) ); ASSERT(false); // ChangeCoordinatorsRequest should always throw } @@ -1077,10 +1248,10 @@ ACTOR Future commitBatch( state int transactionNum = 0; state int yieldBytes = 0; - for (; transactionNum* pMutations = &trs[transactionNum].transaction.mutations; + state VectorRef* pMutations = &context.trs[transactionNum].transaction.mutations; for (; mutationNum < pMutations->size(); mutationNum++) { if(yieldBytes > SERVER_KNOBS->DESIRED_TOTAL_BYTES) { yieldBytes = 0; @@ -1099,37 +1270,37 @@ ACTOR Future commitBatch( // if necessary. Serialize (splits of) the mutation into the message buffer and add the tags. if (isSingleKeyMutation((MutationRef::Type) m.type)) { - auto& tags = self->tagsForKey(m.param1); + auto& tags = context.pProxyCommitData->tagsForKey(m.param1); - if(self->singleKeyMutationEvent->enabled) { - KeyRangeRef shard = self->keyInfo.rangeContaining(m.param1).range(); - self->singleKeyMutationEvent->tag1 = (int64_t)tags[0].id; - self->singleKeyMutationEvent->tag2 = (int64_t)tags[1].id; - self->singleKeyMutationEvent->tag3 = (int64_t)tags[2].id; - self->singleKeyMutationEvent->shardBegin = shard.begin; - self->singleKeyMutationEvent->shardEnd = shard.end; - self->singleKeyMutationEvent->log(); + if(context.pProxyCommitData->singleKeyMutationEvent->enabled) { + KeyRangeRef shard = context.pProxyCommitData->keyInfo.rangeContaining(m.param1).range(); + context.pProxyCommitData->singleKeyMutationEvent->tag1 = (int64_t)tags[0].id; + context.pProxyCommitData->singleKeyMutationEvent->tag2 = (int64_t)tags[1].id; + context.pProxyCommitData->singleKeyMutationEvent->tag3 = (int64_t)tags[2].id; + context.pProxyCommitData->singleKeyMutationEvent->shardBegin = shard.begin; + context.pProxyCommitData->singleKeyMutationEvent->shardEnd = shard.end; + context.pProxyCommitData->singleKeyMutationEvent->log(); } - DEBUG_MUTATION("ProxyCommit", commitVersion, m).detail("Dbgid", self->dbgid).detail("To", tags).detail("Mutation", m); - - toCommit.addTags(tags); - if(self->cacheInfo[m.param1]) { - toCommit.addTag(cacheTag); + DEBUG_MUTATION("ProxyCommit", context.commitVersion, m).detail("Dbgid", context.pProxyCommitData->dbgid).detail("To", tags).detail("Mutation", m); + + context.toCommit.addTags(tags); + if(context.pProxyCommitData->cacheInfo[m.param1]) { + context.toCommit.addTag(cacheTag); } - toCommit.addTypedMessage(m); + context.toCommit.addTypedMessage(m); } else if (m.type == MutationRef::ClearRange) { KeyRangeRef clearRange(KeyRangeRef(m.param1, m.param2)); - auto ranges = self->keyInfo.intersectingRanges(clearRange); + auto ranges = context.pProxyCommitData->keyInfo.intersectingRanges(clearRange); auto firstRange = ranges.begin(); ++firstRange; if (firstRange == ranges.end()) { // Fast path - DEBUG_MUTATION("ProxyCommit", commitVersion, m).detail("Dbgid", self->dbgid).detail("To", ranges.begin().value().tags).detail("Mutation", m); + DEBUG_MUTATION("ProxyCommit", context.commitVersion, m).detail("Dbgid", context.pProxyCommitData->dbgid).detail("To", ranges.begin().value().tags).detail("Mutation", m); ranges.begin().value().populateTags(); - toCommit.addTags(ranges.begin().value().tags); + context.toCommit.addTags(ranges.begin().value().tags); } else { TEST(true); //A clear range extends past a shard boundary @@ -1138,23 +1309,23 @@ ACTOR Future commitBatch( r.value().populateTags(); allSources.insert(r.value().tags.begin(), r.value().tags.end()); } - DEBUG_MUTATION("ProxyCommit", commitVersion, m).detail("Dbgid", self->dbgid).detail("To", allSources).detail("Mutation", m); + DEBUG_MUTATION("ProxyCommit", context.commitVersion, m).detail("Dbgid", context.pProxyCommitData->dbgid).detail("To", allSources).detail("Mutation", m); - toCommit.addTags(allSources); + context.toCommit.addTags(allSources); } - if(self->needsCacheTag(clearRange)) { - toCommit.addTag(cacheTag); + if(context.pProxyCommitData->needsCacheTag(clearRange)) { + context.toCommit.addTag(cacheTag); } - toCommit.addTypedMessage(m); - } else + context.toCommit.addTypedMessage(m); + } else { UNREACHABLE(); - + } // Check on backing up key, if backup ranges are defined and a normal key - if (self->vecBackupKeys.size() > 1 && (normalKeys.contains(m.param1) || m.param1 == metadataVersionKey)) { + if (context.pProxyCommitData->vecBackupKeys.size() > 1 && (normalKeys.contains(m.param1) || m.param1 == metadataVersionKey)) { if (m.type != MutationRef::Type::ClearRange) { // Add the mutation to the relevant backup tag - for (auto backupName : self->vecBackupKeys[m.param1]) { + for (auto backupName : context.pProxyCommitData->vecBackupKeys[m.param1]) { logRangeMutations[backupName].push_back_deep(logRangeMutationsArena, m); } } @@ -1163,7 +1334,7 @@ ACTOR Future commitBatch( KeyRangeRef intersectionRange; // Identify and add the intersecting ranges of the mutation to the array of mutations to serialize - for (auto backupRange : self->vecBackupKeys.intersectingRanges(mutationRange)) + for (auto backupRange : context.pProxyCommitData->vecBackupKeys.intersectingRanges(mutationRange)) { // Get the backup sub range const auto& backupSubrange = backupRange.range(); @@ -1186,38 +1357,38 @@ ACTOR Future commitBatch( } // Serialize and backup the mutations as a single mutation - if ((self->vecBackupKeys.size() > 1) && logRangeMutations.size()) { - wait( addBackupMutations(self, &logRangeMutations, &toCommit, commitVersion, &computeDuration, &computeStart) ); + if ((context.pProxyCommitData->vecBackupKeys.size() > 1) && logRangeMutations.size()) { + wait( addBackupMutations(context.pProxyCommitData, &logRangeMutations, &context.toCommit, context.commitVersion, &computeDuration, &computeStart) ); } - self->stats.mutations += mutationCount; - self->stats.mutationBytes += mutationBytes; + context.pProxyCommitData->stats.mutations += mutationCount; + context.pProxyCommitData->stats.mutationBytes += mutationBytes; // Storage servers mustn't make durable versions which are not fully committed (because then they are impossible to roll back) // We prevent this by limiting the number of versions which are semi-committed but not fully committed to be less than the MVCC window - if(self->committedVersion.get() < commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS) { + if(context.pProxyCommitData->committedVersion.get() < context.commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS) { computeDuration += g_network->timer() - computeStart; state Span waitVersionSpan; - while (self->committedVersion.get() < commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS) { + while (context.pProxyCommitData->committedVersion.get() < context.commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS) { // This should be *extremely* rare in the real world, but knob buggification should make it happen in simulation TEST(true); // Semi-committed pipeline limited by MVCC window - //TraceEvent("ProxyWaitingForCommitted", self->dbgid).detail("CommittedVersion", self->committedVersion.get()).detail("NeedToCommit", commitVersion); - waitVersionSpan = Span(deterministicRandom()->randomUniqueID(), "MP:overMaxReadTransactionLifeVersions"_loc, {span.context}); + //TraceEvent("ProxyWaitingForCommitted", context.pProxyCommitData->dbgid).detail("CommittedVersion", context.pProxyCommitData->committedVersion.get()).detail("NeedToCommit", commitVersion); + waitVersionSpan = Span(deterministicRandom()->randomUniqueID(), "MP:overMaxReadTransactionLifeVersions"_loc, {context.span.context}); choose{ - when(wait(self->committedVersion.whenAtLeast(commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS))) { + when(wait(context.pProxyCommitData->committedVersion.whenAtLeast(context.commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS))) { wait(yield()); - break; + break; } - when(GetReadVersionReply v = wait(self->getConsistentReadVersion.getReply( - GetReadVersionRequest(waitVersionSpan.context, 0, TransactionPriority::IMMEDIATE, - GetReadVersionRequest::FLAG_CAUSAL_READ_RISKY)))) { - if (v.version > self->committedVersion.get()) { - self->locked = v.locked; - self->metadataVersion = v.metadataVersion; - self->committedVersion.set(v.version); + when(GetReadVersionReply v = wait(context.pProxyCommitData->getConsistentReadVersion.getReply( + GetReadVersionRequest(waitVersionSpan.context, 0, TransactionPriority::IMMEDIATE, + GetReadVersionRequest::FLAG_CAUSAL_READ_RISKY)))) { + if(v.version > context.pProxyCommitData->committedVersion.get()) { + context.pProxyCommitData->locked = v.locked; + context.pProxyCommitData->metadataVersion = v.metadataVersion; + context.pProxyCommitData->committedVersion.set(v.version); } - if (self->committedVersion.get() < commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS) + if (context.pProxyCommitData->committedVersion.get() < context.commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS) wait(delay(SERVER_KNOBS->PROXY_SPIN_DELAY)); } } @@ -1228,44 +1399,44 @@ ACTOR Future commitBatch( state LogSystemDiskQueueAdapter::CommitMessage msg = storeCommits.back().first.get(); - if (debugID.present()) - g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "MasterProxyServer.commitBatch.AfterStoreCommits"); + if (context.debugID.present()) + g_traceBatch.addEvent("CommitDebug", context.debugID.get().first(), "MasterProxyServer.commitBatch.AfterStoreCommits"); // txnState (transaction subsystem state) tag: message extracted from log adapter bool firstMessage = true; for(auto m : msg.messages) { if(firstMessage) { - toCommit.addTxsTag(); + context.toCommit.addTxsTag(); } - toCommit.addMessage(StringRef(m.begin(), m.size()), !firstMessage); + context.toCommit.addMessage(StringRef(m.begin(), m.size()), !firstMessage); firstMessage = false; } - if ( prevVersion && commitVersion - prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT/2 ) - debug_advanceMaxCommittedVersion( UID(), commitVersion ); //< Is this valid? + if ( context.prevVersion && context.commitVersion - context.prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT/2 ) + debug_advanceMaxCommittedVersion( UID(), context.commitVersion ); //< Is this valid? - //TraceEvent("ProxyPush", self->dbgid).detail("PrevVersion", prevVersion).detail("Version", commitVersion) + //TraceEvent("ProxyPush", context.pProxyCommitData->dbgid).detail("PrevVersion", prevVersion).detail("Version", commitVersion) // .detail("TransactionsSubmitted", trs.size()).detail("TransactionsCommitted", commitCount).detail("TxsPopTo", msg.popTo); - if ( prevVersion && commitVersion - prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT/2 ) - debug_advanceMaxCommittedVersion(UID(), commitVersion); + if ( context.prevVersion && context.commitVersion - context.prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT/2 ) + debug_advanceMaxCommittedVersion(UID(), context.commitVersion); state double commitStartTime = now(); - self->lastStartCommit = commitStartTime; - Future loggingComplete = self->logSystem->push( prevVersion, commitVersion, self->committedVersion.get(), self->minKnownCommittedVersion, toCommit, debugID ); + context.pProxyCommitData->lastStartCommit = commitStartTime; + Future loggingComplete = context.pProxyCommitData->logSystem->push( context.prevVersion, context.commitVersion, context.pProxyCommitData->committedVersion.get(), context.pProxyCommitData->minKnownCommittedVersion, context.toCommit, context.debugID ); - if (!forceRecovery) { - ASSERT(self->latestLocalCommitBatchLogging.get() == localBatchNumber-1); - self->latestLocalCommitBatchLogging.set(localBatchNumber); + if (!context.forceRecovery) { + ASSERT(context.pProxyCommitData->latestLocalCommitBatchLogging.get() == context.localBatchNumber-1); + context.pProxyCommitData->latestLocalCommitBatchLogging.set(context.localBatchNumber); } computeDuration += g_network->timer() - computeStart; - if(computeDuration > SERVER_KNOBS->MIN_PROXY_COMPUTE && batchOperations > 0) { - double computePerOperation = computeDuration/batchOperations; - if(computePerOperation <= self->commitComputePerOperation[latencyBucket]) { - self->commitComputePerOperation[latencyBucket] = computePerOperation; + if(computeDuration > SERVER_KNOBS->MIN_PROXY_COMPUTE && context.batchOperations > 0) { + double computePerOperation = computeDuration / context.batchOperations; + if(computePerOperation <= context.pProxyCommitData->commitComputePerOperation[context.latencyBucket]) { + context.pProxyCommitData->commitComputePerOperation[context.latencyBucket] = computePerOperation; } else { - self->commitComputePerOperation[latencyBucket] = SERVER_KNOBS->PROXY_COMPUTE_GROWTH_RATE*computePerOperation + ((1.0-SERVER_KNOBS->PROXY_COMPUTE_GROWTH_RATE)*self->commitComputePerOperation[latencyBucket]); + context.pProxyCommitData->commitComputePerOperation[context.latencyBucket] = SERVER_KNOBS->PROXY_COMPUTE_GROWTH_RATE*computePerOperation + ((1.0-SERVER_KNOBS->PROXY_COMPUTE_GROWTH_RATE)*context.pProxyCommitData->commitComputePerOperation[context.latencyBucket]); } } @@ -1274,9 +1445,9 @@ ACTOR Future commitBatch( try { choose { when(Version ver = wait(loggingComplete)) { - self->minKnownCommittedVersion = std::max(self->minKnownCommittedVersion, ver); + context.pProxyCommitData->minKnownCommittedVersion = std::max(context.pProxyCommitData->minKnownCommittedVersion, ver); } - when(wait(self->committedVersion.whenAtLeast( commitVersion+1 ))) {} + when(wait(context.pProxyCommitData->committedVersion.whenAtLeast( context.commitVersion+1 ))) {} } } catch(Error &e) { if(e.code() == error_code_broken_promise) { @@ -1285,27 +1456,28 @@ ACTOR Future commitBatch( throw; } - self->lastCommitLatency = now()-commitStartTime; - self->lastCommitTime = std::max(self->lastCommitTime.get(), commitStartTime); + context.pProxyCommitData->lastCommitLatency = now()-commitStartTime; + context.pProxyCommitData->lastCommitTime = std::max(context.pProxyCommitData->lastCommitTime.get(), commitStartTime); + wait(yield(TaskPriority::ProxyCommitYield2)); - if( self->popRemoteTxs && msg.popTo > ( self->txsPopVersions.size() ? self->txsPopVersions.back().second : self->lastTxsPop ) ) { - if(self->txsPopVersions.size() >= SERVER_KNOBS->MAX_TXS_POP_VERSION_HISTORY) { + if( context.pProxyCommitData->popRemoteTxs && msg.popTo > ( context.pProxyCommitData->txsPopVersions.size() ? context.pProxyCommitData->txsPopVersions.back().second : context.pProxyCommitData->lastTxsPop ) ) { + if(context.pProxyCommitData->txsPopVersions.size() >= SERVER_KNOBS->MAX_TXS_POP_VERSION_HISTORY) { TraceEvent(SevWarnAlways, "DiscardingTxsPopHistory").suppressFor(1.0); - self->txsPopVersions.pop_front(); + context.pProxyCommitData->txsPopVersions.pop_front(); } - self->txsPopVersions.emplace_back(commitVersion, msg.popTo); + context.pProxyCommitData->txsPopVersions.emplace_back(context.commitVersion, msg.popTo); } - self->logSystem->popTxs(msg.popTo); + context.pProxyCommitData->logSystem->popTxs(msg.popTo); /////// Phase 5: Replies (CPU bound; no particular order required, though ordered execution would be best for latency) - if ( prevVersion && commitVersion - prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT/2 ) - debug_advanceMinCommittedVersion(UID(), commitVersion); + if ( context.prevVersion && context.commitVersion - context.prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT/2 ) + debug_advanceMinCommittedVersion(UID(), context.commitVersion); - //TraceEvent("ProxyPushed", self->dbgid).detail("PrevVersion", prevVersion).detail("Version", commitVersion); - if (debugID.present()) - g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "MasterProxyServer.commitBatch.AfterLogPush"); + //TraceEvent("ProxyPushed", context.pProxyCommitData->dbgid).detail("PrevVersion", prevVersion).detail("Version", commitVersion); + if (context.debugID.present()) + g_traceBatch.addEvent("CommitDebug", context.debugID.get().first(), "MasterProxyServer.commitBatch.AfterLogPush"); for (auto &p : storeCommits) { ASSERT(!p.second.isReady()); @@ -1317,18 +1489,18 @@ ACTOR Future commitBatch( // up-to-date live committed version. We also maintain the invariant that master's committed version >= self->committedVersion // by reporting commit version first before updating self->committedVersion. Otherwise, a client may get a commit // version that the master is not aware of, and next GRV request may get a version less than self->committedVersion. - TEST(self->committedVersion.get() > commitVersion); // A later version was reported committed first - if (SERVER_KNOBS->ASK_READ_VERSION_FROM_MASTER && commitVersion > self->committedVersion.get()) { - wait(self->master.reportLiveCommittedVersion.getReply(ReportRawCommittedVersionRequest(commitVersion, lockedAfter, metadataVersionAfter), TaskPriority::ProxyMasterVersionReply)); + TEST(context.pProxyCommitData->committedVersion.get() > context.commitVersion); // A later version was reported committed first + if (SERVER_KNOBS->ASK_READ_VERSION_FROM_MASTER && context.commitVersion > context.pProxyCommitData->committedVersion.get()) { + wait(context.pProxyCommitData->master.reportLiveCommittedVersion.getReply(ReportRawCommittedVersionRequest(context.commitVersion, lockedAfter, metadataVersionAfter), TaskPriority::ProxyMasterVersionReply)); } - if( commitVersion > self->committedVersion.get() ) { - self->locked = lockedAfter; - self->metadataVersion = metadataVersionAfter; - self->committedVersion.set(commitVersion); + if( context.commitVersion > context.pProxyCommitData->committedVersion.get() ) { + context.pProxyCommitData->locked = lockedAfter; + context.pProxyCommitData->metadataVersion = metadataVersionAfter; + context.pProxyCommitData->committedVersion.set(context.commitVersion); } - if (forceRecovery) { - TraceEvent(SevWarn, "RestartingTxnSubsystem", self->dbgid).detail("Stage", "ProxyShutdown"); + if (context.forceRecovery) { + TraceEvent(SevWarn, "RestartingTxnSubsystem", context.pProxyCommitData->dbgid).detail("Stage", "ProxyShutdown"); throw worker_removed(); } @@ -1336,91 +1508,92 @@ ACTOR Future commitBatch( double endTime = g_network->timer(); // Reset all to zero, used to track the correct index of each commitTransacitonRef on each resolver std::fill(nextTr.begin(), nextTr.end(), 0); - for (int t = 0; t < trs.size(); t++) { - if (committed[t] == ConflictBatch::TransactionCommitted && (!locked || trs[t].isLockAware())) { - ASSERT_WE_THINK(commitVersion != invalidVersion); - trs[t].reply.send(CommitID(commitVersion, t, metadataVersionAfter)); + for (int t = 0; t < context.trs.size(); t++) { + if (committed[t] == ConflictBatch::TransactionCommitted && (!locked || context.trs[t].isLockAware())) { + ASSERT_WE_THINK(context.commitVersion != invalidVersion); + context.trs[t].reply.send(CommitID(context.commitVersion, t, metadataVersionAfter)); + // aggregate commit cost estimation if committed - ASSERT(trs[t].commitCostEstimation.present() == trs[t].tagSet.present()); - if (trs[t].tagSet.present()) { - TransactionCommitCostEstimation& costEstimation = trs[t].commitCostEstimation.get(); - for (auto& tag : trs[t].tagSet.get()) { + ASSERT(context.trs[t].commitCostEstimation.present() == context.trs[t].tagSet.present()); + if (context.trs[t].tagSet.present()) { + TransactionCommitCostEstimation& costEstimation = context.trs[t].commitCostEstimation.get(); + for (auto& tag : context.trs[t].tagSet.get()) { self->transactionTagCommitCostEst[tag] += costEstimation; } } } else if (committed[t] == ConflictBatch::TransactionTooOld) { - trs[t].reply.sendError(transaction_too_old()); + context.trs[t].reply.sendError(transaction_too_old()); } else { // If enable the option to report conflicting keys from resolvers, we send back all keyranges' indices // through CommitID - if (trs[t].transaction.report_conflicting_keys) { + if (context.trs[t].transaction.report_conflicting_keys) { Standalone> conflictingKRIndices; - for (int resolverInd : transactionResolverMap[t]) { + for (int resolverInd : context.transactionResolverMap[t]) { auto const& cKRs = - resolution[resolverInd] + context.resolution[resolverInd] .conflictingKeyRangeMap[nextTr[resolverInd]]; // nextTr[resolverInd] -> index of this trs[t] // on the resolver for (auto const& rCRIndex : cKRs) // read_conflict_range can change when sent to resolvers, mapping the index from resolver-side // to original index in commitTransactionRef conflictingKRIndices.push_back(conflictingKRIndices.arena(), - txReadConflictRangeIndexMap[t][resolverInd][rCRIndex]); + context.txReadConflictRangeIndexMap[t][resolverInd][rCRIndex]); } // At least one keyRange index should be returned ASSERT(conflictingKRIndices.size()); - trs[t].reply.send(CommitID(invalidVersion, t, Optional(), + context.trs[t].reply.send(CommitID(invalidVersion, t, Optional(), Optional>>(conflictingKRIndices))); } else { - trs[t].reply.sendError(not_committed()); + context.trs[t].reply.sendError(not_committed()); } } // Update corresponding transaction indices on each resolver - for (int resolverInd : transactionResolverMap[t]) nextTr[resolverInd]++; + for (int resolverInd : context.transactionResolverMap[t]) nextTr[resolverInd]++; // TODO: filter if pipelined with large commit - double duration = endTime - trs[t].requestTime(); - self->stats.commitLatencySample.addMeasurement(duration); - if(self->latencyBandConfig.present()) { - bool filter = maxTransactionBytes > self->latencyBandConfig.get().commitConfig.maxCommitBytes.orDefault(std::numeric_limits::max()); - self->stats.commitLatencyBands.addMeasurement(duration, filter); + double duration = endTime - context.trs[t].requestTime(); + context.pProxyCommitData->stats.commitLatencySample.addMeasurement(duration); + if(context.pProxyCommitData->latencyBandConfig.present()) { + bool filter = context.maxTransactionBytes > context.pProxyCommitData->latencyBandConfig.get().commitConfig.maxCommitBytes.orDefault(std::numeric_limits::max()); + context.pProxyCommitData->stats.commitLatencyBands.addMeasurement(endTime - context.trs[t].requestTime(), filter); } } - ++self->stats.commitBatchOut; - self->stats.txnCommitOut += trs.size(); - self->stats.txnConflicts += trs.size() - commitCount; - self->stats.txnCommitOutSuccess += commitCount; + ++context.pProxyCommitData->stats.commitBatchOut; + context.pProxyCommitData->stats.txnCommitOut += context.trs.size(); + context.pProxyCommitData->stats.txnConflicts += context.trs.size() - commitCount; + context.pProxyCommitData->stats.txnCommitOutSuccess += commitCount; - if(now() - self->lastCoalesceTime > SERVER_KNOBS->RESOLVER_COALESCE_TIME) { - self->lastCoalesceTime = now(); - int lastSize = self->keyResolvers.size(); - auto rs = self->keyResolvers.ranges(); - Version oldestVersion = prevVersion - SERVER_KNOBS->MAX_WRITE_TRANSACTION_LIFE_VERSIONS; + if(now() - context.pProxyCommitData->lastCoalesceTime > SERVER_KNOBS->RESOLVER_COALESCE_TIME) { + context.pProxyCommitData->lastCoalesceTime = now(); + int lastSize = context.pProxyCommitData->keyResolvers.size(); + auto rs = context.pProxyCommitData->keyResolvers.ranges(); + Version oldestVersion = context.prevVersion - SERVER_KNOBS->MAX_WRITE_TRANSACTION_LIFE_VERSIONS; for(auto r = rs.begin(); r != rs.end(); ++r) { while(r->value().size() > 1 && r->value()[1].first < oldestVersion) r->value().pop_front(); if(r->value().size() && r->value().front().first < oldestVersion) r->value().front().first = 0; } - self->keyResolvers.coalesce(allKeys); - if(self->keyResolvers.size() != lastSize) - TraceEvent("KeyResolverSize", self->dbgid).detail("Size", self->keyResolvers.size()); + context.pProxyCommitData->keyResolvers.coalesce(allKeys); + if(context.pProxyCommitData->keyResolvers.size() != lastSize) + TraceEvent("KeyResolverSize", context.pProxyCommitData->dbgid).detail("Size", context.pProxyCommitData->keyResolvers.size()); } // Dynamic batching for commits - double target_latency = (now() - t1) * SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_LATENCY_FRACTION; - self->commitBatchInterval = std::max( + double target_latency = (now() - context.startTime) * SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_LATENCY_FRACTION; + context.pProxyCommitData->commitBatchInterval = std::max( SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_MIN, std::min(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_MAX, target_latency * SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA + - self->commitBatchInterval * (1 - SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA))); + context.pProxyCommitData->commitBatchInterval * (1 - SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA))); - self->commitBatchesMemBytesCount -= currentBatchMemBytesCount; - ASSERT_ABORT(self->commitBatchesMemBytesCount >= 0); - wait(releaseFuture); + context.pProxyCommitData->commitBatchesMemBytesCount -= currentBatchMemBytesCount; + ASSERT_ABORT(context.pProxyCommitData->commitBatchesMemBytesCount >= 0); + wait(context.releaseFuture); return Void(); } @@ -1515,7 +1688,7 @@ ACTOR Future sendGrvReplies(Future replyFuture, std:: if (request.flags & GetReadVersionRequest::FLAG_USE_MIN_KNOWN_COMMITTED_VERSION) { // Only backup worker may infrequently use this flag. reply.version = minKnownCommittedVersion; - } + } else { reply.version = replyVersion; } @@ -1640,7 +1813,7 @@ ACTOR static Future transactionStarter( break; } else if(req.priority < TransactionPriority::IMMEDIATE && !normalRateInfo.canStart(transactionsStarted[0] + transactionsStarted[1], tc)) { - break; + break; } if (req.debugID.present()) { From a7cff484bcc016a64488cc6b5d7916c68908d313 Mon Sep 17 00:00:00 2001 From: Xiaoge Su Date: Sat, 25 Jul 2020 21:55:22 -0700 Subject: [PATCH 2/8] Extract Step 3 from commitBatch --- fdbserver/MasterProxyServer.actor.cpp | 830 ++++++++++++++------------ 1 file changed, 439 insertions(+), 391 deletions(-) diff --git a/fdbserver/MasterProxyServer.actor.cpp b/fdbserver/MasterProxyServer.actor.cpp index efdc097377..8cf9664e7c 100644 --- a/fdbserver/MasterProxyServer.actor.cpp +++ b/fdbserver/MasterProxyServer.actor.cpp @@ -811,6 +811,11 @@ ACTOR Future releaseResolvingAfter(ProxyCommitData* self, Future rel namespace CommitBatch { struct Context { + typedef std::vector< + std::pair< + Future, Future + >> StoreCommit_t; + ProxyCommitData* const pProxyCommitData; std::vector trs; int currentBatchMemBytesCount; @@ -844,6 +849,48 @@ struct Context { std::vector resolution; + double computeStart; + double computeDuration = 0; + + Arena arena; + + /// true if the batch is the 1st batch for this proxy, additional metadata + /// processing is involved for this batch. + bool isMyFirstBatch; + bool firstStateMutations; + + Optional oldCoordinators; + + StoreCommit_t storeCommits; + + std::vector committed; + + Optional lockedKey; + bool locked; + + int commitCount = 0; + + std::vector nextTr; + + bool lockedAfter; + + Optional metadataVersionAfter; + + int mutationCount = 0; + int mutationBytes = 0; + + std::map logRangeMutations; + Arena logRangeMutationsArena; + + int transactionNum = 0; + int yieldBytes = 0; + + LogSystemDiskQueueAdapter::CommitMessage msg; + + Future loggingComplete; + + double commitStartTime; + Context( ProxyCommitData*, const std::vector*, @@ -867,7 +914,9 @@ Context::Context( startTime(g_network->now()), localBatchNumber(++pProxyCommitData->localCommitBatchesStarted), - toCommit(pProxyCommitData->logSystem) { + toCommit(pProxyCommitData->logSystem), + + committed(trs.size()) { evaluateBatchSize(); @@ -1044,6 +1093,376 @@ ACTOR Future getResolution(Context* self) { return Void(); } +ACTOR Future postResolution(Context* self) { + state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData; + state std::vector& trs = self->trs; + state const int64_t localBatchNumber = self->localBatchNumber; + state const Optional& debugID = self->debugID; + + TEST(pProxyCommitData->latestLocalCommitBatchLogging.get() < localBatchNumber - 1); // Queuing post-resolution commit processing + wait(pProxyCommitData->latestLocalCommitBatchLogging.whenAtLeast(localBatchNumber - 1)); + wait(yield(TaskPriority::ProxyCommitYield1)); + + self->computeStart = g_network->now(); + + pProxyCommitData->stats.txnCommitResolved += trs.size(); + + if (debugID.present()) { + g_traceBatch.addEvent( + "CommitDebug", debugID.get().first(), + "MasterProxyServer.commitBatch.ProcessingMutations" + ); + } + + self->isMyFirstBatch = !pProxyCommitData->version; + self->oldCoordinators = pProxyCommitData->txnStateStore->readValue(coordinatorsKey).get(); + + //TraceEvent("ResolutionResult", pProxyCommitData->dbgid).detail("Sequence", sequence).detail("Version", commitVersion).detail("StateMutationProxies", resolution[0].stateMutations.size()).detail("WaitForResolution", now()-t1).detail("R0Committed", resolution[0].committed.size()) + // .detail("Transactions", trs.size()); + + for(int r=1; r < self->resolution.size(); r++) { + ASSERT( self->resolution[r].stateMutations.size() == self->resolution[0].stateMutations.size() ); + for(int s = 0; s < self->resolution[r].stateMutations.size(); s++) + ASSERT( self->resolution[r].stateMutations[s].size() == self->resolution[0].stateMutations[s].size() ); + } + + // Compute and apply "metadata" effects of each other proxy's most recent batch + bool initialState = self->isMyFirstBatch; + self->firstStateMutations = self->isMyFirstBatch; + for (int versionIndex = 0; versionIndex < self->resolution[0].stateMutations.size(); versionIndex++) { + // pProxyCommitData->logAdapter->setNextVersion( ??? ); << Ideally we would be telling the log adapter that the pushes in this commit will be in the version at which these state mutations were committed by another proxy, but at present we don't have that information here. So the disk queue may be unnecessarily conservative about popping. + + for (int transactionIndex = 0; transactionIndex < self->resolution[0].stateMutations[versionIndex].size() && !self->forceRecovery; transactionIndex++) { + bool committed = true; + for (int resolver = 0; resolver < self->resolution.size(); resolver++) + committed = committed && self->resolution[resolver].stateMutations[versionIndex][transactionIndex].committed; + if (committed) + applyMetadataMutations( + pProxyCommitData->dbgid, self->arena, + self->resolution[0].stateMutations[versionIndex][transactionIndex].mutations, + pProxyCommitData->txnStateStore, nullptr, &self->forceRecovery, pProxyCommitData->logSystem, 0, &pProxyCommitData->vecBackupKeys, &pProxyCommitData->keyInfo, &pProxyCommitData->cacheInfo, pProxyCommitData->firstProxy ? &pProxyCommitData->uid_applyMutationsData : nullptr, pProxyCommitData->commit, pProxyCommitData->cx, &pProxyCommitData->committedVersion, &pProxyCommitData->storageCache, &pProxyCommitData->tag_popped); + + if( self->resolution[0].stateMutations[versionIndex][transactionIndex].mutations.size() && self->firstStateMutations ) { + ASSERT(committed); + self->firstStateMutations = false; + self->forceRecovery = false; + } + //TraceEvent("MetadataTransaction", pProxyCommitData->dbgid).detail("Committed", committed).detail("Mutations", resolution[0].stateMutations[versionIndex][transactionIndex].second.size()).detail("R1Mutations", resolution.back().stateMutations[versionIndex][transactionIndex].second.size()); + } + //TraceEvent("MetadataBatch", pProxyCommitData->dbgid).detail("Transactions", resolution[0].stateMutations[versionIndex].size()); + + // These changes to txnStateStore will be committed by the other proxy, so we simply discard the commit message + auto fcm = pProxyCommitData->logAdapter->getCommitMessage(); + self->storeCommits.emplace_back(fcm, pProxyCommitData->txnStateStore->commit()); + //discardCommit( dbgid, fcm, txnStateStore->commit() ); + + if (initialState) { + //TraceEvent("ResyncLog", dbgid); + initialState = false; + self->forceRecovery = false; + pProxyCommitData->txnStateStore->resyncLog(); + + for (auto &p : self->storeCommits) { + ASSERT(!p.second.isReady()); + p.first.get().acknowledge.send(Void()); + ASSERT(p.second.isReady()); + } + self->storeCommits.clear(); + } + } + + // Determine which transactions actually committed (conservatively) by combining results from the resolvers + ASSERT(self->transactionResolverMap.size() == self->committed.size()); + // For each commitTransactionRef, it is only sent to resolvers specified in transactionResolverMap + // Thus, we use this nextTr to track the correct transaction index on each resolver. + self->nextTr.resize(self->resolution.size()); + for (int t = 0; t < trs.size(); t++) { + uint8_t commit = ConflictBatch::TransactionCommitted; + for (int r : self->transactionResolverMap[t]) + { + commit = std::min(self->resolution[r].committed[self->nextTr[r]++], commit); + } + self->committed[t] = commit; + } + for (int r = 0; r < self->resolution.size(); r++) + ASSERT(self->nextTr[r] == self->resolution[r].committed.size()); + + pProxyCommitData->logAdapter->setNextVersion(self->commitVersion); + + self->lockedKey = pProxyCommitData->txnStateStore->readValue(databaseLockedKey).get(); + self->locked = self->lockedKey.present() && self->lockedKey.get().size(); + + const auto& mustContainSystemKey = pProxyCommitData->txnStateStore->readValue(mustContainSystemMutationsKey).get(); + if (mustContainSystemKey.present() && mustContainSystemKey.get().size()) { + for (int t = 0; t < trs.size(); t++) { + if( self->committed[t] == ConflictBatch::TransactionCommitted ) { + bool foundSystem = false; + for(auto& m : trs[t].transaction.mutations) { + if( ( m.type == MutationRef::ClearRange ? m.param2 : m.param1 ) >= nonMetadataSystemKeys.end) { + foundSystem = true; + break; + } + } + if(!foundSystem) { + self->committed[t] = ConflictBatch::TransactionConflict; + } + } + } + } + + if(self->forceRecovery) { + wait( Future(Never()) ); + } + + // This first pass through committed transactions deals with "metadata" effects (modifications of txnStateStore, changes to storage servers' responsibilities) + int t; + for (t = 0; t < trs.size() && !self->forceRecovery; t++) + { + if (self->committed[t] == ConflictBatch::TransactionCommitted && (!self->locked || trs[t].isLockAware())) { + self->commitCount++; + applyMetadataMutations( + pProxyCommitData->dbgid, self->arena, + trs[t].transaction.mutations, + pProxyCommitData->txnStateStore, + &self->toCommit, + &self->forceRecovery, + pProxyCommitData->logSystem, + self->commitVersion + 1, + &pProxyCommitData->vecBackupKeys, + &pProxyCommitData->keyInfo, + &pProxyCommitData->cacheInfo, + pProxyCommitData->firstProxy ? &pProxyCommitData->uid_applyMutationsData : nullptr, + pProxyCommitData->commit, + pProxyCommitData->cx, + &pProxyCommitData->committedVersion, + &pProxyCommitData->storageCache, + &pProxyCommitData->tag_popped + ); + } + if(self->firstStateMutations) { + ASSERT(self->committed[t] == ConflictBatch::TransactionCommitted); + self->firstStateMutations = false; + self->forceRecovery = false; + } + } + if (self->forceRecovery) { + for (; t< trs.size(); t++) + self->committed[t] = ConflictBatch::TransactionConflict; + TraceEvent(SevWarn, "RestartingTxnSubsystem", pProxyCommitData->dbgid).detail("Stage", "AwaitCommit"); + } + + self->lockedKey = pProxyCommitData->txnStateStore->readValue(databaseLockedKey).get(); + self->lockedAfter = self->lockedKey.present() && self->lockedKey.get().size(); + + self->metadataVersionAfter = pProxyCommitData->txnStateStore->readValue(metadataVersionKey).get(); + + auto fcm = pProxyCommitData->logAdapter->getCommitMessage(); + self->storeCommits.emplace_back(fcm, pProxyCommitData->txnStateStore->commit()); + pProxyCommitData->version = self->commitVersion; + if (!pProxyCommitData->validState.isSet()) pProxyCommitData->validState.send(Void()); + ASSERT(self->commitVersion); + + if (!self->isMyFirstBatch && pProxyCommitData->txnStateStore->readValue( coordinatorsKey ).get().get() != self->oldCoordinators.get()) { + wait( brokenPromiseToNever( pProxyCommitData->master.changeCoordinators.getReply( ChangeCoordinatorsRequest( pProxyCommitData->txnStateStore->readValue( coordinatorsKey ).get().get() ) ) ) ); + ASSERT(false); // ChangeCoordinatorsRequest should always throw + } + + // This second pass through committed transactions assigns the actual mutations to the appropriate storage servers' tags + for (; self->transactionNum < trs.size(); self->transactionNum++) { + if (self->committed[self->transactionNum] == ConflictBatch::TransactionCommitted && (!self->locked || trs[self->transactionNum].isLockAware())) { + state int mutationNum = 0; + state VectorRef* pMutations = &trs[self->transactionNum].transaction.mutations; + for (; mutationNum < pMutations->size(); mutationNum++) { + if(self->yieldBytes > SERVER_KNOBS->DESIRED_TOTAL_BYTES) { + self->yieldBytes = 0; + if(g_network->check_yield(TaskPriority::ProxyCommitYield1)) { + self->computeDuration += g_network->timer() - self->computeStart; + wait(delay(0, TaskPriority::ProxyCommitYield1)); + self->computeStart = g_network->timer(); + } + } + + auto& m = (*pMutations)[mutationNum]; + self->mutationCount++; + self->mutationBytes += m.expectedSize(); + self->yieldBytes += m.expectedSize(); + // Determine the set of tags (responsible storage servers) for the mutation, splitting it + // if necessary. Serialize (splits of) the mutation into the message buffer and add the tags. + + if (isSingleKeyMutation((MutationRef::Type) m.type)) { + auto& tags = pProxyCommitData->tagsForKey(m.param1); + + if(pProxyCommitData->singleKeyMutationEvent->enabled) { + KeyRangeRef shard = pProxyCommitData->keyInfo.rangeContaining(m.param1).range(); + pProxyCommitData->singleKeyMutationEvent->tag1 = (int64_t)tags[0].id; + pProxyCommitData->singleKeyMutationEvent->tag2 = (int64_t)tags[1].id; + pProxyCommitData->singleKeyMutationEvent->tag3 = (int64_t)tags[2].id; + pProxyCommitData->singleKeyMutationEvent->shardBegin = shard.begin; + pProxyCommitData->singleKeyMutationEvent->shardEnd = shard.end; + pProxyCommitData->singleKeyMutationEvent->log(); + } + + DEBUG_MUTATION("ProxyCommit", self->commitVersion, m).detail("Dbgid", pProxyCommitData->dbgid).detail("To", tags).detail("Mutation", m); + + self->toCommit.addTags(tags); + if(pProxyCommitData->cacheInfo[m.param1]) { + self->toCommit.addTag(cacheTag); + } + self->toCommit.addTypedMessage(m); + } + else if (m.type == MutationRef::ClearRange) { + KeyRangeRef clearRange(KeyRangeRef(m.param1, m.param2)); + auto ranges = pProxyCommitData->keyInfo.intersectingRanges(clearRange); + auto firstRange = ranges.begin(); + ++firstRange; + if (firstRange == ranges.end()) { + // Fast path + DEBUG_MUTATION("ProxyCommit", self->commitVersion, m).detail("Dbgid", pProxyCommitData->dbgid).detail("To", ranges.begin().value().tags).detail("Mutation", m); + + ranges.begin().value().populateTags(); + self->toCommit.addTags(ranges.begin().value().tags); + } + else { + TEST(true); //A clear range extends past a shard boundary + std::set allSources; + for (auto r : ranges) { + r.value().populateTags(); + allSources.insert(r.value().tags.begin(), r.value().tags.end()); + } + DEBUG_MUTATION("ProxyCommit", self->commitVersion, m).detail("Dbgid", pProxyCommitData->dbgid).detail("To", allSources).detail("Mutation", m); + + self->toCommit.addTags(allSources); + } + if(pProxyCommitData->needsCacheTag(clearRange)) { + self->toCommit.addTag(cacheTag); + } + self->toCommit.addTypedMessage(m); + } else { + UNREACHABLE(); + } + + // Check on backing up key, if backup ranges are defined and a normal key + if (pProxyCommitData->vecBackupKeys.size() > 1 && (normalKeys.contains(m.param1) || m.param1 == metadataVersionKey)) { + if (m.type != MutationRef::Type::ClearRange) { + // Add the mutation to the relevant backup tag + for (auto backupName : pProxyCommitData->vecBackupKeys[m.param1]) { + self->logRangeMutations[backupName].push_back_deep(self->logRangeMutationsArena, m); + } + } + else { + KeyRangeRef mutationRange(m.param1, m.param2); + KeyRangeRef intersectionRange; + + // Identify and add the intersecting ranges of the mutation to the array of mutations to serialize + for (auto backupRange : pProxyCommitData->vecBackupKeys.intersectingRanges(mutationRange)) + { + // Get the backup sub range + const auto& backupSubrange = backupRange.range(); + + // Determine the intersecting range + intersectionRange = mutationRange & backupSubrange; + + // Create the custom mutation for the specific backup tag + MutationRef backupMutation(MutationRef::Type::ClearRange, intersectionRange.begin, intersectionRange.end); + + // Add the mutation to the relevant backup tag + for (auto backupName : backupRange.value()) { + self->logRangeMutations[backupName].push_back_deep(self->logRangeMutationsArena, backupMutation); + } + } + } + } + } + } + } + + // Serialize and backup the mutations as a single mutation + if ((pProxyCommitData->vecBackupKeys.size() > 1) && self->logRangeMutations.size()) { + wait( addBackupMutations(pProxyCommitData, &self->logRangeMutations, &self->toCommit, self->commitVersion, &self->computeDuration, &self->computeStart) ); + } + + pProxyCommitData->stats.mutations += self->mutationCount; + pProxyCommitData->stats.mutationBytes += self->mutationBytes; + + // Storage servers mustn't make durable versions which are not fully committed (because then they are impossible to roll back) + // We prevent this by limiting the number of versions which are semi-committed but not fully committed to be less than the MVCC window + if (pProxyCommitData->committedVersion.get() < self->commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS) { + self->computeDuration += g_network->timer() - self->computeStart; + state Span waitVersionSpan; + while (pProxyCommitData->committedVersion.get() < self->commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS) { + // This should be *extremely* rare in the real world, but knob buggification should make it happen in simulation + TEST(true); // Semi-committed pipeline limited by MVCC window + //TraceEvent("ProxyWaitingForCommitted", pProxyCommitData->dbgid).detail("CommittedVersion", pProxyCommitData->committedVersion.get()).detail("NeedToCommit", commitVersion); + waitVersionSpan = Span(deterministicRandom()->randomUniqueID(), "MP:overMaxReadTransactionLifeVersions"_loc, {self->span.context}); + choose{ + when(wait(pProxyCommitData->committedVersion.whenAtLeast(self->commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS))) { + wait(yield()); + break; + } + when(GetReadVersionReply v = wait(pProxyCommitData->getConsistentReadVersion.getReply( + GetReadVersionRequest(waitVersionSpan.context, 0, TransactionPriority::IMMEDIATE, + GetReadVersionRequest::FLAG_CAUSAL_READ_RISKY)))) { + if(v.version > pProxyCommitData->committedVersion.get()) { + pProxyCommitData->locked = v.locked; + pProxyCommitData->metadataVersion = v.metadataVersion; + pProxyCommitData->committedVersion.set(v.version); + } + + if (pProxyCommitData->committedVersion.get() < self->commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS) + wait(delay(SERVER_KNOBS->PROXY_SPIN_DELAY)); + } + } + } + waitVersionSpan = Span{}; + self->computeStart = g_network->timer(); + } + + self->msg = self->storeCommits.back().first.get(); + + if (self->debugID.present()) + g_traceBatch.addEvent("CommitDebug", self->debugID.get().first(), "MasterProxyServer.commitBatch.AfterStoreCommits"); + + // txnState (transaction subsystem state) tag: message extracted from log adapter + bool firstMessage = true; + for(auto m : self->msg.messages) { + if(firstMessage) { + self->toCommit.addTxsTag(); + } + self->toCommit.addMessage(StringRef(m.begin(), m.size()), !firstMessage); + firstMessage = false; + } + + if ( self->prevVersion && self->commitVersion - self->prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT/2 ) + debug_advanceMaxCommittedVersion( UID(), self->commitVersion ); //< Is this valid? + + //TraceEvent("ProxyPush", pProxyCommitData->dbgid).detail("PrevVersion", prevVersion).detail("Version", commitVersion) + // .detail("TransactionsSubmitted", trs.size()).detail("TransactionsCommitted", commitCount).detail("TxsPopTo", msg.popTo); + + if ( self->prevVersion && self->commitVersion - self->prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT/2 ) + debug_advanceMaxCommittedVersion(UID(), self->commitVersion); + + self->commitStartTime = now(); + pProxyCommitData->lastStartCommit = self->commitStartTime; + self->loggingComplete = pProxyCommitData->logSystem->push( self->prevVersion, self->commitVersion, pProxyCommitData->committedVersion.get(), pProxyCommitData->minKnownCommittedVersion, self->toCommit, self->debugID ); + + if (!self->forceRecovery) { + ASSERT(pProxyCommitData->latestLocalCommitBatchLogging.get() == self->localBatchNumber-1); + pProxyCommitData->latestLocalCommitBatchLogging.set(self->localBatchNumber); + } + + self->computeDuration += g_network->timer() - self->computeStart; + if(self->computeDuration > SERVER_KNOBS->MIN_PROXY_COMPUTE && self->batchOperations > 0) { + double computePerOperation = self->computeDuration / self->batchOperations; + if(computePerOperation <= pProxyCommitData->commitComputePerOperation[self->latencyBucket]) { + pProxyCommitData->commitComputePerOperation[self->latencyBucket] = computePerOperation; + } else { + pProxyCommitData->commitComputePerOperation[self->latencyBucket] = SERVER_KNOBS->PROXY_COMPUTE_GROWTH_RATE*computePerOperation + ((1.0-SERVER_KNOBS->PROXY_COMPUTE_GROWTH_RATE)*pProxyCommitData->commitComputePerOperation[self->latencyBucket]); + } + } + + return Void(); +} + } // namespace CommitBatch // Commit one batch of transactions trs @@ -1067,384 +1486,13 @@ ACTOR Future commitBatch( wait(CommitBatch::getResolution(&context)); ////// Phase 3: Post-resolution processing (CPU bound except for very rare situations; ordered; currently atomic but doesn't need to be) - TEST(context.pProxyCommitData->latestLocalCommitBatchLogging.get() < context.localBatchNumber - 1); // Queuing post-resolution commit processing - wait(context.pProxyCommitData->latestLocalCommitBatchLogging.whenAtLeast(context.localBatchNumber - 1)); - wait(yield(TaskPriority::ProxyCommitYield1)); - - state double computeStart = g_network->timer(); - state double computeDuration = 0; - self->stats.txnCommitResolved += context.trs.size(); - - if (context.debugID.present()) { - g_traceBatch.addEvent( - "CommitDebug", context.debugID.get().first(), - "MasterProxyServer.commitBatch.ProcessingMutations" - ); - } - - state Arena arena; - state bool isMyFirstBatch = !context.pProxyCommitData->version; - state Optional oldCoordinators = context.pProxyCommitData->txnStateStore->readValue(coordinatorsKey).get(); - - //TraceEvent("ResolutionResult", context.pProxyCommitData->dbgid).detail("Sequence", sequence).detail("Version", commitVersion).detail("StateMutationProxies", resolution[0].stateMutations.size()).detail("WaitForResolution", now()-t1).detail("R0Committed", resolution[0].committed.size()) - // .detail("Transactions", trs.size()); - - for(int r=1; r < context.resolution.size(); r++) { - ASSERT( context.resolution[r].stateMutations.size() == context.resolution[0].stateMutations.size() ); - for(int s = 0; s < context.resolution[r].stateMutations.size(); s++) - ASSERT( context.resolution[r].stateMutations[s].size() == context.resolution[0].stateMutations[s].size() ); - } - - // Compute and apply "metadata" effects of each other proxy's most recent batch - bool initialState = isMyFirstBatch; - state bool firstStateMutations = isMyFirstBatch; - state vector< std::pair, Future> > storeCommits; - for (int versionIndex = 0; versionIndex < context.resolution[0].stateMutations.size(); versionIndex++) { - // context.pProxyCommitData->logAdapter->setNextVersion( ??? ); << Ideally we would be telling the log adapter that the pushes in this commit will be in the version at which these state mutations were committed by another proxy, but at present we don't have that information here. So the disk queue may be unnecessarily conservative about popping. - - for (int transactionIndex = 0; transactionIndex < context.resolution[0].stateMutations[versionIndex].size() && !context.forceRecovery; transactionIndex++) { - bool committed = true; - for (int resolver = 0; resolver < context.resolution.size(); resolver++) - committed = committed && context.resolution[resolver].stateMutations[versionIndex][transactionIndex].committed; - if (committed) - applyMetadataMutations( - context.pProxyCommitData->dbgid, arena, - context.resolution[0].stateMutations[versionIndex][transactionIndex].mutations, - context.pProxyCommitData->txnStateStore, nullptr, &context.forceRecovery, context.pProxyCommitData->logSystem, 0, &context.pProxyCommitData->vecBackupKeys, &context.pProxyCommitData->keyInfo, &context.pProxyCommitData->cacheInfo, context.pProxyCommitData->firstProxy ? &context.pProxyCommitData->uid_applyMutationsData : nullptr, context.pProxyCommitData->commit, context.pProxyCommitData->cx, &context.pProxyCommitData->committedVersion, &context.pProxyCommitData->storageCache, &context.pProxyCommitData->tag_popped); - - if( context.resolution[0].stateMutations[versionIndex][transactionIndex].mutations.size() && firstStateMutations ) { - ASSERT(committed); - firstStateMutations = false; - context.forceRecovery = false; - } - //TraceEvent("MetadataTransaction", context.pProxyCommitData->dbgid).detail("Committed", committed).detail("Mutations", resolution[0].stateMutations[versionIndex][transactionIndex].second.size()).detail("R1Mutations", resolution.back().stateMutations[versionIndex][transactionIndex].second.size()); - } - //TraceEvent("MetadataBatch", context.pProxyCommitData->dbgid).detail("Transactions", resolution[0].stateMutations[versionIndex].size()); - - // These changes to txnStateStore will be committed by the other proxy, so we simply discard the commit message - auto fcm = context.pProxyCommitData->logAdapter->getCommitMessage(); - storeCommits.emplace_back(fcm, context.pProxyCommitData->txnStateStore->commit()); - //discardCommit( dbgid, fcm, txnStateStore->commit() ); - - if (initialState) { - //TraceEvent("ResyncLog", dbgid); - initialState = false; - context.forceRecovery = false; - context.pProxyCommitData->txnStateStore->resyncLog(); - - for (auto &p : storeCommits) { - ASSERT(!p.second.isReady()); - p.first.get().acknowledge.send(Void()); - ASSERT(p.second.isReady()); - } - storeCommits.clear(); - } - } - - // Determine which transactions actually committed (conservatively) by combining results from the resolvers - state vector committed(context.trs.size()); - ASSERT(context.transactionResolverMap.size() == committed.size()); - // For each commitTransactionRef, it is only sent to resolvers specified in transactionResolverMap - // Thus, we use this nextTr to track the correct transaction index on each resolver. - state vector nextTr(context.resolution.size()); - for (int t = 0; t < context.trs.size(); t++) { - uint8_t commit = ConflictBatch::TransactionCommitted; - for (int r : context.transactionResolverMap[t]) - { - commit = std::min(context.resolution[r].committed[nextTr[r]++], commit); - } - committed[t] = commit; - } - for (int r = 0; r < context.resolution.size(); r++) - ASSERT(nextTr[r] == context.resolution[r].committed.size()); - - context.pProxyCommitData->logAdapter->setNextVersion(context.commitVersion); - - state Optional lockedKey = context.pProxyCommitData->txnStateStore->readValue(databaseLockedKey).get(); - state bool locked = lockedKey.present() && lockedKey.get().size(); - - state Optional mustContainSystemKey = context.pProxyCommitData->txnStateStore->readValue(mustContainSystemMutationsKey).get(); - if(mustContainSystemKey.present() && mustContainSystemKey.get().size()) { - for (int t = 0; t < context.trs.size(); t++) { - if( committed[t] == ConflictBatch::TransactionCommitted ) { - bool foundSystem = false; - for(auto& m : context.trs[t].transaction.mutations) { - if( ( m.type == MutationRef::ClearRange ? m.param2 : m.param1 ) >= nonMetadataSystemKeys.end) { - foundSystem = true; - break; - } - } - if(!foundSystem) { - committed[t] = ConflictBatch::TransactionConflict; - } - } - } - } - - if(context.forceRecovery) { - wait( Future(Never()) ); - } - - // This first pass through committed transactions deals with "metadata" effects (modifications of txnStateStore, changes to storage servers' responsibilities) - int t; - state int commitCount = 0; - for (t = 0; t < context.trs.size() && !context.forceRecovery; t++) - { - if (committed[t] == ConflictBatch::TransactionCommitted && (!locked || context.trs[t].isLockAware())) { - commitCount++; - applyMetadataMutations( - context.pProxyCommitData->dbgid, arena, - context.trs[t].transaction.mutations, - context.pProxyCommitData->txnStateStore, - &context.toCommit, - &context.forceRecovery, - context.pProxyCommitData->logSystem, - context.commitVersion + 1, - &context.pProxyCommitData->vecBackupKeys, - &context.pProxyCommitData->keyInfo, - &context.pProxyCommitData->cacheInfo, - context.pProxyCommitData->firstProxy ? &context.pProxyCommitData->uid_applyMutationsData : nullptr, - context.pProxyCommitData->commit, - context.pProxyCommitData->cx, - &context.pProxyCommitData->committedVersion, - &context.pProxyCommitData->storageCache, - &context.pProxyCommitData->tag_popped - ); - } - if(firstStateMutations) { - ASSERT(committed[t] == ConflictBatch::TransactionCommitted); - firstStateMutations = false; - context.forceRecovery = false; - } - } - if (context.forceRecovery) { - for (; t< context.trs.size(); t++) - committed[t] = ConflictBatch::TransactionConflict; - TraceEvent(SevWarn, "RestartingTxnSubsystem", context.pProxyCommitData->dbgid).detail("Stage", "AwaitCommit"); - } - - lockedKey = context.pProxyCommitData->txnStateStore->readValue(databaseLockedKey).get(); - state bool lockedAfter = lockedKey.present() && lockedKey.get().size(); - - state Optional metadataVersionAfter = context.pProxyCommitData->txnStateStore->readValue(metadataVersionKey).get(); - - auto fcm = context.pProxyCommitData->logAdapter->getCommitMessage(); - storeCommits.emplace_back(fcm, context.pProxyCommitData->txnStateStore->commit()); - context.pProxyCommitData->version = context.commitVersion; - if (!context.pProxyCommitData->validState.isSet()) context.pProxyCommitData->validState.send(Void()); - ASSERT(context.commitVersion); - - if (!isMyFirstBatch && context.pProxyCommitData->txnStateStore->readValue( coordinatorsKey ).get().get() != oldCoordinators.get()) { - wait( brokenPromiseToNever( context.pProxyCommitData->master.changeCoordinators.getReply( ChangeCoordinatorsRequest( context.pProxyCommitData->txnStateStore->readValue( coordinatorsKey ).get().get() ) ) ) ); - ASSERT(false); // ChangeCoordinatorsRequest should always throw - } - - // This second pass through committed transactions assigns the actual mutations to the appropriate storage servers' tags - state int mutationCount = 0; - state int mutationBytes = 0; - - state std::map logRangeMutations; - state Arena logRangeMutationsArena; - state int transactionNum = 0; - state int yieldBytes = 0; - - for (; transactionNum < context.trs.size(); transactionNum++) { - if (committed[transactionNum] == ConflictBatch::TransactionCommitted && (!locked || context.trs[transactionNum].isLockAware())) { - state int mutationNum = 0; - state VectorRef* pMutations = &context.trs[transactionNum].transaction.mutations; - for (; mutationNum < pMutations->size(); mutationNum++) { - if(yieldBytes > SERVER_KNOBS->DESIRED_TOTAL_BYTES) { - yieldBytes = 0; - if(g_network->check_yield(TaskPriority::ProxyCommitYield1)) { - computeDuration += g_network->timer() - computeStart; - wait(delay(0, TaskPriority::ProxyCommitYield1)); - computeStart = g_network->timer(); - } - } - - auto& m = (*pMutations)[mutationNum]; - mutationCount++; - mutationBytes += m.expectedSize(); - yieldBytes += m.expectedSize(); - // Determine the set of tags (responsible storage servers) for the mutation, splitting it - // if necessary. Serialize (splits of) the mutation into the message buffer and add the tags. - - if (isSingleKeyMutation((MutationRef::Type) m.type)) { - auto& tags = context.pProxyCommitData->tagsForKey(m.param1); - - if(context.pProxyCommitData->singleKeyMutationEvent->enabled) { - KeyRangeRef shard = context.pProxyCommitData->keyInfo.rangeContaining(m.param1).range(); - context.pProxyCommitData->singleKeyMutationEvent->tag1 = (int64_t)tags[0].id; - context.pProxyCommitData->singleKeyMutationEvent->tag2 = (int64_t)tags[1].id; - context.pProxyCommitData->singleKeyMutationEvent->tag3 = (int64_t)tags[2].id; - context.pProxyCommitData->singleKeyMutationEvent->shardBegin = shard.begin; - context.pProxyCommitData->singleKeyMutationEvent->shardEnd = shard.end; - context.pProxyCommitData->singleKeyMutationEvent->log(); - } - - DEBUG_MUTATION("ProxyCommit", context.commitVersion, m).detail("Dbgid", context.pProxyCommitData->dbgid).detail("To", tags).detail("Mutation", m); - - context.toCommit.addTags(tags); - if(context.pProxyCommitData->cacheInfo[m.param1]) { - context.toCommit.addTag(cacheTag); - } - context.toCommit.addTypedMessage(m); - } - else if (m.type == MutationRef::ClearRange) { - KeyRangeRef clearRange(KeyRangeRef(m.param1, m.param2)); - auto ranges = context.pProxyCommitData->keyInfo.intersectingRanges(clearRange); - auto firstRange = ranges.begin(); - ++firstRange; - if (firstRange == ranges.end()) { - // Fast path - DEBUG_MUTATION("ProxyCommit", context.commitVersion, m).detail("Dbgid", context.pProxyCommitData->dbgid).detail("To", ranges.begin().value().tags).detail("Mutation", m); - - ranges.begin().value().populateTags(); - context.toCommit.addTags(ranges.begin().value().tags); - } - else { - TEST(true); //A clear range extends past a shard boundary - std::set allSources; - for (auto r : ranges) { - r.value().populateTags(); - allSources.insert(r.value().tags.begin(), r.value().tags.end()); - } - DEBUG_MUTATION("ProxyCommit", context.commitVersion, m).detail("Dbgid", context.pProxyCommitData->dbgid).detail("To", allSources).detail("Mutation", m); - - context.toCommit.addTags(allSources); - } - if(context.pProxyCommitData->needsCacheTag(clearRange)) { - context.toCommit.addTag(cacheTag); - } - context.toCommit.addTypedMessage(m); - } else { - UNREACHABLE(); - } - - // Check on backing up key, if backup ranges are defined and a normal key - if (context.pProxyCommitData->vecBackupKeys.size() > 1 && (normalKeys.contains(m.param1) || m.param1 == metadataVersionKey)) { - if (m.type != MutationRef::Type::ClearRange) { - // Add the mutation to the relevant backup tag - for (auto backupName : context.pProxyCommitData->vecBackupKeys[m.param1]) { - logRangeMutations[backupName].push_back_deep(logRangeMutationsArena, m); - } - } - else { - KeyRangeRef mutationRange(m.param1, m.param2); - KeyRangeRef intersectionRange; - - // Identify and add the intersecting ranges of the mutation to the array of mutations to serialize - for (auto backupRange : context.pProxyCommitData->vecBackupKeys.intersectingRanges(mutationRange)) - { - // Get the backup sub range - const auto& backupSubrange = backupRange.range(); - - // Determine the intersecting range - intersectionRange = mutationRange & backupSubrange; - - // Create the custom mutation for the specific backup tag - MutationRef backupMutation(MutationRef::Type::ClearRange, intersectionRange.begin, intersectionRange.end); - - // Add the mutation to the relevant backup tag - for (auto backupName : backupRange.value()) { - logRangeMutations[backupName].push_back_deep(logRangeMutationsArena, backupMutation); - } - } - } - } - } - } - } - - // Serialize and backup the mutations as a single mutation - if ((context.pProxyCommitData->vecBackupKeys.size() > 1) && logRangeMutations.size()) { - wait( addBackupMutations(context.pProxyCommitData, &logRangeMutations, &context.toCommit, context.commitVersion, &computeDuration, &computeStart) ); - } - - context.pProxyCommitData->stats.mutations += mutationCount; - context.pProxyCommitData->stats.mutationBytes += mutationBytes; - - // Storage servers mustn't make durable versions which are not fully committed (because then they are impossible to roll back) - // We prevent this by limiting the number of versions which are semi-committed but not fully committed to be less than the MVCC window - if(context.pProxyCommitData->committedVersion.get() < context.commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS) { - computeDuration += g_network->timer() - computeStart; - state Span waitVersionSpan; - while (context.pProxyCommitData->committedVersion.get() < context.commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS) { - // This should be *extremely* rare in the real world, but knob buggification should make it happen in simulation - TEST(true); // Semi-committed pipeline limited by MVCC window - //TraceEvent("ProxyWaitingForCommitted", context.pProxyCommitData->dbgid).detail("CommittedVersion", context.pProxyCommitData->committedVersion.get()).detail("NeedToCommit", commitVersion); - waitVersionSpan = Span(deterministicRandom()->randomUniqueID(), "MP:overMaxReadTransactionLifeVersions"_loc, {context.span.context}); - choose{ - when(wait(context.pProxyCommitData->committedVersion.whenAtLeast(context.commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS))) { - wait(yield()); - break; - } - when(GetReadVersionReply v = wait(context.pProxyCommitData->getConsistentReadVersion.getReply( - GetReadVersionRequest(waitVersionSpan.context, 0, TransactionPriority::IMMEDIATE, - GetReadVersionRequest::FLAG_CAUSAL_READ_RISKY)))) { - if(v.version > context.pProxyCommitData->committedVersion.get()) { - context.pProxyCommitData->locked = v.locked; - context.pProxyCommitData->metadataVersion = v.metadataVersion; - context.pProxyCommitData->committedVersion.set(v.version); - } - - if (context.pProxyCommitData->committedVersion.get() < context.commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS) - wait(delay(SERVER_KNOBS->PROXY_SPIN_DELAY)); - } - } - } - waitVersionSpan = Span{}; - computeStart = g_network->timer(); - } - - state LogSystemDiskQueueAdapter::CommitMessage msg = storeCommits.back().first.get(); - - if (context.debugID.present()) - g_traceBatch.addEvent("CommitDebug", context.debugID.get().first(), "MasterProxyServer.commitBatch.AfterStoreCommits"); - - // txnState (transaction subsystem state) tag: message extracted from log adapter - bool firstMessage = true; - for(auto m : msg.messages) { - if(firstMessage) { - context.toCommit.addTxsTag(); - } - context.toCommit.addMessage(StringRef(m.begin(), m.size()), !firstMessage); - firstMessage = false; - } - - if ( context.prevVersion && context.commitVersion - context.prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT/2 ) - debug_advanceMaxCommittedVersion( UID(), context.commitVersion ); //< Is this valid? - - //TraceEvent("ProxyPush", context.pProxyCommitData->dbgid).detail("PrevVersion", prevVersion).detail("Version", commitVersion) - // .detail("TransactionsSubmitted", trs.size()).detail("TransactionsCommitted", commitCount).detail("TxsPopTo", msg.popTo); - - if ( context.prevVersion && context.commitVersion - context.prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT/2 ) - debug_advanceMaxCommittedVersion(UID(), context.commitVersion); - - state double commitStartTime = now(); - context.pProxyCommitData->lastStartCommit = commitStartTime; - Future loggingComplete = context.pProxyCommitData->logSystem->push( context.prevVersion, context.commitVersion, context.pProxyCommitData->committedVersion.get(), context.pProxyCommitData->minKnownCommittedVersion, context.toCommit, context.debugID ); - - if (!context.forceRecovery) { - ASSERT(context.pProxyCommitData->latestLocalCommitBatchLogging.get() == context.localBatchNumber-1); - context.pProxyCommitData->latestLocalCommitBatchLogging.set(context.localBatchNumber); - } - - computeDuration += g_network->timer() - computeStart; - if(computeDuration > SERVER_KNOBS->MIN_PROXY_COMPUTE && context.batchOperations > 0) { - double computePerOperation = computeDuration / context.batchOperations; - if(computePerOperation <= context.pProxyCommitData->commitComputePerOperation[context.latencyBucket]) { - context.pProxyCommitData->commitComputePerOperation[context.latencyBucket] = computePerOperation; - } else { - context.pProxyCommitData->commitComputePerOperation[context.latencyBucket] = SERVER_KNOBS->PROXY_COMPUTE_GROWTH_RATE*computePerOperation + ((1.0-SERVER_KNOBS->PROXY_COMPUTE_GROWTH_RATE)*context.pProxyCommitData->commitComputePerOperation[context.latencyBucket]); - } - } + wait(CommitBatch::postResolution(&context)); /////// Phase 4: Logging (network bound; pipelined up to MAX_READ_TRANSACTION_LIFE_VERSIONS (limited by loop above)) try { choose { - when(Version ver = wait(loggingComplete)) { + when(Version ver = wait(context.loggingComplete)) { context.pProxyCommitData->minKnownCommittedVersion = std::max(context.pProxyCommitData->minKnownCommittedVersion, ver); } when(wait(context.pProxyCommitData->committedVersion.whenAtLeast( context.commitVersion+1 ))) {} @@ -1456,20 +1504,20 @@ ACTOR Future commitBatch( throw; } - context.pProxyCommitData->lastCommitLatency = now()-commitStartTime; - context.pProxyCommitData->lastCommitTime = std::max(context.pProxyCommitData->lastCommitTime.get(), commitStartTime); + context.pProxyCommitData->lastCommitLatency = now() - context.commitStartTime; + context.pProxyCommitData->lastCommitTime = std::max(context.pProxyCommitData->lastCommitTime.get(), context.commitStartTime); wait(yield(TaskPriority::ProxyCommitYield2)); - if( context.pProxyCommitData->popRemoteTxs && msg.popTo > ( context.pProxyCommitData->txsPopVersions.size() ? context.pProxyCommitData->txsPopVersions.back().second : context.pProxyCommitData->lastTxsPop ) ) { + if( context.pProxyCommitData->popRemoteTxs && context.msg.popTo > ( context.pProxyCommitData->txsPopVersions.size() ? context.pProxyCommitData->txsPopVersions.back().second : context.pProxyCommitData->lastTxsPop ) ) { if(context.pProxyCommitData->txsPopVersions.size() >= SERVER_KNOBS->MAX_TXS_POP_VERSION_HISTORY) { TraceEvent(SevWarnAlways, "DiscardingTxsPopHistory").suppressFor(1.0); context.pProxyCommitData->txsPopVersions.pop_front(); } - context.pProxyCommitData->txsPopVersions.emplace_back(context.commitVersion, msg.popTo); + context.pProxyCommitData->txsPopVersions.emplace_back(context.commitVersion, context.msg.popTo); } - context.pProxyCommitData->logSystem->popTxs(msg.popTo); + context.pProxyCommitData->logSystem->popTxs(context.msg.popTo); /////// Phase 5: Replies (CPU bound; no particular order required, though ordered execution would be best for latency) if ( context.prevVersion && context.commitVersion - context.prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT/2 ) @@ -1479,7 +1527,7 @@ ACTOR Future commitBatch( if (context.debugID.present()) g_traceBatch.addEvent("CommitDebug", context.debugID.get().first(), "MasterProxyServer.commitBatch.AfterLogPush"); - for (auto &p : storeCommits) { + for (auto &p : context.storeCommits) { ASSERT(!p.second.isReady()); p.first.get().acknowledge.send(Void()); ASSERT(p.second.isReady()); @@ -1491,11 +1539,11 @@ ACTOR Future commitBatch( // version that the master is not aware of, and next GRV request may get a version less than self->committedVersion. TEST(context.pProxyCommitData->committedVersion.get() > context.commitVersion); // A later version was reported committed first if (SERVER_KNOBS->ASK_READ_VERSION_FROM_MASTER && context.commitVersion > context.pProxyCommitData->committedVersion.get()) { - wait(context.pProxyCommitData->master.reportLiveCommittedVersion.getReply(ReportRawCommittedVersionRequest(context.commitVersion, lockedAfter, metadataVersionAfter), TaskPriority::ProxyMasterVersionReply)); + wait(context.pProxyCommitData->master.reportLiveCommittedVersion.getReply(ReportRawCommittedVersionRequest(context.commitVersion, context.lockedAfter, context.metadataVersionAfter), TaskPriority::ProxyMasterVersionReply)); } if( context.commitVersion > context.pProxyCommitData->committedVersion.get() ) { - context.pProxyCommitData->locked = lockedAfter; - context.pProxyCommitData->metadataVersion = metadataVersionAfter; + context.pProxyCommitData->locked = context.lockedAfter; + context.pProxyCommitData->metadataVersion = context.metadataVersionAfter; context.pProxyCommitData->committedVersion.set(context.commitVersion); } @@ -1507,11 +1555,11 @@ ACTOR Future commitBatch( // Send replies to clients double endTime = g_network->timer(); // Reset all to zero, used to track the correct index of each commitTransacitonRef on each resolver - std::fill(nextTr.begin(), nextTr.end(), 0); + std::fill(context.nextTr.begin(), context.nextTr.end(), 0); for (int t = 0; t < context.trs.size(); t++) { - if (committed[t] == ConflictBatch::TransactionCommitted && (!locked || context.trs[t].isLockAware())) { + if (context.committed[t] == ConflictBatch::TransactionCommitted && (!context.locked || context.trs[t].isLockAware())) { ASSERT_WE_THINK(context.commitVersion != invalidVersion); - context.trs[t].reply.send(CommitID(context.commitVersion, t, metadataVersionAfter)); + context.trs[t].reply.send(CommitID(context.commitVersion, t, context.metadataVersionAfter)); // aggregate commit cost estimation if committed ASSERT(context.trs[t].commitCostEstimation.present() == context.trs[t].tagSet.present()); @@ -1522,7 +1570,7 @@ ACTOR Future commitBatch( } } } - else if (committed[t] == ConflictBatch::TransactionTooOld) { + else if (context.committed[t] == ConflictBatch::TransactionTooOld) { context.trs[t].reply.sendError(transaction_too_old()); } else { @@ -1533,7 +1581,7 @@ ACTOR Future commitBatch( for (int resolverInd : context.transactionResolverMap[t]) { auto const& cKRs = context.resolution[resolverInd] - .conflictingKeyRangeMap[nextTr[resolverInd]]; // nextTr[resolverInd] -> index of this trs[t] + .conflictingKeyRangeMap[context.nextTr[resolverInd]]; // nextTr[resolverInd] -> index of this trs[t] // on the resolver for (auto const& rCRIndex : cKRs) // read_conflict_range can change when sent to resolvers, mapping the index from resolver-side @@ -1551,7 +1599,7 @@ ACTOR Future commitBatch( } // Update corresponding transaction indices on each resolver - for (int resolverInd : context.transactionResolverMap[t]) nextTr[resolverInd]++; + for (int resolverInd : context.transactionResolverMap[t]) context.nextTr[resolverInd]++; // TODO: filter if pipelined with large commit double duration = endTime - context.trs[t].requestTime(); @@ -1564,8 +1612,8 @@ ACTOR Future commitBatch( ++context.pProxyCommitData->stats.commitBatchOut; context.pProxyCommitData->stats.txnCommitOut += context.trs.size(); - context.pProxyCommitData->stats.txnConflicts += context.trs.size() - commitCount; - context.pProxyCommitData->stats.txnCommitOutSuccess += commitCount; + context.pProxyCommitData->stats.txnConflicts += context.trs.size() - context.commitCount; + context.pProxyCommitData->stats.txnCommitOutSuccess += context.commitCount; if(now() - context.pProxyCommitData->lastCoalesceTime > SERVER_KNOBS->RESOLVER_COALESCE_TIME) { context.pProxyCommitData->lastCoalesceTime = now(); From 376aff079c62381e2c35521a1948d432baf7da3b Mon Sep 17 00:00:00 2001 From: Xiaoge Su Date: Sat, 25 Jul 2020 18:11:25 -0700 Subject: [PATCH 3/8] Extract Step 4 and 5 from commitBatch --- fdbserver/MasterProxyServer.actor.cpp | 324 ++++++++++++++------------ 1 file changed, 174 insertions(+), 150 deletions(-) diff --git a/fdbserver/MasterProxyServer.actor.cpp b/fdbserver/MasterProxyServer.actor.cpp index 8cf9664e7c..841801d2e3 100644 --- a/fdbserver/MasterProxyServer.actor.cpp +++ b/fdbserver/MasterProxyServer.actor.cpp @@ -1463,6 +1463,178 @@ ACTOR Future postResolution(Context* self) { return Void(); } +ACTOR Future transactionLogging(Context* self) { + state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData; + + try { + choose { + when(Version ver = wait(self->loggingComplete)) { + pProxyCommitData->minKnownCommittedVersion = std::max(pProxyCommitData->minKnownCommittedVersion, ver); + } + when(wait(pProxyCommitData->committedVersion.whenAtLeast( self->commitVersion + 1 ))) {} + } + } catch(Error &e) { + if(e.code() == error_code_broken_promise) { + throw master_tlog_failed(); + } + throw; + } + + pProxyCommitData->lastCommitLatency = now() - self->commitStartTime; + pProxyCommitData->lastCommitTime = std::max(pProxyCommitData->lastCommitTime.get(), self->commitStartTime); + + wait(yield(TaskPriority::ProxyCommitYield2)); + + if( pProxyCommitData->popRemoteTxs && self->msg.popTo > ( pProxyCommitData->txsPopVersions.size() ? pProxyCommitData->txsPopVersions.back().second : pProxyCommitData->lastTxsPop ) ) { + if(pProxyCommitData->txsPopVersions.size() >= SERVER_KNOBS->MAX_TXS_POP_VERSION_HISTORY) { + TraceEvent(SevWarnAlways, "DiscardingTxsPopHistory").suppressFor(1.0); + pProxyCommitData->txsPopVersions.pop_front(); + } + + pProxyCommitData->txsPopVersions.emplace_back(self->commitVersion, self->msg.popTo); + } + pProxyCommitData->logSystem->popTxs(self->msg.popTo); + + return Void(); +} + +ACTOR Future reply(Context* self) { + state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData; + + const Optional& debugID = self->debugID; + + if ( self->prevVersion && self->commitVersion - self->prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT/2 ) + debug_advanceMinCommittedVersion(UID(), self->commitVersion); + + //TraceEvent("ProxyPushed", pProxyCommitData->dbgid).detail("PrevVersion", prevVersion).detail("Version", commitVersion); + if (debugID.present()) + g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "MasterProxyServer.commitBatch.AfterLogPush"); + + for (auto &p : self->storeCommits) { + ASSERT(!p.second.isReady()); + p.first.get().acknowledge.send(Void()); + ASSERT(p.second.isReady()); + } + + // After logging finishes, we report the commit version to master so that every other proxy can get the most + // up-to-date live committed version. We also maintain the invariant that master's committed version >= self->committedVersion + // by reporting commit version first before updating self->committedVersion. Otherwise, a client may get a commit + // version that the master is not aware of, and next GRV request may get a version less than self->committedVersion. + TEST(pProxyCommitData->committedVersion.get() > self->commitVersion); // A later version was reported committed first + if (SERVER_KNOBS->ASK_READ_VERSION_FROM_MASTER && self->commitVersion > pProxyCommitData->committedVersion.get()) { + wait(pProxyCommitData->master.reportLiveCommittedVersion.getReply( + ReportRawCommittedVersionRequest( + self->commitVersion, + self->lockedAfter, + self->metadataVersionAfter), TaskPriority::ProxyMasterVersionReply)); + } + if( self->commitVersion > pProxyCommitData->committedVersion.get() ) { + pProxyCommitData->locked = self->lockedAfter; + pProxyCommitData->metadataVersion = self->metadataVersionAfter; + pProxyCommitData->committedVersion.set(self->commitVersion); + } + + if (self->forceRecovery) { + TraceEvent(SevWarn, "RestartingTxnSubsystem", pProxyCommitData->dbgid).detail("Stage", "ProxyShutdown"); + throw worker_removed(); + } + + // Send replies to clients + double endTime = g_network->timer(); + // Reset all to zero, used to track the correct index of each commitTransacitonRef on each resolver + + std::fill(self->nextTr.begin(), self->nextTr.end(), 0); + for (int t = 0; t < self->trs.size(); t++) { + auto& tr = self->trs[t]; + if (self->committed[t] == ConflictBatch::TransactionCommitted && (!self->locked || tr.isLockAware())) { + ASSERT_WE_THINK(self->commitVersion != invalidVersion); + tr.reply.send(CommitID(self->commitVersion, t, self->metadataVersionAfter)); + + // aggregate commit cost estimation if committed + ASSERT(tr.commitCostEstimation.present() == tr.tagSet.present()); + if (tr.tagSet.present()) { + TransactionCommitCostEstimation& costEstimation = tr.commitCostEstimation.get(); + for (auto& tag : tr.tagSet.get()) { + pProxyCommitData->transactionTagCommitCostEst[tag] += costEstimation; + } + } + } + else if (self->committed[t] == ConflictBatch::TransactionTooOld) { + tr.reply.sendError(transaction_too_old()); + } + else { + // If enable the option to report conflicting keys from resolvers, we send back all keyranges' indices + // through CommitID + if (tr.transaction.report_conflicting_keys) { + Standalone> conflictingKRIndices; + for (int resolverInd : self->transactionResolverMap[t]) { + auto const& cKRs = + self->resolution[resolverInd] + .conflictingKeyRangeMap[self->nextTr[resolverInd]]; // nextTr[resolverInd] -> index of this trs[t] + // on the resolver + for (auto const& rCRIndex : cKRs) + // read_conflict_range can change when sent to resolvers, mapping the index from resolver-side + // to original index in commitTransactionRef + conflictingKRIndices.push_back(conflictingKRIndices.arena(), + self->txReadConflictRangeIndexMap[t][resolverInd][rCRIndex]); + } + // At least one keyRange index should be returned + ASSERT(conflictingKRIndices.size()); + tr.reply.send(CommitID(invalidVersion, t, Optional(), + Optional>>(conflictingKRIndices))); + } else { + tr.reply.sendError(not_committed()); + } + } + + // Update corresponding transaction indices on each resolver + for (int resolverInd : self->transactionResolverMap[t]) self->nextTr[resolverInd]++; + + // TODO: filter if pipelined with large commit + double duration = endTime - tr.requestTime(); + pProxyCommitData->stats.commitLatencySample.addMeasurement(duration); + if(pProxyCommitData->latencyBandConfig.present()) { + bool filter = self->maxTransactionBytes > pProxyCommitData->latencyBandConfig.get().commitConfig.maxCommitBytes.orDefault(std::numeric_limits::max()); + pProxyCommitData->stats.commitLatencyBands.addMeasurement(endTime - tr.requestTime(), filter); + } + } + + ++pProxyCommitData->stats.commitBatchOut; + pProxyCommitData->stats.txnCommitOut += self->trs.size(); + pProxyCommitData->stats.txnConflicts += self->trs.size() - self->commitCount; + pProxyCommitData->stats.txnCommitOutSuccess += self->commitCount; + + if(now() - pProxyCommitData->lastCoalesceTime > SERVER_KNOBS->RESOLVER_COALESCE_TIME) { + pProxyCommitData->lastCoalesceTime = now(); + int lastSize = pProxyCommitData->keyResolvers.size(); + auto rs = pProxyCommitData->keyResolvers.ranges(); + Version oldestVersion = self->prevVersion - SERVER_KNOBS->MAX_WRITE_TRANSACTION_LIFE_VERSIONS; + for(auto r = rs.begin(); r != rs.end(); ++r) { + while(r->value().size() > 1 && r->value()[1].first < oldestVersion) + r->value().pop_front(); + if(r->value().size() && r->value().front().first < oldestVersion) + r->value().front().first = 0; + } + pProxyCommitData->keyResolvers.coalesce(allKeys); + if(pProxyCommitData->keyResolvers.size() != lastSize) + TraceEvent("KeyResolverSize", pProxyCommitData->dbgid).detail("Size", pProxyCommitData->keyResolvers.size()); + } + + // Dynamic batching for commits + double target_latency = (now() - self->startTime) * SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_LATENCY_FRACTION; + pProxyCommitData->commitBatchInterval = std::max( + SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_MIN, + std::min(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_MAX, + target_latency * SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA + + pProxyCommitData->commitBatchInterval * (1 - SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA))); + + pProxyCommitData->commitBatchesMemBytesCount -= self->currentBatchMemBytesCount; + ASSERT_ABORT(pProxyCommitData->commitBatchesMemBytesCount >= 0); + wait(self->releaseFuture); + + return Void(); +} + } // namespace CommitBatch // Commit one batch of transactions trs @@ -1489,159 +1661,11 @@ ACTOR Future commitBatch( wait(CommitBatch::postResolution(&context)); /////// Phase 4: Logging (network bound; pipelined up to MAX_READ_TRANSACTION_LIFE_VERSIONS (limited by loop above)) - - try { - choose { - when(Version ver = wait(context.loggingComplete)) { - context.pProxyCommitData->minKnownCommittedVersion = std::max(context.pProxyCommitData->minKnownCommittedVersion, ver); - } - when(wait(context.pProxyCommitData->committedVersion.whenAtLeast( context.commitVersion+1 ))) {} - } - } catch(Error &e) { - if(e.code() == error_code_broken_promise) { - throw master_tlog_failed(); - } - throw; - } - - context.pProxyCommitData->lastCommitLatency = now() - context.commitStartTime; - context.pProxyCommitData->lastCommitTime = std::max(context.pProxyCommitData->lastCommitTime.get(), context.commitStartTime); - - wait(yield(TaskPriority::ProxyCommitYield2)); - - if( context.pProxyCommitData->popRemoteTxs && context.msg.popTo > ( context.pProxyCommitData->txsPopVersions.size() ? context.pProxyCommitData->txsPopVersions.back().second : context.pProxyCommitData->lastTxsPop ) ) { - if(context.pProxyCommitData->txsPopVersions.size() >= SERVER_KNOBS->MAX_TXS_POP_VERSION_HISTORY) { - TraceEvent(SevWarnAlways, "DiscardingTxsPopHistory").suppressFor(1.0); - context.pProxyCommitData->txsPopVersions.pop_front(); - } - - context.pProxyCommitData->txsPopVersions.emplace_back(context.commitVersion, context.msg.popTo); - } - context.pProxyCommitData->logSystem->popTxs(context.msg.popTo); + wait(CommitBatch::transactionLogging(&context)); /////// Phase 5: Replies (CPU bound; no particular order required, though ordered execution would be best for latency) - if ( context.prevVersion && context.commitVersion - context.prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT/2 ) - debug_advanceMinCommittedVersion(UID(), context.commitVersion); + wait(CommitBatch::reply(&context)); - //TraceEvent("ProxyPushed", context.pProxyCommitData->dbgid).detail("PrevVersion", prevVersion).detail("Version", commitVersion); - if (context.debugID.present()) - g_traceBatch.addEvent("CommitDebug", context.debugID.get().first(), "MasterProxyServer.commitBatch.AfterLogPush"); - - for (auto &p : context.storeCommits) { - ASSERT(!p.second.isReady()); - p.first.get().acknowledge.send(Void()); - ASSERT(p.second.isReady()); - } - - // After logging finishes, we report the commit version to master so that every other proxy can get the most - // up-to-date live committed version. We also maintain the invariant that master's committed version >= self->committedVersion - // by reporting commit version first before updating self->committedVersion. Otherwise, a client may get a commit - // version that the master is not aware of, and next GRV request may get a version less than self->committedVersion. - TEST(context.pProxyCommitData->committedVersion.get() > context.commitVersion); // A later version was reported committed first - if (SERVER_KNOBS->ASK_READ_VERSION_FROM_MASTER && context.commitVersion > context.pProxyCommitData->committedVersion.get()) { - wait(context.pProxyCommitData->master.reportLiveCommittedVersion.getReply(ReportRawCommittedVersionRequest(context.commitVersion, context.lockedAfter, context.metadataVersionAfter), TaskPriority::ProxyMasterVersionReply)); - } - if( context.commitVersion > context.pProxyCommitData->committedVersion.get() ) { - context.pProxyCommitData->locked = context.lockedAfter; - context.pProxyCommitData->metadataVersion = context.metadataVersionAfter; - context.pProxyCommitData->committedVersion.set(context.commitVersion); - } - - if (context.forceRecovery) { - TraceEvent(SevWarn, "RestartingTxnSubsystem", context.pProxyCommitData->dbgid).detail("Stage", "ProxyShutdown"); - throw worker_removed(); - } - - // Send replies to clients - double endTime = g_network->timer(); - // Reset all to zero, used to track the correct index of each commitTransacitonRef on each resolver - std::fill(context.nextTr.begin(), context.nextTr.end(), 0); - for (int t = 0; t < context.trs.size(); t++) { - if (context.committed[t] == ConflictBatch::TransactionCommitted && (!context.locked || context.trs[t].isLockAware())) { - ASSERT_WE_THINK(context.commitVersion != invalidVersion); - context.trs[t].reply.send(CommitID(context.commitVersion, t, context.metadataVersionAfter)); - - // aggregate commit cost estimation if committed - ASSERT(context.trs[t].commitCostEstimation.present() == context.trs[t].tagSet.present()); - if (context.trs[t].tagSet.present()) { - TransactionCommitCostEstimation& costEstimation = context.trs[t].commitCostEstimation.get(); - for (auto& tag : context.trs[t].tagSet.get()) { - self->transactionTagCommitCostEst[tag] += costEstimation; - } - } - } - else if (context.committed[t] == ConflictBatch::TransactionTooOld) { - context.trs[t].reply.sendError(transaction_too_old()); - } - else { - // If enable the option to report conflicting keys from resolvers, we send back all keyranges' indices - // through CommitID - if (context.trs[t].transaction.report_conflicting_keys) { - Standalone> conflictingKRIndices; - for (int resolverInd : context.transactionResolverMap[t]) { - auto const& cKRs = - context.resolution[resolverInd] - .conflictingKeyRangeMap[context.nextTr[resolverInd]]; // nextTr[resolverInd] -> index of this trs[t] - // on the resolver - for (auto const& rCRIndex : cKRs) - // read_conflict_range can change when sent to resolvers, mapping the index from resolver-side - // to original index in commitTransactionRef - conflictingKRIndices.push_back(conflictingKRIndices.arena(), - context.txReadConflictRangeIndexMap[t][resolverInd][rCRIndex]); - } - // At least one keyRange index should be returned - ASSERT(conflictingKRIndices.size()); - context.trs[t].reply.send(CommitID(invalidVersion, t, Optional(), - Optional>>(conflictingKRIndices))); - } else { - context.trs[t].reply.sendError(not_committed()); - } - } - - // Update corresponding transaction indices on each resolver - for (int resolverInd : context.transactionResolverMap[t]) context.nextTr[resolverInd]++; - - // TODO: filter if pipelined with large commit - double duration = endTime - context.trs[t].requestTime(); - context.pProxyCommitData->stats.commitLatencySample.addMeasurement(duration); - if(context.pProxyCommitData->latencyBandConfig.present()) { - bool filter = context.maxTransactionBytes > context.pProxyCommitData->latencyBandConfig.get().commitConfig.maxCommitBytes.orDefault(std::numeric_limits::max()); - context.pProxyCommitData->stats.commitLatencyBands.addMeasurement(endTime - context.trs[t].requestTime(), filter); - } - } - - ++context.pProxyCommitData->stats.commitBatchOut; - context.pProxyCommitData->stats.txnCommitOut += context.trs.size(); - context.pProxyCommitData->stats.txnConflicts += context.trs.size() - context.commitCount; - context.pProxyCommitData->stats.txnCommitOutSuccess += context.commitCount; - - if(now() - context.pProxyCommitData->lastCoalesceTime > SERVER_KNOBS->RESOLVER_COALESCE_TIME) { - context.pProxyCommitData->lastCoalesceTime = now(); - int lastSize = context.pProxyCommitData->keyResolvers.size(); - auto rs = context.pProxyCommitData->keyResolvers.ranges(); - Version oldestVersion = context.prevVersion - SERVER_KNOBS->MAX_WRITE_TRANSACTION_LIFE_VERSIONS; - for(auto r = rs.begin(); r != rs.end(); ++r) { - while(r->value().size() > 1 && r->value()[1].first < oldestVersion) - r->value().pop_front(); - if(r->value().size() && r->value().front().first < oldestVersion) - r->value().front().first = 0; - } - context.pProxyCommitData->keyResolvers.coalesce(allKeys); - if(context.pProxyCommitData->keyResolvers.size() != lastSize) - TraceEvent("KeyResolverSize", context.pProxyCommitData->dbgid).detail("Size", context.pProxyCommitData->keyResolvers.size()); - } - - // Dynamic batching for commits - double target_latency = (now() - context.startTime) * SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_LATENCY_FRACTION; - context.pProxyCommitData->commitBatchInterval = std::max( - SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_MIN, - std::min(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_MAX, - target_latency * SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA + - context.pProxyCommitData->commitBatchInterval * (1 - SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA))); - - context.pProxyCommitData->commitBatchesMemBytesCount -= currentBatchMemBytesCount; - ASSERT_ABORT(context.pProxyCommitData->commitBatchesMemBytesCount >= 0); - wait(context.releaseFuture); return Void(); } From 9f8244f2dd50bb2ba8488dd80c0e6b8a3f05d824 Mon Sep 17 00:00:00 2001 From: Xiaoge Su Date: Mon, 3 Aug 2020 21:43:16 -0700 Subject: [PATCH 4/8] Add a shortcut for applyMetadataMutations --- fdbserver/MasterProxyServer.actor.cpp | 54 ++++++++++++++------------- 1 file changed, 29 insertions(+), 25 deletions(-) diff --git a/fdbserver/MasterProxyServer.actor.cpp b/fdbserver/MasterProxyServer.actor.cpp index 841801d2e3..cc39aec637 100644 --- a/fdbserver/MasterProxyServer.actor.cpp +++ b/fdbserver/MasterProxyServer.actor.cpp @@ -523,6 +523,23 @@ struct ProxyCommitData { } }; +/// A handy applyMetadataMutations uses ProxyCommitData class +void applyMetadataMutations(ProxyCommitData& proxyCommitData, Arena& arena, Reference logSystem, + const VectorRef& mutations, LogPushData* pToCommit, bool& confChange, + Version popVersion, bool initialCommit) { + + std::map* pUidApplyMutationsData = nullptr; + if (proxyCommitData.firstProxy) { + pUidApplyMutationsData = &proxyCommitData.uid_applyMutationsData; + } + + applyMetadataMutations(proxyCommitData.dbgid, arena, mutations, proxyCommitData.txnStateStore, pToCommit, + &confChange, logSystem, popVersion, &proxyCommitData.vecBackupKeys, &proxyCommitData.keyInfo, + &proxyCommitData.cacheInfo, pUidApplyMutationsData, proxyCommitData.commit, + proxyCommitData.cx, &proxyCommitData.committedVersion, &proxyCommitData.storageCache, + &proxyCommitData.tag_popped, initialCommit); +} + struct ResolutionRequestBuilder { ProxyCommitData* self; vector requests; @@ -1136,12 +1153,12 @@ ACTOR Future postResolution(Context* self) { bool committed = true; for (int resolver = 0; resolver < self->resolution.size(); resolver++) committed = committed && self->resolution[resolver].stateMutations[versionIndex][transactionIndex].committed; - if (committed) - applyMetadataMutations( - pProxyCommitData->dbgid, self->arena, - self->resolution[0].stateMutations[versionIndex][transactionIndex].mutations, - pProxyCommitData->txnStateStore, nullptr, &self->forceRecovery, pProxyCommitData->logSystem, 0, &pProxyCommitData->vecBackupKeys, &pProxyCommitData->keyInfo, &pProxyCommitData->cacheInfo, pProxyCommitData->firstProxy ? &pProxyCommitData->uid_applyMutationsData : nullptr, pProxyCommitData->commit, pProxyCommitData->cx, &pProxyCommitData->committedVersion, &pProxyCommitData->storageCache, &pProxyCommitData->tag_popped); - + if (committed) { + applyMetadataMutations(*self->pProxyCommitData, self->arena, self->pProxyCommitData->logSystem, + self->resolution[0].stateMutations[versionIndex][transactionIndex].mutations, + /* pToCommit= */ nullptr, self->forceRecovery, + /* popVersion= */ 0, /* initialCommit */ false); + } if( self->resolution[0].stateMutations[versionIndex][transactionIndex].mutations.size() && self->firstStateMutations ) { ASSERT(committed); self->firstStateMutations = false; @@ -1220,24 +1237,9 @@ ACTOR Future postResolution(Context* self) { { if (self->committed[t] == ConflictBatch::TransactionCommitted && (!self->locked || trs[t].isLockAware())) { self->commitCount++; - applyMetadataMutations( - pProxyCommitData->dbgid, self->arena, - trs[t].transaction.mutations, - pProxyCommitData->txnStateStore, - &self->toCommit, - &self->forceRecovery, - pProxyCommitData->logSystem, - self->commitVersion + 1, - &pProxyCommitData->vecBackupKeys, - &pProxyCommitData->keyInfo, - &pProxyCommitData->cacheInfo, - pProxyCommitData->firstProxy ? &pProxyCommitData->uid_applyMutationsData : nullptr, - pProxyCommitData->commit, - pProxyCommitData->cx, - &pProxyCommitData->committedVersion, - &pProxyCommitData->storageCache, - &pProxyCommitData->tag_popped - ); + applyMetadataMutations(*pProxyCommitData, self->arena, pProxyCommitData->logSystem, + trs[t].transaction.mutations, &self->toCommit, self->forceRecovery, + self->commitVersion + 1, /* initialCommit= */ false); } if(self->firstStateMutations) { ASSERT(self->committed[t] == ConflictBatch::TransactionCommitted); @@ -2492,7 +2494,9 @@ ACTOR Future masterProxyServerCore( Arena arena; bool confChanges; - applyMetadataMutations(commitData.dbgid, arena, mutations, commitData.txnStateStore, nullptr, &confChanges, Reference(), 0, &commitData.vecBackupKeys, &commitData.keyInfo, &commitData.cacheInfo, commitData.firstProxy ? &commitData.uid_applyMutationsData : nullptr, commitData.commit, commitData.cx, &commitData.committedVersion, &commitData.storageCache, &commitData.tag_popped, true ); + applyMetadataMutations(commitData, arena, Reference(), mutations, + /* pToCommit= */ nullptr, confChanges, + /* popVersion= */ 0, /* initialCommit= */ true); } auto lockedKey = commitData.txnStateStore->readValue(databaseLockedKey).get(); From 377f0695b032bb454b283d9716cc942830e44ec4 Mon Sep 17 00:00:00 2001 From: Xiaoge Su Date: Wed, 29 Jul 2020 00:42:52 -0700 Subject: [PATCH 5/8] Refactor postResolution --- fdbserver/MasterProxyServer.actor.cpp | 309 ++++++++++++++------------ 1 file changed, 171 insertions(+), 138 deletions(-) diff --git a/fdbserver/MasterProxyServer.actor.cpp b/fdbserver/MasterProxyServer.actor.cpp index cc39aec637..71e1b16604 100644 --- a/fdbserver/MasterProxyServer.actor.cpp +++ b/fdbserver/MasterProxyServer.actor.cpp @@ -1110,40 +1110,19 @@ ACTOR Future getResolution(Context* self) { return Void(); } -ACTOR Future postResolution(Context* self) { - state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData; - state std::vector& trs = self->trs; - state const int64_t localBatchNumber = self->localBatchNumber; - state const Optional& debugID = self->debugID; +void assertResolutionStateMutationsSizeConsistent( + const std::vector& resolution) { - TEST(pProxyCommitData->latestLocalCommitBatchLogging.get() < localBatchNumber - 1); // Queuing post-resolution commit processing - wait(pProxyCommitData->latestLocalCommitBatchLogging.whenAtLeast(localBatchNumber - 1)); - wait(yield(TaskPriority::ProxyCommitYield1)); - - self->computeStart = g_network->now(); - - pProxyCommitData->stats.txnCommitResolved += trs.size(); - - if (debugID.present()) { - g_traceBatch.addEvent( - "CommitDebug", debugID.get().first(), - "MasterProxyServer.commitBatch.ProcessingMutations" - ); + for (int r = 1; r < resolution.size(); r++) { + ASSERT(resolution[r].stateMutations.size() == resolution[0].stateMutations.size()); + for(int s = 0; s < resolution[r].stateMutations.size(); s++) { + ASSERT(resolution[r].stateMutations[s].size() == resolution[0].stateMutations[s].size()); + } } +} - self->isMyFirstBatch = !pProxyCommitData->version; - self->oldCoordinators = pProxyCommitData->txnStateStore->readValue(coordinatorsKey).get(); - - //TraceEvent("ResolutionResult", pProxyCommitData->dbgid).detail("Sequence", sequence).detail("Version", commitVersion).detail("StateMutationProxies", resolution[0].stateMutations.size()).detail("WaitForResolution", now()-t1).detail("R0Committed", resolution[0].committed.size()) - // .detail("Transactions", trs.size()); - - for(int r=1; r < self->resolution.size(); r++) { - ASSERT( self->resolution[r].stateMutations.size() == self->resolution[0].stateMutations.size() ); - for(int s = 0; s < self->resolution[r].stateMutations.size(); s++) - ASSERT( self->resolution[r].stateMutations[s].size() == self->resolution[0].stateMutations[s].size() ); - } - - // Compute and apply "metadata" effects of each other proxy's most recent batch +// Compute and apply "metadata" effects of each other proxy's most recent batch +void applyMetadataEffect(Context* self) { bool initialState = self->isMyFirstBatch; self->firstStateMutations = self->isMyFirstBatch; for (int versionIndex = 0; versionIndex < self->resolution[0].stateMutations.size(); versionIndex++) { @@ -1164,20 +1143,16 @@ ACTOR Future postResolution(Context* self) { self->firstStateMutations = false; self->forceRecovery = false; } - //TraceEvent("MetadataTransaction", pProxyCommitData->dbgid).detail("Committed", committed).detail("Mutations", resolution[0].stateMutations[versionIndex][transactionIndex].second.size()).detail("R1Mutations", resolution.back().stateMutations[versionIndex][transactionIndex].second.size()); } - //TraceEvent("MetadataBatch", pProxyCommitData->dbgid).detail("Transactions", resolution[0].stateMutations[versionIndex].size()); // These changes to txnStateStore will be committed by the other proxy, so we simply discard the commit message - auto fcm = pProxyCommitData->logAdapter->getCommitMessage(); - self->storeCommits.emplace_back(fcm, pProxyCommitData->txnStateStore->commit()); - //discardCommit( dbgid, fcm, txnStateStore->commit() ); + auto fcm = self->pProxyCommitData->logAdapter->getCommitMessage(); + self->storeCommits.emplace_back(fcm, self->pProxyCommitData->txnStateStore->commit()); if (initialState) { - //TraceEvent("ResyncLog", dbgid); initialState = false; self->forceRecovery = false; - pProxyCommitData->txnStateStore->resyncLog(); + self->pProxyCommitData->txnStateStore->resyncLog(); for (auto &p : self->storeCommits) { ASSERT(!p.second.isReady()); @@ -1187,16 +1162,20 @@ ACTOR Future postResolution(Context* self) { self->storeCommits.clear(); } } +} + +/// Determine which transactions actually committed (conservatively) by combining results from the resolvers +void determineCommittedTransactions(Context* self) { + auto pProxyCommitData = self->pProxyCommitData; + const auto& trs = self->trs; - // Determine which transactions actually committed (conservatively) by combining results from the resolvers ASSERT(self->transactionResolverMap.size() == self->committed.size()); // For each commitTransactionRef, it is only sent to resolvers specified in transactionResolverMap // Thus, we use this nextTr to track the correct transaction index on each resolver. self->nextTr.resize(self->resolution.size()); for (int t = 0; t < trs.size(); t++) { uint8_t commit = ConflictBatch::TransactionCommitted; - for (int r : self->transactionResolverMap[t]) - { + for (int r : self->transactionResolverMap[t]) { commit = std::min(self->resolution[r].committed[self->nextTr[r]++], commit); } self->committed[t] = commit; @@ -1226,15 +1205,15 @@ ACTOR Future postResolution(Context* self) { } } } +} - if(self->forceRecovery) { - wait( Future(Never()) ); - } +// This first pass through committed transactions deals with "metadata" effects (modifications of txnStateStore, changes to storage servers' responsibilities) +ACTOR Future applyMetadataToCommittedTransactions(Context* self) { + auto pProxyCommitData = self->pProxyCommitData; + const auto& trs = self->trs; - // This first pass through committed transactions deals with "metadata" effects (modifications of txnStateStore, changes to storage servers' responsibilities) int t; - for (t = 0; t < trs.size() && !self->forceRecovery; t++) - { + for (t = 0; t < trs.size() && !self->forceRecovery; t++) { if (self->committed[t] == ConflictBatch::TransactionCommitted && (!self->locked || trs[t].isLockAware())) { self->commitCount++; applyMetadataMutations(*pProxyCommitData, self->arena, pProxyCommitData->logSystem, @@ -1248,7 +1227,7 @@ ACTOR Future postResolution(Context* self) { } } if (self->forceRecovery) { - for (; t< trs.size(); t++) + for (; t < trs.size(); t++) self->committed[t] = ConflictBatch::TransactionConflict; TraceEvent(SevWarn, "RestartingTxnSubsystem", pProxyCommitData->dbgid).detail("Stage", "AwaitCommit"); } @@ -1269,115 +1248,169 @@ ACTOR Future postResolution(Context* self) { ASSERT(false); // ChangeCoordinatorsRequest should always throw } - // This second pass through committed transactions assigns the actual mutations to the appropriate storage servers' tags + return Void(); +} + +/// This second pass through committed transactions assigns the actual mutations to the appropriate storage servers' tags +ACTOR Future assignMutationsToStorageServers(Context* self) { + state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData; + state std::vector& trs = self->trs; + for (; self->transactionNum < trs.size(); self->transactionNum++) { - if (self->committed[self->transactionNum] == ConflictBatch::TransactionCommitted && (!self->locked || trs[self->transactionNum].isLockAware())) { - state int mutationNum = 0; - state VectorRef* pMutations = &trs[self->transactionNum].transaction.mutations; - for (; mutationNum < pMutations->size(); mutationNum++) { - if(self->yieldBytes > SERVER_KNOBS->DESIRED_TOTAL_BYTES) { - self->yieldBytes = 0; - if(g_network->check_yield(TaskPriority::ProxyCommitYield1)) { - self->computeDuration += g_network->timer() - self->computeStart; - wait(delay(0, TaskPriority::ProxyCommitYield1)); - self->computeStart = g_network->timer(); - } + if (!(self->committed[self->transactionNum] == ConflictBatch::TransactionCommitted && (!self->locked || trs[self->transactionNum].isLockAware()))) { + continue; + } + + state int mutationNum = 0; + state VectorRef* pMutations = &trs[self->transactionNum].transaction.mutations; + for (; mutationNum < pMutations->size(); mutationNum++) { + if(self->yieldBytes > SERVER_KNOBS->DESIRED_TOTAL_BYTES) { + self->yieldBytes = 0; + if(g_network->check_yield(TaskPriority::ProxyCommitYield1)) { + self->computeDuration += g_network->timer() - self->computeStart; + wait(delay(0, TaskPriority::ProxyCommitYield1)); + self->computeStart = g_network->timer(); + } + } + + auto& m = (*pMutations)[mutationNum]; + self->mutationCount++; + self->mutationBytes += m.expectedSize(); + self->yieldBytes += m.expectedSize(); + // Determine the set of tags (responsible storage servers) for the mutation, splitting it + // if necessary. Serialize (splits of) the mutation into the message buffer and add the tags. + + if (isSingleKeyMutation((MutationRef::Type) m.type)) { + auto& tags = pProxyCommitData->tagsForKey(m.param1); + + if(pProxyCommitData->singleKeyMutationEvent->enabled) { + KeyRangeRef shard = pProxyCommitData->keyInfo.rangeContaining(m.param1).range(); + pProxyCommitData->singleKeyMutationEvent->tag1 = (int64_t)tags[0].id; + pProxyCommitData->singleKeyMutationEvent->tag2 = (int64_t)tags[1].id; + pProxyCommitData->singleKeyMutationEvent->tag3 = (int64_t)tags[2].id; + pProxyCommitData->singleKeyMutationEvent->shardBegin = shard.begin; + pProxyCommitData->singleKeyMutationEvent->shardEnd = shard.end; + pProxyCommitData->singleKeyMutationEvent->log(); } - auto& m = (*pMutations)[mutationNum]; - self->mutationCount++; - self->mutationBytes += m.expectedSize(); - self->yieldBytes += m.expectedSize(); - // Determine the set of tags (responsible storage servers) for the mutation, splitting it - // if necessary. Serialize (splits of) the mutation into the message buffer and add the tags. - - if (isSingleKeyMutation((MutationRef::Type) m.type)) { - auto& tags = pProxyCommitData->tagsForKey(m.param1); - - if(pProxyCommitData->singleKeyMutationEvent->enabled) { - KeyRangeRef shard = pProxyCommitData->keyInfo.rangeContaining(m.param1).range(); - pProxyCommitData->singleKeyMutationEvent->tag1 = (int64_t)tags[0].id; - pProxyCommitData->singleKeyMutationEvent->tag2 = (int64_t)tags[1].id; - pProxyCommitData->singleKeyMutationEvent->tag3 = (int64_t)tags[2].id; - pProxyCommitData->singleKeyMutationEvent->shardBegin = shard.begin; - pProxyCommitData->singleKeyMutationEvent->shardEnd = shard.end; - pProxyCommitData->singleKeyMutationEvent->log(); - } - - DEBUG_MUTATION("ProxyCommit", self->commitVersion, m).detail("Dbgid", pProxyCommitData->dbgid).detail("To", tags).detail("Mutation", m); - - self->toCommit.addTags(tags); - if(pProxyCommitData->cacheInfo[m.param1]) { - self->toCommit.addTag(cacheTag); - } - self->toCommit.addTypedMessage(m); + DEBUG_MUTATION("ProxyCommit", self->commitVersion, m).detail("Dbgid", pProxyCommitData->dbgid).detail("To", tags).detail("Mutation", m); + self->toCommit.addTags(tags); + if(pProxyCommitData->cacheInfo[m.param1]) { + self->toCommit.addTag(cacheTag); } - else if (m.type == MutationRef::ClearRange) { - KeyRangeRef clearRange(KeyRangeRef(m.param1, m.param2)); - auto ranges = pProxyCommitData->keyInfo.intersectingRanges(clearRange); - auto firstRange = ranges.begin(); - ++firstRange; - if (firstRange == ranges.end()) { - // Fast path - DEBUG_MUTATION("ProxyCommit", self->commitVersion, m).detail("Dbgid", pProxyCommitData->dbgid).detail("To", ranges.begin().value().tags).detail("Mutation", m); + self->toCommit.addTypedMessage(m); + } + else if (m.type == MutationRef::ClearRange) { + KeyRangeRef clearRange(KeyRangeRef(m.param1, m.param2)); + auto ranges = pProxyCommitData->keyInfo.intersectingRanges(clearRange); + auto firstRange = ranges.begin(); + ++firstRange; + if (firstRange == ranges.end()) { + // Fast path + DEBUG_MUTATION("ProxyCommit", self->commitVersion, m).detail("Dbgid", pProxyCommitData->dbgid).detail("To", ranges.begin().value().tags).detail("Mutation", m); - ranges.begin().value().populateTags(); - self->toCommit.addTags(ranges.begin().value().tags); + ranges.begin().value().populateTags(); + self->toCommit.addTags(ranges.begin().value().tags); + } + else { + TEST(true); //A clear range extends past a shard boundary + std::set allSources; + for (auto r : ranges) { + r.value().populateTags(); + allSources.insert(r.value().tags.begin(), r.value().tags.end()); } - else { - TEST(true); //A clear range extends past a shard boundary - std::set allSources; - for (auto r : ranges) { - r.value().populateTags(); - allSources.insert(r.value().tags.begin(), r.value().tags.end()); - } - DEBUG_MUTATION("ProxyCommit", self->commitVersion, m).detail("Dbgid", pProxyCommitData->dbgid).detail("To", allSources).detail("Mutation", m); + DEBUG_MUTATION("ProxyCommit", self->commitVersion, m).detail("Dbgid", pProxyCommitData->dbgid).detail("To", allSources).detail("Mutation", m); - self->toCommit.addTags(allSources); - } - if(pProxyCommitData->needsCacheTag(clearRange)) { - self->toCommit.addTag(cacheTag); - } - self->toCommit.addTypedMessage(m); - } else { - UNREACHABLE(); + self->toCommit.addTags(allSources); } - // Check on backing up key, if backup ranges are defined and a normal key - if (pProxyCommitData->vecBackupKeys.size() > 1 && (normalKeys.contains(m.param1) || m.param1 == metadataVersionKey)) { - if (m.type != MutationRef::Type::ClearRange) { - // Add the mutation to the relevant backup tag - for (auto backupName : pProxyCommitData->vecBackupKeys[m.param1]) { - self->logRangeMutations[backupName].push_back_deep(self->logRangeMutationsArena, m); - } - } - else { - KeyRangeRef mutationRange(m.param1, m.param2); - KeyRangeRef intersectionRange; + if(pProxyCommitData->needsCacheTag(clearRange)) { + self->toCommit.addTag(cacheTag); + } + self->toCommit.addTypedMessage(m); + } else { + UNREACHABLE(); + } - // Identify and add the intersecting ranges of the mutation to the array of mutations to serialize - for (auto backupRange : pProxyCommitData->vecBackupKeys.intersectingRanges(mutationRange)) - { - // Get the backup sub range - const auto& backupSubrange = backupRange.range(); + // Check on backing up key, if backup ranges are defined and a normal key + if (!(pProxyCommitData->vecBackupKeys.size() > 1 && (normalKeys.contains(m.param1) || m.param1 == metadataVersionKey))) { + continue; + } - // Determine the intersecting range - intersectionRange = mutationRange & backupSubrange; + if (m.type != MutationRef::Type::ClearRange) { + // Add the mutation to the relevant backup tag + for (auto backupName : pProxyCommitData->vecBackupKeys[m.param1]) { + self->logRangeMutations[backupName].push_back_deep(self->logRangeMutationsArena, m); + } + } + else { + KeyRangeRef mutationRange(m.param1, m.param2); + KeyRangeRef intersectionRange; - // Create the custom mutation for the specific backup tag - MutationRef backupMutation(MutationRef::Type::ClearRange, intersectionRange.begin, intersectionRange.end); + // Identify and add the intersecting ranges of the mutation to the array of mutations to serialize + for (auto backupRange : pProxyCommitData->vecBackupKeys.intersectingRanges(mutationRange)) + { + // Get the backup sub range + const auto& backupSubrange = backupRange.range(); - // Add the mutation to the relevant backup tag - for (auto backupName : backupRange.value()) { - self->logRangeMutations[backupName].push_back_deep(self->logRangeMutationsArena, backupMutation); - } - } + // Determine the intersecting range + intersectionRange = mutationRange & backupSubrange; + + // Create the custom mutation for the specific backup tag + MutationRef backupMutation(MutationRef::Type::ClearRange, intersectionRange.begin, intersectionRange.end); + + // Add the mutation to the relevant backup tag + for (auto backupName : backupRange.value()) { + self->logRangeMutations[backupName].push_back_deep(self->logRangeMutationsArena, backupMutation); } } } } } + return Void(); +} + +ACTOR Future postResolution(Context* self) { + state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData; + state std::vector& trs = self->trs; + state const int64_t localBatchNumber = self->localBatchNumber; + state const Optional& debugID = self->debugID; + + TEST(pProxyCommitData->latestLocalCommitBatchLogging.get() < localBatchNumber - 1); // Queuing post-resolution commit processing + wait(pProxyCommitData->latestLocalCommitBatchLogging.whenAtLeast(localBatchNumber - 1)); + wait(yield(TaskPriority::ProxyCommitYield1)); + + self->computeStart = g_network->now(); + + pProxyCommitData->stats.txnCommitResolved += trs.size(); + + if (debugID.present()) { + g_traceBatch.addEvent( + "CommitDebug", debugID.get().first(), + "MasterProxyServer.commitBatch.ProcessingMutations" + ); + } + + self->isMyFirstBatch = !pProxyCommitData->version; + self->oldCoordinators = pProxyCommitData->txnStateStore->readValue(coordinatorsKey).get(); + + assertResolutionStateMutationsSizeConsistent(self->resolution); + + applyMetadataEffect(self); + + determineCommittedTransactions(self); + + if(self->forceRecovery) { + wait( Future(Never()) ); + } + + // First pass + wait(applyMetadataToCommittedTransactions(self)); + + // Second pass + wait(assignMutationsToStorageServers(self)); + // Serialize and backup the mutations as a single mutation if ((pProxyCommitData->vecBackupKeys.size() > 1) && self->logRangeMutations.size()) { wait( addBackupMutations(pProxyCommitData, &self->logRangeMutations, &self->toCommit, self->commitVersion, &self->computeDuration, &self->computeStart) ); From ed1bbdb8a7759a17e7610a033d2df0288ef17cda Mon Sep 17 00:00:00 2001 From: Xiaoge Su Date: Mon, 3 Aug 2020 21:19:14 -0700 Subject: [PATCH 6/8] fixup! Update code per review --- fdbserver/MasterProxyServer.actor.cpp | 40 +++++++++++---------------- 1 file changed, 16 insertions(+), 24 deletions(-) diff --git a/fdbserver/MasterProxyServer.actor.cpp b/fdbserver/MasterProxyServer.actor.cpp index 71e1b16604..cc3d8d33e4 100644 --- a/fdbserver/MasterProxyServer.actor.cpp +++ b/fdbserver/MasterProxyServer.actor.cpp @@ -828,11 +828,8 @@ ACTOR Future releaseResolvingAfter(ProxyCommitData* self, Future rel namespace CommitBatch { struct Context { - typedef std::vector< - std::pair< - Future, Future - >> StoreCommit_t; - + using StoreCommit_t = std::vector, Future>>; + ProxyCommitData* const pProxyCommitData; std::vector trs; int currentBatchMemBytesCount; @@ -848,7 +845,7 @@ struct Context { int batchOperations = 0; - Span span = Span("MP:commitBatch"_loc); + Span span = Span("MP:commitBatch"_loc); int64_t batchBytes = 0; @@ -913,7 +910,7 @@ struct Context { const std::vector*, const int); - void setupDebugTrack(); + void setupTraceBatch(); private: void evaluateBatchSize(); @@ -952,9 +949,8 @@ Context::Context( ASSERT(SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS <= SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT); } -void Context::setupDebugTrack() { - for (int t = 0; t < trs.size(); ++t) { - auto& tr = trs[t]; +void Context::setupTraceBatch() { + for (const auto& tr : trs) { if (tr.debugID.present()) { if (!debugID.present()) { debugID = nondeterministicRandom()->randomUniqueID(); @@ -979,8 +975,8 @@ void Context::setupDebugTrack() { } void Context::evaluateBatchSize() { - for (auto& tr: trs) { - auto& mutations = tr.transaction.mutations; + for (const auto& tr : trs) { + const auto& mutations = tr.transaction.mutations; batchOperations += mutations.size(); batchBytes += mutations.expectedSize(); } @@ -1010,12 +1006,8 @@ ACTOR Future preresolutionProcessing(Context* self) { ); } - GetCommitVersionRequest req( - self->span.context, - pProxyCommitData->commitVersionRequestNumber++, - pProxyCommitData->mostRecentProcessedRequestNumber, - pProxyCommitData->dbgid - ); + GetCommitVersionRequest req(self->span.context, pProxyCommitData->commitVersionRequestNumber++, + pProxyCommitData->mostRecentProcessedRequestNumber, pProxyCommitData->dbgid); GetCommitVersionReply versionReply = wait(brokenPromiseToNever( pProxyCommitData->master.getCommitVersion.getReply( req, TaskPriority::ProxyMasterVersionReply @@ -1049,6 +1041,8 @@ ACTOR Future preresolutionProcessing(Context* self) { } ACTOR Future getResolution(Context* self) { + // Sending these requests is the fuzzy border between phase 1 and phase 2; it could conceivably overlap with + // resolution processing but is still using CPU ProxyCommitData* pProxyCommitData = self->pProxyCommitData; std::vector& trs = self->trs; @@ -1063,10 +1057,8 @@ ACTOR Future getResolution(Context* self) { self->maxTransactionBytes = 0; for (int t = 0; t < trs.size(); t++) { requests.addTransaction(trs[t].transaction, t); - conflictRangeCount += ( - trs[t].transaction.read_conflict_ranges.size() + - trs[t].transaction.write_conflict_ranges.size() - ); + conflictRangeCount += + trs[t].transaction.read_conflict_ranges.size() + trs[t].transaction.write_conflict_ranges.size(); //TraceEvent("MPTransactionDump", self->dbgid).detail("Snapshot", trs[t].transaction.read_snapshot); //for(auto& m : trs[t].transaction.mutations) self->maxTransactionBytes = std::max( @@ -1626,11 +1618,11 @@ ACTOR Future reply(Context* self) { for (int resolverInd : self->transactionResolverMap[t]) self->nextTr[resolverInd]++; // TODO: filter if pipelined with large commit - double duration = endTime - tr.requestTime(); + const double duration = endTime - tr.requestTime(); pProxyCommitData->stats.commitLatencySample.addMeasurement(duration); if(pProxyCommitData->latencyBandConfig.present()) { bool filter = self->maxTransactionBytes > pProxyCommitData->latencyBandConfig.get().commitConfig.maxCommitBytes.orDefault(std::numeric_limits::max()); - pProxyCommitData->stats.commitLatencyBands.addMeasurement(endTime - tr.requestTime(), filter); + pProxyCommitData->stats.commitLatencyBands.addMeasurement(duration, filter); } } From d8bb36c4c8289e74428c131abe7131af07910f4a Mon Sep 17 00:00:00 2001 From: Xiaoge Su Date: Wed, 5 Aug 2020 12:20:52 -0700 Subject: [PATCH 7/8] Refactor applyMetadataMutations to accept less args --- fdbserver/ApplyMetadataMutation.cpp | 54 +++++- fdbserver/ApplyMetadataMutation.h | 15 +- fdbserver/CMakeLists.txt | 1 + fdbserver/MasterProxyServer.actor.cpp | 234 +--------------------- fdbserver/ProxyCommitData.actor.h | 269 ++++++++++++++++++++++++++ fdbserver/masterserver.actor.cpp | 36 ++-- 6 files changed, 342 insertions(+), 267 deletions(-) create mode 100644 fdbserver/ProxyCommitData.actor.h diff --git a/fdbserver/ApplyMetadataMutation.cpp b/fdbserver/ApplyMetadataMutation.cpp index eda632aefb..2df5f5af42 100644 --- a/fdbserver/ApplyMetadataMutation.cpp +++ b/fdbserver/ApplyMetadataMutation.cpp @@ -45,9 +45,14 @@ Reference getStorageInfo(UID id, std::map const& mutations, IKeyValueStore* txnStateStore, LogPushData* toCommit, bool *confChange, Reference logSystem, Version popVersion, - KeyRangeMap >* vecBackupKeys, KeyRangeMap* keyInfo, KeyRangeMap* cacheInfo, std::map* uid_applyMutationsData, RequestStream commit, - Database cx, NotifiedVersion* commitVersion, std::map>* storageCache, std::map* tag_popped, bool initialCommit ) { +void applyMetadataMutations(UID const& dbgid, Arena& arena, VectorRef const& mutations, + IKeyValueStore* txnStateStore, LogPushData* toCommit, bool& confChange, + Reference logSystem, Version popVersion, + KeyRangeMap>* vecBackupKeys, KeyRangeMap* keyInfo, + KeyRangeMap* cacheInfo, std::map* uid_applyMutationsData, + RequestStream commit, Database cx, NotifiedVersion* commitVersion, + std::map>* storageCache, std::map* tag_popped, + bool initialCommit) { //std::map> cacheRangeInfo; std::map cachedRangeInfo; for (auto const& m : mutations) { @@ -175,7 +180,7 @@ void applyMetadataMutations(UID const& dbgid, Arena &arena, VectorRefset(KeyValueRef(m.param1, m.param2)); @@ -293,7 +298,7 @@ void applyMetadataMutations(UID const& dbgid, Arena &arena, VectorRef(m.param2, Unversioned()); TraceEvent("MinRequiredCommitVersion", dbgid).detail("Min", requested).detail("Current", popVersion).detail("HasConf", !!confChange); if(!initialCommit) txnStateStore->set(KeyValueRef(m.param1, m.param2)); - if (confChange) *confChange = true; + confChange = true; TEST(true); // Recovering at a higher version. } } @@ -313,7 +318,7 @@ void applyMetadataMutations(UID const& dbgid, Arena &arena, VectorRefclear(range & configKeys); if(!excludedServersKeys.contains(range) && !failedServersKeys.contains(range)) { TraceEvent("MutationRequiresRestart", dbgid).detail("M", m.toString()); - if(confChange) *confChange = true; + confChange = true; } } if ( serverListKeys.intersects( range )) { @@ -329,11 +334,14 @@ void applyMetadataMutations(UID const& dbgid, Arena &arena, VectorRefreadRange( range & serverTagKeys ).get(); // read is expected to be immediately available for(auto &kv : serverKeysCleared) { Tag tag = decodeServerTagValue(kv.value); - TraceEvent("ServerTagRemove").detail("PopVersion", popVersion).detail("Tag", tag.toString()).detail("Server", decodeServerTagKey(kv.key)); - logSystem->pop( popVersion, decodeServerTagValue(kv.value) ); + TraceEvent("ServerTagRemove") + .detail("PopVersion", popVersion) + .detail("Tag", tag.toString()) + .detail("Server", decodeServerTagKey(kv.key)); + logSystem->pop(popVersion, decodeServerTagValue(kv.value)); (*tag_popped)[tag] = popVersion; - if(toCommit) { + if (toCommit) { MutationRef privatized = m; privatized.param1 = kv.key.withPrefix(systemKeys.begin, arena); privatized.param2 = keyAfter(kv.key, arena).withPrefix(systemKeys.begin, arena); @@ -536,3 +544,31 @@ void applyMetadataMutations(UID const& dbgid, Arena &arena, VectorRef logSystem, + const VectorRef& mutations, LogPushData* toCommit, bool& confChange, + Version popVersion, bool initialCommit) { + + std::map* uid_applyMutationsData = nullptr; + if (proxyCommitData.firstProxy) { + uid_applyMutationsData = &proxyCommitData.uid_applyMutationsData; + } + + applyMetadataMutations(proxyCommitData.dbgid, arena, mutations, proxyCommitData.txnStateStore, toCommit, confChange, + logSystem, popVersion, &proxyCommitData.vecBackupKeys, &proxyCommitData.keyInfo, + &proxyCommitData.cacheInfo, uid_applyMutationsData, proxyCommitData.commit, + proxyCommitData.cx, &proxyCommitData.committedVersion, &proxyCommitData.storageCache, + &proxyCommitData.tag_popped, initialCommit); +} + +void applyMetadataMutations(const UID& dbgid, Arena& arena, const VectorRef& mutations, + IKeyValueStore* txnStateStore) { + + bool confChange; // Dummy variable, not used. + + applyMetadataMutations(dbgid, arena, mutations, txnStateStore, /* toCommit= */ nullptr, confChange, + Reference(), /* popVersion= */ 0, /* vecBackupKeys= */ nullptr, + /* keyInfo= */ nullptr, /* cacheInfo= */ nullptr, /* uid_applyMutationsData= */ nullptr, + RequestStream(), Database(), /* commitVersion= */ nullptr, + /* storageCache= */ nullptr, /* tag_popped= */ nullptr, /* initialCommit= */ false); +} \ No newline at end of file diff --git a/fdbserver/ApplyMetadataMutation.h b/fdbserver/ApplyMetadataMutation.h index 5b0a9d3e99..52355f2c19 100644 --- a/fdbserver/ApplyMetadataMutation.h +++ b/fdbserver/ApplyMetadataMutation.h @@ -29,6 +29,7 @@ #include "fdbserver/IKeyValueStore.h" #include "fdbserver/LogSystem.h" #include "fdbserver/LogProtocolMessage.h" +#include "fdbserver/ProxyCommitData.actor.h" inline bool isMetadataMutation(MutationRef const& m) { // FIXME: This is conservative - not everything in system keyspace is necessarily processed by applyMetadataMutations @@ -36,16 +37,12 @@ inline bool isMetadataMutation(MutationRef const& m) { (m.type == MutationRef::ClearRange && m.param2.size() && m.param2[0] == systemKeys.begin[0] && !nonMetadataSystemKeys.contains(KeyRangeRef(m.param1, m.param2)) ); } -struct applyMutationsData { - Future worker; - Version endVersion; - Reference> keyVersion; -}; - Reference getStorageInfo(UID id, std::map>* storageCache, IKeyValueStore* txnStateStore); -void applyMetadataMutations(UID const& dbgid, Arena &arena, VectorRef const& mutations, IKeyValueStore* txnStateStore, LogPushData* toCommit, bool *confChange, Reference logSystem = Reference(), Version popVersion = 0, - KeyRangeMap >* vecBackupKeys = nullptr, KeyRangeMap* keyInfo = nullptr, KeyRangeMap* cacheInfo = nullptr, std::map* uid_applyMutationsData = nullptr, RequestStream commit = RequestStream(), - Database cx = Database(), NotifiedVersion* commitVersion = nullptr, std::map>* storageCache = nullptr, std::map* tag_popped = nullptr, bool initialCommit = false ); +void applyMetadataMutations(ProxyCommitData& proxyCommitData, Arena& arena, Reference logSystem, + const VectorRef& mutations, LogPushData* pToCommit, bool& confChange, + Version popVersion, bool initialCommit); +void applyMetadataMutations(const UID& dbgid, Arena& arena, const VectorRef& mutations, + IKeyValueStore* txnStateStore); #endif diff --git a/fdbserver/CMakeLists.txt b/fdbserver/CMakeLists.txt index 4e9babb95d..3f9e4f005f 100644 --- a/fdbserver/CMakeLists.txt +++ b/fdbserver/CMakeLists.txt @@ -57,6 +57,7 @@ set(FDBSERVER_SRCS OldTLogServer_6_0.actor.cpp OldTLogServer_6_2.actor.cpp Orderer.actor.h + ProxyCommitData.actor.h pubsub.actor.cpp pubsub.h QuietDatabase.actor.cpp diff --git a/fdbserver/MasterProxyServer.actor.cpp b/fdbserver/MasterProxyServer.actor.cpp index cc3d8d33e4..bbba0733fd 100644 --- a/fdbserver/MasterProxyServer.actor.cpp +++ b/fdbserver/MasterProxyServer.actor.cpp @@ -43,6 +43,7 @@ #include "fdbserver/LogSystemDiskQueueAdapter.h" #include "fdbserver/MasterInterface.h" #include "fdbserver/MutationTracking.h" +#include "fdbserver/ProxyCommitData.actor.h" #include "fdbserver/RecoveryState.h" #include "fdbserver/ServerDBInfo.h" #include "fdbserver/WaitFailure.h" @@ -78,89 +79,6 @@ ACTOR Future broadcastTxnRequest(TxnStateRequest req, int sendAmount, bool return Void(); } -struct ProxyStats { - CounterCollection cc; - Counter txnRequestIn, txnRequestOut, txnRequestErrors; - Counter txnStartIn, txnStartOut, txnStartBatch; - Counter txnSystemPriorityStartIn, txnSystemPriorityStartOut; - Counter txnBatchPriorityStartIn, txnBatchPriorityStartOut; - Counter txnDefaultPriorityStartIn, txnDefaultPriorityStartOut; - Counter txnCommitIn, txnCommitVersionAssigned, txnCommitResolving, txnCommitResolved, txnCommitOut, txnCommitOutSuccess, txnCommitErrors; - Counter txnConflicts; - Counter txnThrottled; - Counter commitBatchIn, commitBatchOut; - Counter mutationBytes; - Counter mutations; - Counter conflictRanges; - Counter keyServerLocationIn, keyServerLocationOut, keyServerLocationErrors; - Version lastCommitVersionAssigned; - - LatencySample commitLatencySample; - LatencySample grvLatencySample; - - LatencyBands commitLatencyBands; - LatencyBands grvLatencyBands; - - Future logger; - - int recentRequests; - Deque requestBuckets; - double lastBucketBegin; - double bucketInterval; - - void updateRequestBuckets() { - while(now() - lastBucketBegin > bucketInterval) { - lastBucketBegin += bucketInterval; - recentRequests -= requestBuckets.front(); - requestBuckets.pop_front(); - requestBuckets.push_back(0); - } - } - - void addRequest() { - updateRequestBuckets(); - ++recentRequests; - ++requestBuckets.back(); - } - - int getRecentRequests() { - updateRequestBuckets(); - return recentRequests*FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE/(FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE-(lastBucketBegin+bucketInterval-now())); - } - - explicit ProxyStats(UID id, Version* pVersion, NotifiedVersion* pCommittedVersion, int64_t *commitBatchesMemBytesCountPtr) - : cc("ProxyStats", id.toString()), recentRequests(0), lastBucketBegin(now()), bucketInterval(FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE/FLOW_KNOBS->BASIC_LOAD_BALANCE_BUCKETS), - txnRequestIn("TxnRequestIn", cc), txnRequestOut("TxnRequestOut", cc), - txnRequestErrors("TxnRequestErrors", cc), txnStartIn("TxnStartIn", cc), txnStartOut("TxnStartOut", cc), - txnStartBatch("TxnStartBatch", cc), txnSystemPriorityStartIn("TxnSystemPriorityStartIn", cc), - txnSystemPriorityStartOut("TxnSystemPriorityStartOut", cc), - txnBatchPriorityStartIn("TxnBatchPriorityStartIn", cc), - txnBatchPriorityStartOut("TxnBatchPriorityStartOut", cc), - txnDefaultPriorityStartIn("TxnDefaultPriorityStartIn", cc), - txnDefaultPriorityStartOut("TxnDefaultPriorityStartOut", cc), txnCommitIn("TxnCommitIn", cc), - txnCommitVersionAssigned("TxnCommitVersionAssigned", cc), txnCommitResolving("TxnCommitResolving", cc), - txnCommitResolved("TxnCommitResolved", cc), txnCommitOut("TxnCommitOut", cc), - txnCommitOutSuccess("TxnCommitOutSuccess", cc), txnCommitErrors("TxnCommitErrors", cc), - txnConflicts("TxnConflicts", cc), txnThrottled("TxnThrottled", cc), commitBatchIn("CommitBatchIn", cc), - commitBatchOut("CommitBatchOut", cc), mutationBytes("MutationBytes", cc), mutations("Mutations", cc), - conflictRanges("ConflictRanges", cc), keyServerLocationIn("KeyServerLocationIn", cc), - keyServerLocationOut("KeyServerLocationOut", cc), keyServerLocationErrors("KeyServerLocationErrors", cc), - lastCommitVersionAssigned(0), - commitLatencySample("CommitLatencyMetrics", id, SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL, SERVER_KNOBS->LATENCY_SAMPLE_SIZE), - grvLatencySample("GRVLatencyMetrics", id, SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL, SERVER_KNOBS->LATENCY_SAMPLE_SIZE), - commitLatencyBands("CommitLatencyMetrics", id, SERVER_KNOBS->STORAGE_LOGGING_DELAY), - grvLatencyBands("GRVLatencyMetrics", id, SERVER_KNOBS->STORAGE_LOGGING_DELAY) { - specialCounter(cc, "LastAssignedCommitVersion", [this](){return this->lastCommitVersionAssigned;}); - specialCounter(cc, "Version", [pVersion](){return *pVersion; }); - specialCounter(cc, "CommittedVersion", [pCommittedVersion](){ return pCommittedVersion->get(); }); - specialCounter(cc, "CommitBatchesMemBytesCount", [commitBatchesMemBytesCountPtr]() { return *commitBatchesMemBytesCountPtr; }); - logger = traceCounters("ProxyMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "ProxyMetrics"); - for(int i = 0; i < FLOW_KNOBS->BASIC_LOAD_BALANCE_BUCKETS; i++) { - requestBuckets.push_back(0); - } - } -}; - struct TransactionRateInfo { double rate; double limit; @@ -390,156 +308,6 @@ ACTOR void discardCommit(UID id, Future shardBegin; - Standalone shardEnd; - int64_t tag1; - int64_t tag2; - int64_t tag3; -}; - -struct ProxyCommitData { - UID dbgid; - int64_t commitBatchesMemBytesCount; - ProxyStats stats; - MasterInterface master; - vector resolvers; - LogSystemDiskQueueAdapter* logAdapter; - Reference logSystem; - IKeyValueStore* txnStateStore; - NotifiedVersion committedVersion; // Provided that this recovery has succeeded or will succeed, this version is fully committed (durable) - Version minKnownCommittedVersion; // No version smaller than this one will be used as the known committed version - // during recovery - Version version; // The version at which txnStateStore is up to date - Promise validState; // Set once txnStateStore and version are valid - double lastVersionTime; - KeyRangeMap> vecBackupKeys; - uint64_t commitVersionRequestNumber; - uint64_t mostRecentProcessedRequestNumber; - KeyRangeMap>> keyResolvers; - KeyRangeMap keyInfo; - KeyRangeMap cacheInfo; - std::map uid_applyMutationsData; - bool firstProxy; - double lastCoalesceTime; - bool locked; - Optional metadataVersion; - double commitBatchInterval; - - int64_t localCommitBatchesStarted; - NotifiedVersion latestLocalCommitBatchResolving; - NotifiedVersion latestLocalCommitBatchLogging; - - RequestStream getConsistentReadVersion; - RequestStream commit; - Database cx; - Reference> db; - EventMetricHandle singleKeyMutationEvent; - - std::map> storageCache; - std::map tag_popped; - Deque> txsPopVersions; - Version lastTxsPop; - bool popRemoteTxs; - vector> whitelistedBinPathVec; - - Optional latencyBandConfig; - double lastStartCommit; - double lastCommitLatency; - int updateCommitRequests = 0; - NotifiedDouble lastCommitTime; - - vector commitComputePerOperation; - TransactionTagMap transactionTagCommitCostEst; - - //The tag related to a storage server rarely change, so we keep a vector of tags for each key range to be slightly more CPU efficient. - //When a tag related to a storage server does change, we empty out all of these vectors to signify they must be repopulated. - //We do not repopulate them immediately to avoid a slow task. - const vector& tagsForKey(StringRef key) { - auto& tags = keyInfo[key].tags; - if(!tags.size()) { - auto& r = keyInfo.rangeContaining(key).value(); - for(auto info : r.src_info) { - r.tags.push_back(info->tag); - } - for(auto info : r.dest_info) { - r.tags.push_back(info->tag); - } - uniquify(r.tags); - return r.tags; - } - return tags; - } - - bool needsCacheTag(KeyRangeRef range) { - auto ranges = cacheInfo.intersectingRanges(range); - for(auto r : ranges) { - if(r.value()) { - return true; - } - } - return false; - } - - void updateLatencyBandConfig(Optional newLatencyBandConfig) { - if(newLatencyBandConfig.present() != latencyBandConfig.present() - || (newLatencyBandConfig.present() && newLatencyBandConfig.get().grvConfig != latencyBandConfig.get().grvConfig)) - { - TraceEvent("LatencyBandGrvUpdatingConfig").detail("Present", newLatencyBandConfig.present()); - stats.grvLatencyBands.clearBands(); - if(newLatencyBandConfig.present()) { - for(auto band : newLatencyBandConfig.get().grvConfig.bands) { - stats.grvLatencyBands.addThreshold(band); - } - } - } - - if(newLatencyBandConfig.present() != latencyBandConfig.present() - || (newLatencyBandConfig.present() && newLatencyBandConfig.get().commitConfig != latencyBandConfig.get().commitConfig)) - { - TraceEvent("LatencyBandCommitUpdatingConfig").detail("Present", newLatencyBandConfig.present()); - stats.commitLatencyBands.clearBands(); - if(newLatencyBandConfig.present()) { - for(auto band : newLatencyBandConfig.get().commitConfig.bands) { - stats.commitLatencyBands.addThreshold(band); - } - } - } - - latencyBandConfig = newLatencyBandConfig; - } - - ProxyCommitData(UID dbgid, MasterInterface master, RequestStream getConsistentReadVersion, Version recoveryTransactionVersion, RequestStream commit, Reference> db, bool firstProxy) - : dbgid(dbgid), stats(dbgid, &version, &committedVersion, &commitBatchesMemBytesCount), master(master), - logAdapter(NULL), txnStateStore(NULL), popRemoteTxs(false), - committedVersion(recoveryTransactionVersion), version(0), minKnownCommittedVersion(0), - lastVersionTime(0), commitVersionRequestNumber(1), mostRecentProcessedRequestNumber(0), - getConsistentReadVersion(getConsistentReadVersion), commit(commit), lastCoalesceTime(0), - localCommitBatchesStarted(0), locked(false), commitBatchInterval(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_MIN), - firstProxy(firstProxy), cx(openDBOnServer(db, TaskPriority::DefaultEndpoint, true, true)), db(db), - singleKeyMutationEvent(LiteralStringRef("SingleKeyMutation")), commitBatchesMemBytesCount(0), lastTxsPop(0), lastStartCommit(0), lastCommitLatency(SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION), lastCommitTime(0) - { - commitComputePerOperation.resize(SERVER_KNOBS->PROXY_COMPUTE_BUCKETS,0.0); - } -}; - -/// A handy applyMetadataMutations uses ProxyCommitData class -void applyMetadataMutations(ProxyCommitData& proxyCommitData, Arena& arena, Reference logSystem, - const VectorRef& mutations, LogPushData* pToCommit, bool& confChange, - Version popVersion, bool initialCommit) { - - std::map* pUidApplyMutationsData = nullptr; - if (proxyCommitData.firstProxy) { - pUidApplyMutationsData = &proxyCommitData.uid_applyMutationsData; - } - - applyMetadataMutations(proxyCommitData.dbgid, arena, mutations, proxyCommitData.txnStateStore, pToCommit, - &confChange, logSystem, popVersion, &proxyCommitData.vecBackupKeys, &proxyCommitData.keyInfo, - &proxyCommitData.cacheInfo, pUidApplyMutationsData, proxyCommitData.commit, - proxyCommitData.cx, &proxyCommitData.committedVersion, &proxyCommitData.storageCache, - &proxyCommitData.tag_popped, initialCommit); -} - struct ResolutionRequestBuilder { ProxyCommitData* self; vector requests; diff --git a/fdbserver/ProxyCommitData.actor.h b/fdbserver/ProxyCommitData.actor.h new file mode 100644 index 0000000000..2017eec406 --- /dev/null +++ b/fdbserver/ProxyCommitData.actor.h @@ -0,0 +1,269 @@ +/* + * ProxyCommitData.h + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2013-2020 Apple Inc. and the FoundationDB project authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once +#if defined(NO_INTELLISENSE) && !defined(FDBSERVER_PROXYCOMMITDATA_ACTOR_G_H) +#define FDBSERVER_PROXYCOMMITDATA_ACTOR_G_H +#include "fdbserver/ProxyCommitData.actor.g.h" +#elif !defined(FDBSERVER_PROXYCOMMITDATA_ACTOR_H) +#define FDBSERVER_PROXYCOMMITDATA_ACTOR_H + +#include "fdbclient/FDBTypes.h" +#include "fdbrpc/Stats.h" +#include "fdbserver/Knobs.h" +#include "fdbserver/LogSystemDiskQueueAdapter.h" +#include "flow/IRandom.h" + +#include "flow/actorcompiler.h" // This must be the last #include. + +DESCR struct SingleKeyMutation { + Standalone shardBegin; + Standalone shardEnd; + int64_t tag1; + int64_t tag2; + int64_t tag3; +}; + +struct ApplyMutationsData { + Future worker; + Version endVersion; + Reference> keyVersion; +}; + +struct ProxyStats { + CounterCollection cc; + Counter txnRequestIn, txnRequestOut, txnRequestErrors; + Counter txnStartIn, txnStartOut, txnStartBatch; + Counter txnSystemPriorityStartIn, txnSystemPriorityStartOut; + Counter txnBatchPriorityStartIn, txnBatchPriorityStartOut; + Counter txnDefaultPriorityStartIn, txnDefaultPriorityStartOut; + Counter txnCommitIn, txnCommitVersionAssigned, txnCommitResolving, txnCommitResolved, txnCommitOut, + txnCommitOutSuccess, txnCommitErrors; + Counter txnConflicts; + Counter txnThrottled; + Counter commitBatchIn, commitBatchOut; + Counter mutationBytes; + Counter mutations; + Counter conflictRanges; + Counter keyServerLocationIn, keyServerLocationOut, keyServerLocationErrors; + Version lastCommitVersionAssigned; + + LatencySample commitLatencySample; + LatencySample grvLatencySample; + + LatencyBands commitLatencyBands; + LatencyBands grvLatencyBands; + + Future logger; + + int recentRequests; + Deque requestBuckets; + double lastBucketBegin; + double bucketInterval; + + void updateRequestBuckets() { + while (now() - lastBucketBegin > bucketInterval) { + lastBucketBegin += bucketInterval; + recentRequests -= requestBuckets.front(); + requestBuckets.pop_front(); + requestBuckets.push_back(0); + } + } + + void addRequest() { + updateRequestBuckets(); + ++recentRequests; + ++requestBuckets.back(); + } + + int getRecentRequests() { + updateRequestBuckets(); + return recentRequests * FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE / + (FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE - (lastBucketBegin + bucketInterval - now())); + } + + explicit ProxyStats(UID id, Version* pVersion, NotifiedVersion* pCommittedVersion, + int64_t* commitBatchesMemBytesCountPtr) + : cc("ProxyStats", id.toString()), recentRequests(0), lastBucketBegin(now()), + bucketInterval(FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE / FLOW_KNOBS->BASIC_LOAD_BALANCE_BUCKETS), + txnRequestIn("TxnRequestIn", cc), txnRequestOut("TxnRequestOut", cc), txnRequestErrors("TxnRequestErrors", cc), + txnStartIn("TxnStartIn", cc), txnStartOut("TxnStartOut", cc), txnStartBatch("TxnStartBatch", cc), + txnSystemPriorityStartIn("TxnSystemPriorityStartIn", cc), + txnSystemPriorityStartOut("TxnSystemPriorityStartOut", cc), + txnBatchPriorityStartIn("TxnBatchPriorityStartIn", cc), + txnBatchPriorityStartOut("TxnBatchPriorityStartOut", cc), + txnDefaultPriorityStartIn("TxnDefaultPriorityStartIn", cc), + txnDefaultPriorityStartOut("TxnDefaultPriorityStartOut", cc), txnCommitIn("TxnCommitIn", cc), + txnCommitVersionAssigned("TxnCommitVersionAssigned", cc), txnCommitResolving("TxnCommitResolving", cc), + txnCommitResolved("TxnCommitResolved", cc), txnCommitOut("TxnCommitOut", cc), + txnCommitOutSuccess("TxnCommitOutSuccess", cc), txnCommitErrors("TxnCommitErrors", cc), + txnConflicts("TxnConflicts", cc), txnThrottled("TxnThrottled", cc), commitBatchIn("CommitBatchIn", cc), + commitBatchOut("CommitBatchOut", cc), mutationBytes("MutationBytes", cc), mutations("Mutations", cc), + conflictRanges("ConflictRanges", cc), keyServerLocationIn("KeyServerLocationIn", cc), + keyServerLocationOut("KeyServerLocationOut", cc), keyServerLocationErrors("KeyServerLocationErrors", cc), + lastCommitVersionAssigned(0), + commitLatencySample("CommitLatencyMetrics", id, SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL, + SERVER_KNOBS->LATENCY_SAMPLE_SIZE), + grvLatencySample("GRVLatencyMetrics", id, SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL, + SERVER_KNOBS->LATENCY_SAMPLE_SIZE), + commitLatencyBands("CommitLatencyMetrics", id, SERVER_KNOBS->STORAGE_LOGGING_DELAY), + grvLatencyBands("GRVLatencyMetrics", id, SERVER_KNOBS->STORAGE_LOGGING_DELAY) { + specialCounter(cc, "LastAssignedCommitVersion", [this]() { return this->lastCommitVersionAssigned; }); + specialCounter(cc, "Version", [pVersion]() { return *pVersion; }); + specialCounter(cc, "CommittedVersion", [pCommittedVersion]() { return pCommittedVersion->get(); }); + specialCounter(cc, "CommitBatchesMemBytesCount", + [commitBatchesMemBytesCountPtr]() { return *commitBatchesMemBytesCountPtr; }); + logger = traceCounters("ProxyMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "ProxyMetrics"); + for (int i = 0; i < FLOW_KNOBS->BASIC_LOAD_BALANCE_BUCKETS; i++) { + requestBuckets.push_back(0); + } + } +}; + +struct ProxyCommitData { + UID dbgid; + int64_t commitBatchesMemBytesCount; + ProxyStats stats; + MasterInterface master; + vector resolvers; + LogSystemDiskQueueAdapter* logAdapter; + Reference logSystem; + IKeyValueStore* txnStateStore; + NotifiedVersion committedVersion; // Provided that this recovery has succeeded or will succeed, this version is + // fully committed (durable) + Version minKnownCommittedVersion; // No version smaller than this one will be used as the known committed version + // during recovery + Version version; // The version at which txnStateStore is up to date + Promise validState; // Set once txnStateStore and version are valid + double lastVersionTime; + KeyRangeMap> vecBackupKeys; + uint64_t commitVersionRequestNumber; + uint64_t mostRecentProcessedRequestNumber; + KeyRangeMap>> keyResolvers; + KeyRangeMap keyInfo; + KeyRangeMap cacheInfo; + std::map uid_applyMutationsData; + bool firstProxy; + double lastCoalesceTime; + bool locked; + Optional metadataVersion; + double commitBatchInterval; + + int64_t localCommitBatchesStarted; + NotifiedVersion latestLocalCommitBatchResolving; + NotifiedVersion latestLocalCommitBatchLogging; + + RequestStream getConsistentReadVersion; + RequestStream commit; + Database cx; + Reference> db; + EventMetricHandle singleKeyMutationEvent; + + std::map> storageCache; + std::map tag_popped; + Deque> txsPopVersions; + Version lastTxsPop; + bool popRemoteTxs; + vector> whitelistedBinPathVec; + + Optional latencyBandConfig; + double lastStartCommit; + double lastCommitLatency; + int updateCommitRequests = 0; + NotifiedDouble lastCommitTime; + + vector commitComputePerOperation; + TransactionTagMap transactionTagCommitCostEst; + + // The tag related to a storage server rarely change, so we keep a vector of tags for each key range to be slightly + // more CPU efficient. When a tag related to a storage server does change, we empty out all of these vectors to + // signify they must be repopulated. We do not repopulate them immediately to avoid a slow task. + const vector& tagsForKey(StringRef key) { + auto& tags = keyInfo[key].tags; + if (!tags.size()) { + auto& r = keyInfo.rangeContaining(key).value(); + for (auto info : r.src_info) { + r.tags.push_back(info->tag); + } + for (auto info : r.dest_info) { + r.tags.push_back(info->tag); + } + uniquify(r.tags); + return r.tags; + } + return tags; + } + + bool needsCacheTag(KeyRangeRef range) { + auto ranges = cacheInfo.intersectingRanges(range); + for (auto r : ranges) { + if (r.value()) { + return true; + } + } + return false; + } + + void updateLatencyBandConfig(Optional newLatencyBandConfig) { + if (newLatencyBandConfig.present() != latencyBandConfig.present() || + (newLatencyBandConfig.present() && + newLatencyBandConfig.get().grvConfig != latencyBandConfig.get().grvConfig)) { + TraceEvent("LatencyBandGrvUpdatingConfig").detail("Present", newLatencyBandConfig.present()); + stats.grvLatencyBands.clearBands(); + if (newLatencyBandConfig.present()) { + for (auto band : newLatencyBandConfig.get().grvConfig.bands) { + stats.grvLatencyBands.addThreshold(band); + } + } + } + + if (newLatencyBandConfig.present() != latencyBandConfig.present() || + (newLatencyBandConfig.present() && + newLatencyBandConfig.get().commitConfig != latencyBandConfig.get().commitConfig)) { + TraceEvent("LatencyBandCommitUpdatingConfig").detail("Present", newLatencyBandConfig.present()); + stats.commitLatencyBands.clearBands(); + if (newLatencyBandConfig.present()) { + for (auto band : newLatencyBandConfig.get().commitConfig.bands) { + stats.commitLatencyBands.addThreshold(band); + } + } + } + + latencyBandConfig = newLatencyBandConfig; + } + + ProxyCommitData(UID dbgid, MasterInterface master, RequestStream getConsistentReadVersion, + Version recoveryTransactionVersion, RequestStream commit, + Reference> db, bool firstProxy) + : dbgid(dbgid), stats(dbgid, &version, &committedVersion, &commitBatchesMemBytesCount), master(master), + logAdapter(NULL), txnStateStore(NULL), popRemoteTxs(false), committedVersion(recoveryTransactionVersion), + version(0), minKnownCommittedVersion(0), lastVersionTime(0), commitVersionRequestNumber(1), + mostRecentProcessedRequestNumber(0), getConsistentReadVersion(getConsistentReadVersion), commit(commit), + lastCoalesceTime(0), localCommitBatchesStarted(0), locked(false), + commitBatchInterval(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_MIN), firstProxy(firstProxy), + cx(openDBOnServer(db, TaskPriority::DefaultEndpoint, true, true)), db(db), + singleKeyMutationEvent(LiteralStringRef("SingleKeyMutation")), commitBatchesMemBytesCount(0), lastTxsPop(0), + lastStartCommit(0), lastCommitLatency(SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION), lastCommitTime(0) { + commitComputePerOperation.resize(SERVER_KNOBS->PROXY_COMPUTE_BUCKETS, 0.0); + } +}; + +#include "flow/unactorcompiler.h" +#endif // FDBSERVER_PROXYCOMMITDATA_H \ No newline at end of file diff --git a/fdbserver/masterserver.actor.cpp b/fdbserver/masterserver.actor.cpp index 08b7a50115..4f21fa4f0c 100644 --- a/fdbserver/masterserver.actor.cpp +++ b/fdbserver/masterserver.actor.cpp @@ -18,31 +18,34 @@ * limitations under the License. */ -#include "flow/ActorCollection.h" -#include "fdbrpc/PerfMetric.h" -#include "flow/Trace.h" -#include "fdbrpc/FailureMonitor.h" +#include + #include "fdbclient/NativeAPI.actor.h" #include "fdbclient/Notified.h" #include "fdbclient/SystemData.h" -#include "fdbserver/ConflictSet.h" -#include "fdbserver/DataDistribution.actor.h" -#include "fdbserver/Knobs.h" -#include +#include "fdbrpc/FailureMonitor.h" +#include "fdbrpc/PerfMetric.h" +#include "fdbrpc/sim_validation.h" +#include "fdbserver/ApplyMetadataMutation.h" #include "fdbserver/BackupProgress.actor.h" -#include "fdbserver/MasterInterface.h" -#include "fdbserver/WaitFailure.h" -#include "fdbserver/WorkerInterface.actor.h" -#include "fdbserver/ServerDBInfo.h" +#include "fdbserver/ConflictSet.h" #include "fdbserver/CoordinatedState.h" #include "fdbserver/CoordinationInterface.h" // copy constructors for ServerCoordinators class -#include "fdbrpc/sim_validation.h" #include "fdbserver/DBCoreState.h" +#include "fdbserver/DataDistribution.actor.h" +#include "fdbserver/IKeyValueStore.h" +#include "fdbserver/Knobs.h" #include "fdbserver/LogSystem.h" #include "fdbserver/LogSystemDiskQueueAdapter.h" -#include "fdbserver/IKeyValueStore.h" -#include "fdbserver/ApplyMetadataMutation.h" +#include "fdbserver/MasterInterface.h" +#include "fdbserver/ProxyCommitData.actor.h" #include "fdbserver/RecoveryState.h" +#include "fdbserver/ServerDBInfo.h" +#include "fdbserver/WaitFailure.h" +#include "fdbserver/WorkerInterface.actor.h" +#include "flow/ActorCollection.h" +#include "flow/Trace.h" + #include "flow/actorcompiler.h" // This must be the last #include. using std::vector; @@ -1561,7 +1564,8 @@ ACTOR Future masterCore( Reference self ) { } } - applyMetadataMutations(self->dbgid, recoveryCommitRequest.arena, tr.mutations.slice(mmApplied, tr.mutations.size()), self->txnStateStore, nullptr, nullptr); + applyMetadataMutations(self->dbgid, recoveryCommitRequest.arena, tr.mutations.slice(mmApplied, tr.mutations.size()), + self->txnStateStore); mmApplied = tr.mutations.size(); tr.read_snapshot = self->recoveryTransactionVersion; // lastEpochEnd would make more sense, but isn't in the initial window of the resolver(s) From b9f41f397570bd1a593f6b579db709bd117c8eba Mon Sep 17 00:00:00 2001 From: Xiaoge Su Date: Wed, 5 Aug 2020 14:11:43 -0700 Subject: [PATCH 8/8] Rename CommitBatch::Context to CommitBatch::CommitBatchContext --- fdbserver/MasterProxyServer.actor.cpp | 53 ++++++++++++--------------- 1 file changed, 24 insertions(+), 29 deletions(-) diff --git a/fdbserver/MasterProxyServer.actor.cpp b/fdbserver/MasterProxyServer.actor.cpp index bbba0733fd..b473d4dcc2 100644 --- a/fdbserver/MasterProxyServer.actor.cpp +++ b/fdbserver/MasterProxyServer.actor.cpp @@ -595,7 +595,7 @@ ACTOR Future releaseResolvingAfter(ProxyCommitData* self, Future rel namespace CommitBatch { -struct Context { +struct CommitBatchContext { using StoreCommit_t = std::vector, Future>>; ProxyCommitData* const pProxyCommitData; @@ -673,10 +673,7 @@ struct Context { double commitStartTime; - Context( - ProxyCommitData*, - const std::vector*, - const int); + CommitBatchContext(ProxyCommitData*, const std::vector*, const int); void setupTraceBatch(); @@ -684,21 +681,19 @@ private: void evaluateBatchSize(); }; -Context::Context( - ProxyCommitData* const pProxyCommitData_, - const std::vector* trs_, - const int currentBatchMemBytesCount) : +CommitBatchContext::CommitBatchContext(ProxyCommitData* const pProxyCommitData_, + const std::vector* trs_, + const int currentBatchMemBytesCount) + : - pProxyCommitData(pProxyCommitData_), - trs(std::move(*const_cast*>(trs_))), - currentBatchMemBytesCount(currentBatchMemBytesCount), + pProxyCommitData(pProxyCommitData_), trs(std::move(*const_cast*>(trs_))), + currentBatchMemBytesCount(currentBatchMemBytesCount), - startTime(g_network->now()), + startTime(g_network->now()), - localBatchNumber(++pProxyCommitData->localCommitBatchesStarted), - toCommit(pProxyCommitData->logSystem), - - committed(trs.size()) { + localBatchNumber(++pProxyCommitData->localCommitBatchesStarted), toCommit(pProxyCommitData->logSystem), + + committed(trs.size()) { evaluateBatchSize(); @@ -717,7 +712,7 @@ Context::Context( ASSERT(SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS <= SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT); } -void Context::setupTraceBatch() { +void CommitBatchContext::setupTraceBatch() { for (const auto& tr : trs) { if (tr.debugID.present()) { if (!debugID.present()) { @@ -742,7 +737,7 @@ void Context::setupTraceBatch() { } } -void Context::evaluateBatchSize() { +void CommitBatchContext::evaluateBatchSize() { for (const auto& tr : trs) { const auto& mutations = tr.transaction.mutations; batchOperations += mutations.size(); @@ -750,7 +745,7 @@ void Context::evaluateBatchSize() { } } -ACTOR Future preresolutionProcessing(Context* self) { +ACTOR Future preresolutionProcessing(CommitBatchContext* self) { state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData; state std::vector& trs = self->trs; @@ -808,7 +803,7 @@ ACTOR Future preresolutionProcessing(Context* self) { return Void(); } -ACTOR Future getResolution(Context* self) { +ACTOR Future getResolution(CommitBatchContext* self) { // Sending these requests is the fuzzy border between phase 1 and phase 2; it could conceivably overlap with // resolution processing but is still using CPU ProxyCommitData* pProxyCommitData = self->pProxyCommitData; @@ -882,7 +877,7 @@ void assertResolutionStateMutationsSizeConsistent( } // Compute and apply "metadata" effects of each other proxy's most recent batch -void applyMetadataEffect(Context* self) { +void applyMetadataEffect(CommitBatchContext* self) { bool initialState = self->isMyFirstBatch; self->firstStateMutations = self->isMyFirstBatch; for (int versionIndex = 0; versionIndex < self->resolution[0].stateMutations.size(); versionIndex++) { @@ -925,7 +920,7 @@ void applyMetadataEffect(Context* self) { } /// Determine which transactions actually committed (conservatively) by combining results from the resolvers -void determineCommittedTransactions(Context* self) { +void determineCommittedTransactions(CommitBatchContext* self) { auto pProxyCommitData = self->pProxyCommitData; const auto& trs = self->trs; @@ -968,7 +963,7 @@ void determineCommittedTransactions(Context* self) { } // This first pass through committed transactions deals with "metadata" effects (modifications of txnStateStore, changes to storage servers' responsibilities) -ACTOR Future applyMetadataToCommittedTransactions(Context* self) { +ACTOR Future applyMetadataToCommittedTransactions(CommitBatchContext* self) { auto pProxyCommitData = self->pProxyCommitData; const auto& trs = self->trs; @@ -1012,7 +1007,7 @@ ACTOR Future applyMetadataToCommittedTransactions(Context* self) { } /// This second pass through committed transactions assigns the actual mutations to the appropriate storage servers' tags -ACTOR Future assignMutationsToStorageServers(Context* self) { +ACTOR Future assignMutationsToStorageServers(CommitBatchContext* self) { state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData; state std::vector& trs = self->trs; @@ -1131,7 +1126,7 @@ ACTOR Future assignMutationsToStorageServers(Context* self) { return Void(); } -ACTOR Future postResolution(Context* self) { +ACTOR Future postResolution(CommitBatchContext* self) { state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData; state std::vector& trs = self->trs; state const int64_t localBatchNumber = self->localBatchNumber; @@ -1258,7 +1253,7 @@ ACTOR Future postResolution(Context* self) { return Void(); } -ACTOR Future transactionLogging(Context* self) { +ACTOR Future transactionLogging(CommitBatchContext* self) { state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData; try { @@ -1293,7 +1288,7 @@ ACTOR Future transactionLogging(Context* self) { return Void(); } -ACTOR Future reply(Context* self) { +ACTOR Future reply(CommitBatchContext* self) { state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData; const Optional& debugID = self->debugID; @@ -1438,7 +1433,7 @@ ACTOR Future commitBatch( vector* trs, int currentBatchMemBytesCount) { //WARNING: this code is run at a high priority (until the first delay(0)), so it needs to do as little work as possible - state CommitBatch::Context context(self, trs, currentBatchMemBytesCount); + state CommitBatch::CommitBatchContext context(self, trs, currentBatchMemBytesCount); // Active load balancing runs at a very high priority (to obtain accurate estimate of memory used by commit batches) so we need to downgrade here wait(delay(0, TaskPriority::ProxyCommit));