foundationdb/fdbserver/CommitProxyServer.actor.cpp

1839 lines
76 KiB
C++
Raw Normal View History

2017-05-26 04:48:44 +08:00
/*
2020-09-11 08:44:15 +08:00
* CommitProxyServer.actor.cpp
2017-05-26 04:48:44 +08:00
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2018 Apple Inc. and the FoundationDB project authors
*
2017-05-26 04:48:44 +08:00
* 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
*
2017-05-26 04:48:44 +08:00
* http://www.apache.org/licenses/LICENSE-2.0
*
2017-05-26 04:48:44 +08:00
* 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.
*/
2020-07-26 12:44:10 +08:00
#include <algorithm>
2020-07-23 07:08:49 +08:00
#include <tuple>
2020-07-26 12:44:10 +08:00
2020-07-18 13:24:52 +08:00
#include <fdbclient/DatabaseContext.h>
2019-07-20 02:24:26 +08:00
#include "fdbclient/Atomic.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/Knobs.h"
2020-09-11 08:44:15 +08:00
#include "fdbclient/CommitProxyInterface.h"
#include "fdbclient/NativeAPI.actor.h"
#include "fdbclient/SystemData.h"
2019-07-20 02:24:26 +08:00
#include "fdbrpc/sim_validation.h"
#include "fdbserver/ApplyMetadataMutation.h"
2019-07-20 02:24:26 +08:00
#include "fdbserver/ConflictSet.h"
2019-06-20 02:11:50 +08:00
#include "fdbserver/DataDistributorInterface.h"
2019-07-20 02:24:26 +08:00
#include "fdbserver/FDBExecHelper.actor.h"
#include "fdbserver/IKeyValueStore.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/LogSystem.h"
#include "fdbserver/LogSystemDiskQueueAdapter.h"
#include "fdbserver/MasterInterface.h"
#include "fdbserver/MutationTracking.h"
#include "fdbserver/ProxyCommitData.actor.h"
#include "fdbserver/RatekeeperInterface.h"
2019-07-20 02:24:26 +08:00
#include "fdbserver/RecoveryState.h"
#include "fdbserver/WaitFailure.h"
#include "fdbserver/WorkerInterface.actor.h"
#include "flow/ActorCollection.h"
#include "flow/IRandom.h"
2019-07-20 02:24:26 +08:00
#include "flow/Knobs.h"
#include "flow/Trace.h"
#include "flow/Tracing.h"
2020-07-26 12:44:10 +08:00
#include "flow/actorcompiler.h" // This must be the last #include.
2017-05-26 04:48:44 +08:00
ACTOR Future<Void> broadcastTxnRequest(TxnStateRequest req, int sendAmount, bool sendReply) {
state ReplyPromise<Void> reply = req.reply;
resetReply( req );
std::vector<Future<Void>> replies;
int currentStream = 0;
2020-04-11 04:45:16 +08:00
std::vector<Endpoint> broadcastEndpoints = req.broadcastInfo;
for(int i = 0; i < sendAmount && currentStream < broadcastEndpoints.size(); i++) {
2020-04-07 11:58:43 +08:00
std::vector<Endpoint> endpoints;
2020-04-11 04:45:16 +08:00
RequestStream<TxnStateRequest> cur(broadcastEndpoints[currentStream++]);
while(currentStream < broadcastEndpoints.size()*(i+1)/sendAmount) {
endpoints.push_back(broadcastEndpoints[currentStream++]);
}
2020-04-07 11:58:43 +08:00
req.broadcastInfo = endpoints;
replies.push_back(brokenPromiseToNever( cur.getReply( req ) ));
resetReply( req );
}
wait( waitForAll(replies) );
if(sendReply) {
reply.send(Void());
}
return Void();
}
2017-05-26 04:48:44 +08:00
ACTOR void discardCommit(UID id, Future<LogSystemDiskQueueAdapter::CommitMessage> fcm, Future<Void> dummyCommitState) {
ASSERT(!dummyCommitState.isReady());
LogSystemDiskQueueAdapter::CommitMessage cm = wait(fcm);
TraceEvent("Discarding", id).detail("Count", cm.messages.size());
2017-05-26 04:48:44 +08:00
cm.acknowledge.send(Void());
ASSERT(dummyCommitState.isReady());
}
struct ResolutionRequestBuilder {
ProxyCommitData* self;
vector<ResolveTransactionBatchRequest> requests;
vector<vector<int>> transactionResolverMap;
vector<CommitTransactionRef*> outTr;
2020-05-22 20:24:15 +08:00
std::vector<std::vector<std::vector<int>>>
txReadConflictRangeIndexMap; // Used to report conflicting keys, the format is
// [CommitTransactionRef_Index][Resolver_Index][Read_Conflict_Range_Index_on_Resolver]
// -> read_conflict_range's original index in the commitTransactionRef
2017-05-26 04:48:44 +08:00
ResolutionRequestBuilder(ProxyCommitData* self, Version version, Version prevVersion, Version lastReceivedVersion,
Span& parentSpan)
: self(self), requests(self->resolvers.size()) {
for (auto& req : requests) {
2020-07-10 01:49:33 +08:00
req.spanContext = parentSpan.context;
2017-05-26 04:48:44 +08:00
req.prevVersion = prevVersion;
req.version = version;
req.lastReceivedVersion = lastReceivedVersion;
}
}
CommitTransactionRef& getOutTransaction(int resolver, Version read_snapshot) {
CommitTransactionRef *& out = outTr[resolver];
if (!out) {
ResolveTransactionBatchRequest& request = requests[resolver];
request.transactions.resize(request.arena, request.transactions.size() + 1);
out = &request.transactions.back();
out->read_snapshot = read_snapshot;
}
return *out;
}
void addTransaction(CommitTransactionRequest& trRequest, int transactionNumberInBatch) {
auto& trIn = trRequest.transaction;
2017-05-26 04:48:44 +08:00
// SOMEDAY: There are a couple of unnecessary O( # resolvers ) steps here
2020-08-19 05:18:50 +08:00
outTr.assign(requests.size(), nullptr);
2017-05-26 04:48:44 +08:00
ASSERT( transactionNumberInBatch >= 0 && transactionNumberInBatch < 32768 );
bool isTXNStateTransaction = false;
for (auto & m : trIn.mutations) {
if (m.type == MutationRef::SetVersionstampedKey) {
transformVersionstampMutation( m, &MutationRef::param1, requests[0].version, transactionNumberInBatch );
2017-05-26 04:48:44 +08:00
trIn.write_conflict_ranges.push_back( requests[0].arena, singleKeyRange( m.param1, requests[0].arena ) );
} else if (m.type == MutationRef::SetVersionstampedValue) {
transformVersionstampMutation( m, &MutationRef::param2, requests[0].version, transactionNumberInBatch );
2017-05-26 04:48:44 +08:00
}
if (isMetadataMutation(m)) {
isTXNStateTransaction = true;
getOutTransaction(0, trIn.read_snapshot).mutations.push_back(requests[0].arena, m);
}
}
if (isTXNStateTransaction && !trRequest.isLockAware()) {
// This mitigates https://github.com/apple/foundationdb/issues/3647. Since this transaction is not lock
// aware, if this transaction got a read version then \xff/dbLocked must not have been set at this
// transaction's read snapshot. If that changes by commit time, then it won't commit on any proxy because of
// a conflict. A client could set a read version manually so this isn't totally bulletproof.
trIn.read_conflict_ranges.push_back(trRequest.arena, KeyRangeRef(databaseLockedKey, databaseLockedKeyEnd));
}
2020-05-22 20:24:15 +08:00
std::vector<std::vector<int>> rCRIndexMap(
requests.size()); // [resolver_index][read_conflict_range_index_on_the_resolver]
// -> read_conflict_range's original index
for (int idx = 0; idx < trIn.read_conflict_ranges.size(); ++idx) {
const auto& r = trIn.read_conflict_ranges[idx];
2017-05-26 04:48:44 +08:00
auto ranges = self->keyResolvers.intersectingRanges( r );
std::set<int> resolvers;
for(auto &ir : ranges) {
auto& version_resolver = ir.value();
for(int i = version_resolver.size()-1; i >= 0; i--) {
resolvers.insert(version_resolver[i].second);
if( version_resolver[i].first < trIn.read_snapshot )
break;
}
}
ASSERT(resolvers.size());
2020-05-22 20:24:15 +08:00
for (int resolver : resolvers) {
2017-05-26 04:48:44 +08:00
getOutTransaction( resolver, trIn.read_snapshot ).read_conflict_ranges.push_back( requests[resolver].arena, r );
rCRIndexMap[resolver].push_back(idx);
}
2017-05-26 04:48:44 +08:00
}
txReadConflictRangeIndexMap.push_back(std::move(rCRIndexMap));
2017-05-26 04:48:44 +08:00
for(auto& r : trIn.write_conflict_ranges) {
auto ranges = self->keyResolvers.intersectingRanges( r );
std::set<int> resolvers;
for(auto &ir : ranges)
resolvers.insert(ir.value().back().second);
ASSERT(resolvers.size());
for(int resolver : resolvers)
getOutTransaction( resolver, trIn.read_snapshot ).write_conflict_ranges.push_back( requests[resolver].arena, r );
}
if (isTXNStateTransaction)
for (int r = 0; r<requests.size(); r++) {
int transactionNumberInRequest = &getOutTransaction(r, trIn.read_snapshot) - requests[r].transactions.begin();
requests[r].txnStateTransactions.push_back(requests[r].arena, transactionNumberInRequest);
}
vector<int> resolversUsed;
for (int r = 0; r<outTr.size(); r++)
if (outTr[r]) {
2017-05-26 04:48:44 +08:00
resolversUsed.push_back(r);
outTr[r]->report_conflicting_keys = trIn.report_conflicting_keys;
}
transactionResolverMap.emplace_back(std::move(resolversUsed));
2017-05-26 04:48:44 +08:00
}
};
ACTOR Future<Void> commitBatcher(ProxyCommitData *commitData, PromiseStream<std::pair<std::vector<CommitTransactionRequest>, int> > out, FutureStream<CommitTransactionRequest> in, int desiredBytes, int64_t memBytesLimit) {
wait(delayJittered(commitData->commitBatchInterval, TaskPriority::ProxyCommitBatcher));
state double lastBatch = 0;
loop{
state Future<Void> timeout;
state std::vector<CommitTransactionRequest> batch;
state int batchBytes = 0;
if(SERVER_KNOBS->MAX_COMMIT_BATCH_INTERVAL <= 0) {
timeout = Never();
}
else {
timeout = delayJittered(SERVER_KNOBS->MAX_COMMIT_BATCH_INTERVAL, TaskPriority::ProxyCommitBatcher);
}
while(!timeout.isReady() && !(batch.size() == SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_COUNT_MAX || batchBytes >= desiredBytes)) {
choose{
when(CommitTransactionRequest req = waitNext(in)) {
//WARNING: this code is run at a high priority, so it needs to do as little work as possible
2018-11-10 06:25:40 +08:00
int bytes = getBytes(req);
2018-11-10 06:25:40 +08:00
// Drop requests if memory is under severe pressure
if(commitData->commitBatchesMemBytesCount + bytes > memBytesLimit) {
++commitData->stats.txnCommitErrors;
2018-11-10 06:25:40 +08:00
req.reply.sendError(proxy_memory_limit_exceeded());
TraceEvent(SevWarnAlways, "ProxyCommitBatchMemoryThresholdExceeded").suppressFor(60).detail("MemBytesCount", commitData->commitBatchesMemBytesCount).detail("MemLimit", memBytesLimit);
continue;
}
if (bytes > FLOW_KNOBS->PACKET_WARNING) {
TraceEvent(!g_network->isSimulated() ? SevWarnAlways : SevWarn, "LargeTransaction")
.suppressFor(1.0)
2019-07-20 02:24:26 +08:00
.detail("Size", bytes)
.detail("Client", req.reply.getEndpoint().getPrimaryAddress());
}
2018-11-10 06:25:40 +08:00
++commitData->stats.txnCommitIn;
2018-11-10 06:25:40 +08:00
if(req.debugID.present()) {
2020-09-11 08:44:15 +08:00
g_traceBatch.addEvent("CommitDebug", req.debugID.get().first(), "CommitProxyServer.batcher");
2018-11-10 06:25:40 +08:00
}
2018-11-10 06:25:40 +08:00
if(!batch.size()) {
if(now() - lastBatch > commitData->commitBatchInterval) {
timeout = delayJittered(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_FROM_IDLE, TaskPriority::ProxyCommitBatcher);
2018-11-10 06:25:40 +08:00
}
else {
timeout = delayJittered(commitData->commitBatchInterval - (now() - lastBatch), TaskPriority::ProxyCommitBatcher);
2018-11-10 06:25:40 +08:00
}
}
2018-11-10 06:25:40 +08:00
if((batchBytes + bytes > CLIENT_KNOBS->TRANSACTION_SIZE_LIMIT || req.firstInBatch()) && batch.size()) {
out.send({ std::move(batch), batchBytes });
2018-11-10 06:25:40 +08:00
lastBatch = now();
timeout = delayJittered(commitData->commitBatchInterval, TaskPriority::ProxyCommitBatcher);
batch.clear();
2018-11-10 06:25:40 +08:00
batchBytes = 0;
}
2018-11-10 06:25:40 +08:00
batch.push_back(req);
batchBytes += bytes;
commitData->commitBatchesMemBytesCount += bytes;
}
when(wait(timeout)) {}
}
}
out.send({ std::move(batch), batchBytes });
lastBatch = now();
}
}
void createWhitelistBinPathVec(const std::string& binPath, vector<Standalone<StringRef>>& binPathVec) {
TraceEvent(SevDebug, "BinPathConverter").detail("Input", binPath);
StringRef input(binPath);
while (input != StringRef()) {
StringRef token = input.eat(LiteralStringRef(","));
if (token != StringRef()) {
const uint8_t* ptr = token.begin();
while (ptr != token.end() && *ptr == ' ') {
ptr++;
}
if (ptr != token.end()) {
Standalone<StringRef> newElement(token.substr(ptr - token.begin()));
2019-04-23 21:55:55 +08:00
TraceEvent(SevDebug, "BinPathItem").detail("Element", newElement);
binPathVec.push_back(newElement);
}
}
}
return;
}
bool isWhitelisted(const vector<Standalone<StringRef>>& binPathVec, StringRef binPath) {
2019-04-23 21:55:55 +08:00
TraceEvent("BinPath").detail("Value", binPath);
for (const auto& item : binPathVec) {
2019-04-23 21:55:55 +08:00
TraceEvent("Element").detail("Value", item);
}
return std::find(binPathVec.begin(), binPathVec.end(), binPath) != binPathVec.end();
}
ACTOR Future<Void> addBackupMutations(ProxyCommitData* self, std::map<Key, MutationListRef>* logRangeMutations,
LogPushData* toCommit, Version commitVersion, double* computeDuration, double* computeStart) {
2019-10-22 02:43:46 +08:00
state std::map<Key, MutationListRef>::iterator logRangeMutation = logRangeMutations->begin();
state int32_t version = commitVersion / CLIENT_KNOBS->LOG_RANGE_BLOCK_SIZE;
state int yieldBytes = 0;
state BinaryWriter valueWriter(Unversioned());
// Serialize the log range mutations within the map
2019-10-22 02:43:46 +08:00
for (; logRangeMutation != logRangeMutations->end(); ++logRangeMutation)
{
//FIXME: this is re-implementing the serialize function of MutationListRef in order to have a yield
valueWriter = BinaryWriter(IncludeVersion(ProtocolVersion::withBackupMutations()));
valueWriter << logRangeMutation->second.totalSize();
state MutationListRef::Blob* blobIter = logRangeMutation->second.blob_begin;
while(blobIter) {
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();
}
}
valueWriter.serializeBytes(blobIter->data);
yieldBytes += blobIter->data.size();
blobIter = blobIter->next;
2019-10-22 02:43:46 +08:00
}
Key val = valueWriter.toValue();
2020-07-26 12:44:10 +08:00
BinaryWriter wr(Unversioned());
// Serialize the log destination
2019-10-22 02:43:46 +08:00
wr.serializeBytes( logRangeMutation->first );
// Write the log keys and version information
wr << (uint8_t)hashlittle(&version, sizeof(version), 0);
wr << bigEndian64(commitVersion);
2019-10-22 02:43:46 +08:00
MutationRef backupMutation;
backupMutation.type = MutationRef::SetValue;
2020-08-19 05:18:50 +08:00
uint32_t* partBuffer = nullptr;
for (int part = 0; part * CLIENT_KNOBS->MUTATION_BLOCK_SIZE < val.size(); part++) {
2019-10-22 02:43:46 +08:00
// Assign the second parameter as the part
2019-10-22 02:43:46 +08:00
backupMutation.param2 = val.substr(part * CLIENT_KNOBS->MUTATION_BLOCK_SIZE,
std::min(val.size() - part * CLIENT_KNOBS->MUTATION_BLOCK_SIZE, CLIENT_KNOBS->MUTATION_BLOCK_SIZE));
// Write the last part of the mutation to the serialization, if the buffer is not defined
if (!partBuffer) {
// Serialize the part to the writer
wr << bigEndian32(part);
// Define the last buffer part
2019-10-22 02:43:46 +08:00
partBuffer = (uint32_t*) ((char*) wr.getData() + wr.getLength() - sizeof(uint32_t));
}
else {
*partBuffer = bigEndian32(part);
}
// Define the mutation type and and location
backupMutation.param1 = wr.toValue();
2019-10-22 02:43:46 +08:00
ASSERT( backupMutation.param1.startsWith(logRangeMutation->first) ); // We are writing into the configured destination
2020-07-26 12:44:10 +08:00
auto& tags = self->tagsForKey(backupMutation.param1);
toCommit->addTags(tags);
toCommit->addTypedMessage(backupMutation);
// if (DEBUG_MUTATION("BackupProxyCommit", commitVersion, backupMutation)) {
2019-10-22 02:43:46 +08:00
// TraceEvent("BackupProxyCommitTo", self->dbgid).detail("To", describe(tags)).detail("BackupMutation", backupMutation.toString())
// .detail("BackupMutationSize", val.size()).detail("Version", commitVersion).detail("DestPath", logRangeMutation.first)
// .detail("PartIndex", part).detail("PartIndexEndian", bigEndian32(part)).detail("PartData", backupMutation.param1);
// }
}
}
2019-10-22 02:43:46 +08:00
return Void();
}
ACTOR Future<Void> releaseResolvingAfter(ProxyCommitData* self, Future<Void> releaseDelay, int64_t localBatchNumber) {
wait(releaseDelay);
ASSERT(self->latestLocalCommitBatchResolving.get() == localBatchNumber-1);
self->latestLocalCommitBatchResolving.set(localBatchNumber);
return Void();
}
2020-07-26 12:44:10 +08:00
namespace CommitBatch {
struct CommitBatchContext {
2020-08-04 12:19:14 +08:00
using StoreCommit_t = std::vector<std::pair<Future<LogSystemDiskQueueAdapter::CommitMessage>, Future<Void>>>;
2020-07-26 12:44:10 +08:00
ProxyCommitData* const pProxyCommitData;
std::vector<CommitTransactionRequest> trs;
int currentBatchMemBytesCount;
double startTime;
Optional<UID> debugID;
bool forceRecovery = false;
int64_t localBatchNumber;
LogPushData toCommit;
int batchOperations = 0;
2020-08-04 12:19:14 +08:00
Span span = Span("MP:commitBatch"_loc);
2020-07-26 12:44:10 +08:00
int64_t batchBytes = 0;
2017-05-26 04:48:44 +08:00
2020-07-26 12:44:10 +08:00
int latencyBucket = 0;
2017-05-26 04:48:44 +08:00
2020-07-26 12:44:10 +08:00
Version commitVersion;
Version prevVersion;
int64_t maxTransactionBytes;
std::vector<std::vector<int>> transactionResolverMap;
std::vector<std::vector<std::vector<int>>> txReadConflictRangeIndexMap;
2020-07-26 12:44:10 +08:00
Future<Void> releaseDelay;
Future<Void> releaseFuture;
2017-05-26 04:48:44 +08:00
2020-07-26 12:44:10 +08:00
std::vector<ResolveTransactionBatchReply> resolution;
2017-05-26 04:48:44 +08:00
2020-07-26 12:55:22 +08:00
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;
2020-07-26 12:55:22 +08:00
Optional<Value> oldCoordinators;
2017-05-26 04:48:44 +08:00
2020-07-26 12:55:22 +08:00
StoreCommit_t storeCommits;
2017-05-26 04:48:44 +08:00
2020-07-26 12:55:22 +08:00
std::vector<uint8_t> committed;
Optional<Key> lockedKey;
bool locked;
int commitCount = 0;
std::vector<int> nextTr;
bool lockedAfter;
Optional<Value> metadataVersionAfter;
int mutationCount = 0;
int mutationBytes = 0;
std::map<Key, MutationListRef> logRangeMutations;
Arena logRangeMutationsArena;
int transactionNum = 0;
int yieldBytes = 0;
LogSystemDiskQueueAdapter::CommitMessage msg;
Future<Version> loggingComplete;
double commitStartTime;
CommitBatchContext(ProxyCommitData*, const std::vector<CommitTransactionRequest>*, const int);
2020-07-26 12:44:10 +08:00
2020-08-04 12:19:14 +08:00
void setupTraceBatch();
2020-07-26 12:44:10 +08:00
private:
void evaluateBatchSize();
};
CommitBatchContext::CommitBatchContext(ProxyCommitData* const pProxyCommitData_,
const std::vector<CommitTransactionRequest>* trs_,
const int currentBatchMemBytesCount)
:
2017-05-26 04:48:44 +08:00
pProxyCommitData(pProxyCommitData_), trs(std::move(*const_cast<std::vector<CommitTransactionRequest>*>(trs_))),
currentBatchMemBytesCount(currentBatchMemBytesCount),
2017-05-26 04:48:44 +08:00
startTime(g_network->now()),
2020-07-26 12:44:10 +08:00
localBatchNumber(++pProxyCommitData->localCommitBatchesStarted), toCommit(pProxyCommitData->logSystem),
committed(trs.size()) {
2020-07-26 12:44:10 +08:00
evaluateBatchSize();
if (batchOperations != 0) {
latencyBucket = std::min<int>(
SERVER_KNOBS->PROXY_COMPUTE_BUCKETS - 1,
SERVER_KNOBS->PROXY_COMPUTE_BUCKETS * batchBytes /
(batchOperations * (
CLIENT_KNOBS->VALUE_SIZE_LIMIT +
CLIENT_KNOBS->KEY_SIZE_LIMIT
))
);
}
// 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);
}
void CommitBatchContext::setupTraceBatch() {
2020-08-04 12:19:14 +08:00
for (const auto& tr : trs) {
2020-07-26 12:44:10 +08:00
if (tr.debugID.present()) {
if (!debugID.present()) {
debugID = nondeterministicRandom()->randomUniqueID();
2020-07-26 12:44:10 +08:00
}
g_traceBatch.addAttach(
"CommitAttachID",
tr.debugID.get().first(),
debugID.get().first()
);
2017-05-26 04:48:44 +08:00
}
2020-07-26 12:44:10 +08:00
span.addParent(tr.spanContext);
2017-05-26 04:48:44 +08:00
}
2020-07-26 12:44:10 +08:00
if (debugID.present()) {
2020-09-11 08:44:15 +08:00
g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "CommitProxyServer.commitBatch.Before");
2017-05-26 04:48:44 +08:00
}
2020-07-26 12:44:10 +08:00
}
2017-05-26 04:48:44 +08:00
void CommitBatchContext::evaluateBatchSize() {
2020-08-04 12:19:14 +08:00
for (const auto& tr : trs) {
const auto& mutations = tr.transaction.mutations;
2020-07-26 12:44:10 +08:00
batchOperations += mutations.size();
batchBytes += mutations.expectedSize();
2017-05-26 04:48:44 +08:00
}
2020-07-26 12:44:10 +08:00
}
2017-05-26 04:48:44 +08:00
ACTOR Future<Void> preresolutionProcessing(CommitBatchContext* self) {
2020-07-26 12:44:10 +08:00
state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData;
state std::vector<CommitTransactionRequest>& trs = self->trs;
state const int64_t localBatchNumber = self->localBatchNumber;
state const int latencyBucket = self->latencyBucket;
state const Optional<UID>& debugID = self->debugID;
2017-05-26 04:48:44 +08:00
2020-07-26 12:44:10 +08:00
// 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
);
2017-05-26 04:48:44 +08:00
2020-07-26 12:44:10 +08:00
if (debugID.present()) {
2020-09-11 08:44:15 +08:00
g_traceBatch.addEvent("CommitDebug", debugID.get().first(),
"CommitProxyServer.commitBatch.GettingCommitVersion");
2020-07-26 12:44:10 +08:00
}
2020-08-04 12:19:14 +08:00
GetCommitVersionRequest req(self->span.context, pProxyCommitData->commitVersionRequestNumber++,
pProxyCommitData->mostRecentProcessedRequestNumber, pProxyCommitData->dbgid);
2020-07-26 12:44:10 +08:00
GetCommitVersionReply versionReply = wait(brokenPromiseToNever(
pProxyCommitData->master.getCommitVersion.getReply(
req, TaskPriority::ProxyMasterVersionReply
)
));
2017-05-26 04:48:44 +08:00
2020-07-26 12:44:10 +08:00
pProxyCommitData->mostRecentProcessedRequestNumber = versionReply.requestNum;
2017-05-26 04:48:44 +08:00
2020-07-26 12:44:10 +08:00
pProxyCommitData->stats.txnCommitVersionAssigned += trs.size();
pProxyCommitData->stats.lastCommitVersionAssigned = versionReply.version;
2017-05-26 04:48:44 +08:00
2020-07-26 12:44:10 +08:00
self->commitVersion = versionReply.version;
self->prevVersion = versionReply.prevVersion;
2017-05-26 04:48:44 +08:00
for(auto it : versionReply.resolverChanges) {
2020-07-26 12:44:10 +08:00
auto rs = pProxyCommitData->keyResolvers.modify(it.range);
2017-05-26 04:48:44 +08:00
for(auto r = rs.begin(); r != rs.end(); ++r)
r->value().emplace_back(versionReply.resolverChangesVersion,it.dest);
2017-05-26 04:48:44 +08:00
}
2020-07-26 12:44:10 +08:00
//TraceEvent("ProxyGotVer", pProxyContext->dbgid).detail("Commit", commitVersion).detail("Prev", prevVersion);
2017-05-26 04:48:44 +08:00
2020-07-26 12:44:10 +08:00
if (debugID.present()) {
2020-09-11 08:44:15 +08:00
g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "CommitProxyServer.commitBatch.GotCommitVersion");
2020-07-26 12:44:10 +08:00
}
2017-05-26 04:48:44 +08:00
2020-07-26 12:44:10 +08:00
return Void();
}
2017-05-26 04:48:44 +08:00
ACTOR Future<Void> getResolution(CommitBatchContext* self) {
2020-08-04 12:19:14 +08:00
// 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
2020-07-26 12:44:10 +08:00
ProxyCommitData* pProxyCommitData = self->pProxyCommitData;
std::vector<CommitTransactionRequest>& trs = self->trs;
ResolutionRequestBuilder requests(
pProxyCommitData,
self->commitVersion,
self->prevVersion,
pProxyCommitData->version,
self->span
);
2017-05-26 04:48:44 +08:00
int conflictRangeCount = 0;
2020-07-26 12:44:10 +08:00
self->maxTransactionBytes = 0;
for (int t = 0; t < trs.size(); t++) {
requests.addTransaction(trs[t], t);
2020-08-04 12:19:14 +08:00
conflictRangeCount +=
trs[t].transaction.read_conflict_ranges.size() + trs[t].transaction.write_conflict_ranges.size();
2017-05-26 04:48:44 +08:00
//TraceEvent("MPTransactionDump", self->dbgid).detail("Snapshot", trs[t].transaction.read_snapshot);
//for(auto& m : trs[t].transaction.mutations)
2020-07-26 12:44:10 +08:00
self->maxTransactionBytes = std::max<int64_t>(
self->maxTransactionBytes, trs[t].transaction.expectedSize()
);
2017-05-26 04:48:44 +08:00
// TraceEvent("MPTransactionsDump", self->dbgid).detail("Mutation", m.toString());
}
2020-07-26 12:44:10 +08:00
pProxyCommitData->stats.conflictRanges += conflictRangeCount;
2017-05-26 04:48:44 +08:00
2020-07-26 12:44:10 +08:00
for (int r = 1; r < pProxyCommitData->resolvers.size(); r++)
ASSERT(requests.requests[r].txnStateTransactions.size() ==
requests.requests[0].txnStateTransactions.size());
2017-05-26 04:48:44 +08:00
2020-07-26 12:44:10 +08:00
pProxyCommitData->stats.txnCommitResolving += trs.size();
std::vector<Future<ResolveTransactionBatchReply>> 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)));
2017-05-26 04:48:44 +08:00
}
2020-07-26 12:44:10 +08:00
self->transactionResolverMap.swap(requests.transactionResolverMap);
// Used to report conflicting keys
self->txReadConflictRangeIndexMap.swap(requests.txReadConflictRangeIndexMap);
self->releaseFuture = releaseResolvingAfter(
pProxyCommitData, self->releaseDelay, self->localBatchNumber
);
2017-05-26 04:48:44 +08:00
2020-07-26 12:44:10 +08:00
// Wait for the final resolution
std::vector<ResolveTransactionBatchReply> resolutionResp = wait(getAll(replies));
self->resolution.swap(*const_cast<std::vector<ResolveTransactionBatchReply>*>(&resolutionResp));
2017-05-26 04:48:44 +08:00
2020-07-26 12:44:10 +08:00
if (self->debugID.present()) {
2020-09-11 08:44:15 +08:00
g_traceBatch.addEvent("CommitDebug", self->debugID.get().first(),
"CommitProxyServer.commitBatch.AfterResolution");
2017-05-26 04:48:44 +08:00
}
2020-07-26 12:44:10 +08:00
return Void();
}
2017-05-26 04:48:44 +08:00
2020-07-29 15:42:52 +08:00
void assertResolutionStateMutationsSizeConsistent(
const std::vector<ResolveTransactionBatchReply>& resolution) {
2017-05-26 04:48:44 +08:00
2020-07-29 15:42:52 +08:00
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());
}
2017-05-26 04:48:44 +08:00
}
2020-07-29 15:42:52 +08:00
}
2017-05-26 04:48:44 +08:00
2020-07-29 15:42:52 +08:00
// Compute and apply "metadata" effects of each other proxy's most recent batch
void applyMetadataEffect(CommitBatchContext* self) {
2020-07-26 12:55:22 +08:00
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.
2017-05-26 04:48:44 +08:00
2020-07-26 12:55:22 +08:00
for (int transactionIndex = 0; transactionIndex < self->resolution[0].stateMutations[versionIndex].size() && !self->forceRecovery; transactionIndex++) {
2017-05-26 04:48:44 +08:00
bool committed = true;
2020-07-26 12:55:22 +08:00
for (int resolver = 0; resolver < self->resolution.size(); resolver++)
committed = committed && self->resolution[resolver].stateMutations[versionIndex][transactionIndex].committed;
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);
}
2020-07-26 12:55:22 +08:00
if( self->resolution[0].stateMutations[versionIndex][transactionIndex].mutations.size() && self->firstStateMutations ) {
2017-05-26 04:48:44 +08:00
ASSERT(committed);
2020-07-26 12:55:22 +08:00
self->firstStateMutations = false;
self->forceRecovery = false;
2017-05-26 04:48:44 +08:00
}
}
// These changes to txnStateStore will be committed by the other proxy, so we simply discard the commit message
2020-07-29 15:42:52 +08:00
auto fcm = self->pProxyCommitData->logAdapter->getCommitMessage();
self->storeCommits.emplace_back(fcm, self->pProxyCommitData->txnStateStore->commit());
2017-05-26 04:48:44 +08:00
if (initialState) {
initialState = false;
2020-07-26 12:55:22 +08:00
self->forceRecovery = false;
2020-07-29 15:42:52 +08:00
self->pProxyCommitData->txnStateStore->resyncLog();
2017-05-26 04:48:44 +08:00
2020-07-26 12:55:22 +08:00
for (auto &p : self->storeCommits) {
2017-05-26 04:48:44 +08:00
ASSERT(!p.second.isReady());
p.first.get().acknowledge.send(Void());
ASSERT(p.second.isReady());
}
2020-07-26 12:55:22 +08:00
self->storeCommits.clear();
2017-05-26 04:48:44 +08:00
}
}
2020-07-29 15:42:52 +08:00
}
2017-05-26 04:48:44 +08:00
2020-07-29 15:42:52 +08:00
/// Determine which transactions actually committed (conservatively) by combining results from the resolvers
void determineCommittedTransactions(CommitBatchContext* self) {
2020-07-29 15:42:52 +08:00
auto pProxyCommitData = self->pProxyCommitData;
const auto& trs = self->trs;
2017-05-26 04:48:44 +08:00
2020-07-26 12:55:22 +08:00
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.
2020-07-26 12:55:22 +08:00
self->nextTr.resize(self->resolution.size());
for (int t = 0; t < trs.size(); t++) {
2017-05-26 04:48:44 +08:00
uint8_t commit = ConflictBatch::TransactionCommitted;
2020-07-29 15:42:52 +08:00
for (int r : self->transactionResolverMap[t]) {
2020-07-26 12:55:22 +08:00
commit = std::min(self->resolution[r].committed[self->nextTr[r]++], commit);
2017-05-26 04:48:44 +08:00
}
2020-07-26 12:55:22 +08:00
self->committed[t] = commit;
2017-05-26 04:48:44 +08:00
}
2020-07-26 12:55:22 +08:00
for (int r = 0; r < self->resolution.size(); r++)
ASSERT(self->nextTr[r] == self->resolution[r].committed.size());
2017-05-26 04:48:44 +08:00
2020-07-26 12:55:22 +08:00
pProxyCommitData->logAdapter->setNextVersion(self->commitVersion);
2017-05-26 04:48:44 +08:00
2020-07-26 12:55:22 +08:00
self->lockedKey = pProxyCommitData->txnStateStore->readValue(databaseLockedKey).get();
self->locked = self->lockedKey.present() && self->lockedKey.get().size();
2017-05-26 04:48:44 +08:00
const Optional<Value> mustContainSystemKey = pProxyCommitData->txnStateStore->readValue(mustContainSystemMutationsKey).get();
2020-07-26 12:55:22 +08:00
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) {
2020-07-26 12:55:22 +08:00
self->committed[t] = ConflictBatch::TransactionConflict;
}
}
}
}
2020-07-29 15:42:52 +08:00
}
2020-07-29 15:42:52 +08:00
// This first pass through committed transactions deals with "metadata" effects (modifications of txnStateStore, changes to storage servers' responsibilities)
ACTOR Future<Void> applyMetadataToCommittedTransactions(CommitBatchContext* self) {
2020-07-29 15:42:52 +08:00
auto pProxyCommitData = self->pProxyCommitData;
const auto& trs = self->trs;
2017-05-26 04:48:44 +08:00
int t;
2020-07-29 15:42:52 +08:00
for (t = 0; t < trs.size() && !self->forceRecovery; t++) {
2020-07-26 12:55:22 +08:00
if (self->committed[t] == ConflictBatch::TransactionCommitted && (!self->locked || trs[t].isLockAware())) {
self->commitCount++;
applyMetadataMutations(*pProxyCommitData, self->arena, pProxyCommitData->logSystem,
trs[t].transaction.mutations, &self->toCommit, self->forceRecovery,
self->commitVersion + 1, /* initialCommit= */ false);
2017-05-26 04:48:44 +08:00
}
2020-07-26 12:55:22 +08:00
if(self->firstStateMutations) {
ASSERT(self->committed[t] == ConflictBatch::TransactionCommitted);
self->firstStateMutations = false;
self->forceRecovery = false;
2017-05-26 04:48:44 +08:00
}
}
2020-07-26 12:55:22 +08:00
if (self->forceRecovery) {
2020-07-29 15:42:52 +08:00
for (; t < trs.size(); t++)
2020-07-26 12:55:22 +08:00
self->committed[t] = ConflictBatch::TransactionConflict;
TraceEvent(SevWarn, "RestartingTxnSubsystem", pProxyCommitData->dbgid).detail("Stage", "AwaitCommit");
2017-05-26 04:48:44 +08:00
}
2020-07-26 12:55:22 +08:00
self->lockedKey = pProxyCommitData->txnStateStore->readValue(databaseLockedKey).get();
self->lockedAfter = self->lockedKey.present() && self->lockedKey.get().size();
2017-05-26 04:48:44 +08:00
2020-07-26 12:55:22 +08:00
self->metadataVersionAfter = pProxyCommitData->txnStateStore->readValue(metadataVersionKey).get();
2020-07-26 12:55:22 +08:00
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);
2017-05-26 04:48:44 +08:00
2020-07-26 12:55:22 +08:00
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() ) ) ) );
2017-05-26 04:48:44 +08:00
ASSERT(false); // ChangeCoordinatorsRequest should always throw
}
2020-07-29 15:42:52 +08:00
return Void();
}
/// This second pass through committed transactions assigns the actual mutations to the appropriate storage servers' tags
ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
2020-07-29 15:42:52 +08:00
state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData;
state std::vector<CommitTransactionRequest>& trs = self->trs;
2020-07-26 12:55:22 +08:00
for (; self->transactionNum < trs.size(); self->transactionNum++) {
2020-07-29 15:42:52 +08:00
if (!(self->committed[self->transactionNum] == ConflictBatch::TransactionCommitted && (!self->locked || trs[self->transactionNum].isLockAware()))) {
continue;
}
state bool checkSample = trs[self->transactionNum].commitCostEstimation.present();
2020-08-11 08:14:55 +08:00
state Optional<ClientTrCommitCostEstimation>* trCost = &trs[self->transactionNum].commitCostEstimation;
2020-07-29 15:42:52 +08:00
state int mutationNum = 0;
state VectorRef<MutationRef>* 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();
}
2020-07-29 15:42:52 +08:00
}
2017-05-26 04:48:44 +08:00
2020-07-29 15:42:52 +08:00
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);
// sample single key mutation based on cost
2020-08-11 08:14:55 +08:00
// the expectation of sampling is every COMMIT_SAMPLE_COST sample once
if (checkSample) {
double totalCosts = trCost->get().writeCosts;
2020-08-18 04:01:19 +08:00
double cost = getWriteOperationCost(m.expectedSize());
2020-08-11 08:14:55 +08:00
double mul = std::max(1.0, totalCosts / std::max(1.0, (double)CLIENT_KNOBS->COMMIT_SAMPLE_COST));
ASSERT(totalCosts > 0);
double prob = mul * cost / totalCosts;
if (deterministicRandom()->random01() < prob) {
for (const auto& ssInfo : pProxyCommitData->keyInfo[m.param1].src_info) {
2020-08-11 08:14:55 +08:00
auto id = ssInfo->interf.id();
2020-08-20 00:52:50 +08:00
// scale cost
cost = cost < CLIENT_KNOBS->COMMIT_SAMPLE_COST ? CLIENT_KNOBS->COMMIT_SAMPLE_COST : cost;
2020-08-11 08:14:55 +08:00
pProxyCommitData->updateSSTagCost(id, trs[self->transactionNum].tagSet.get(), m, cost);
2020-08-01 14:14:52 +08:00
}
}
2020-08-11 08:14:55 +08:00
}
2017-05-26 04:48:44 +08:00
2020-07-29 15:42:52 +08:00
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();
}
2017-05-26 04:48:44 +08:00
2020-07-29 15:42:52 +08:00
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);
2020-08-11 08:14:55 +08:00
// check whether clear is sampled
if (checkSample && !trCost->get().clearIdxCosts.empty() &&
trCost->get().clearIdxCosts[0].first == mutationNum) {
for (const auto& ssInfo : ranges.begin().value().src_info) {
2020-08-11 08:14:55 +08:00
auto id = ssInfo->interf.id();
pProxyCommitData->updateSSTagCost(id, trs[self->transactionNum].tagSet.get(), m,
trCost->get().clearIdxCosts[0].second);
2020-08-11 08:14:55 +08:00
}
trCost->get().clearIdxCosts.pop_front();
2019-11-13 05:01:29 +08:00
}
2017-05-26 04:48:44 +08:00
}
2020-07-29 15:42:52 +08:00
else {
TEST(true); //A clear range extends past a shard boundary
std::set<Tag> allSources;
for (auto r : ranges) {
r.value().populateTags();
allSources.insert(r.value().tags.begin(), r.value().tags.end());
2020-08-01 14:14:52 +08:00
// check whether clear is sampled
if (checkSample && !trCost->get().clearIdxCosts.empty() &&
trCost->get().clearIdxCosts[0].first == mutationNum) {
for (const auto& ssInfo : r.value().src_info) {
2020-08-01 14:48:40 +08:00
auto id = ssInfo->interf.id();
pProxyCommitData->updateSSTagCost(id, trs[self->transactionNum].tagSet.get(), m,
trCost->get().clearIdxCosts[0].second);
2020-08-01 14:48:40 +08:00
}
trCost->get().clearIdxCosts.pop_front();
2020-08-01 14:48:40 +08:00
}
2017-05-26 04:48:44 +08:00
}
2020-07-29 15:42:52 +08:00
DEBUG_MUTATION("ProxyCommit", self->commitVersion, m).detail("Dbgid", pProxyCommitData->dbgid).detail("To", allSources).detail("Mutation", m);
2019-11-13 05:01:29 +08:00
2020-07-29 15:42:52 +08:00
self->toCommit.addTags(allSources);
}
2017-05-26 04:48:44 +08:00
2020-07-29 15:42:52 +08:00
if(pProxyCommitData->needsCacheTag(clearRange)) {
self->toCommit.addTag(cacheTag);
2017-05-26 04:48:44 +08:00
}
2020-07-29 15:42:52 +08:00
self->toCommit.addTypedMessage(m);
} else {
UNREACHABLE();
}
2020-07-29 15:42:52 +08:00
// 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;
}
2017-05-26 04:48:44 +08:00
2020-07-29 15:42:52 +08:00
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);
2020-07-26 12:44:10 +08:00
}
2020-07-29 15:42:52 +08:00
}
else {
KeyRangeRef mutationRange(m.param1, m.param2);
KeyRangeRef intersectionRange;
2017-05-26 04:48:44 +08:00
2020-07-29 15:42:52 +08:00
// 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();
2017-05-26 04:48:44 +08:00
2020-07-29 15:42:52 +08:00
// Determine the intersecting range
intersectionRange = mutationRange & backupSubrange;
2017-05-26 04:48:44 +08:00
2020-07-29 15:42:52 +08:00
// Create the custom mutation for the specific backup tag
MutationRef backupMutation(MutationRef::Type::ClearRange, intersectionRange.begin, intersectionRange.end);
2017-05-26 04:48:44 +08:00
2020-07-29 15:42:52 +08:00
// Add the mutation to the relevant backup tag
for (auto backupName : backupRange.value()) {
self->logRangeMutations[backupName].push_back_deep(self->logRangeMutationsArena, backupMutation);
2017-05-26 04:48:44 +08:00
}
}
}
}
if (checkSample) {
self->pProxyCommitData->stats.txnExpensiveClearCostEstCount +=
trs[self->transactionNum].commitCostEstimation.get().expensiveCostEstCount;
}
2017-05-26 04:48:44 +08:00
}
2020-07-29 15:42:52 +08:00
return Void();
}
ACTOR Future<Void> postResolution(CommitBatchContext* self) {
2020-07-29 15:42:52 +08:00
state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData;
state std::vector<CommitTransactionRequest>& trs = self->trs;
state const int64_t localBatchNumber = self->localBatchNumber;
state const Optional<UID>& 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->timer();
2020-07-29 15:42:52 +08:00
pProxyCommitData->stats.txnCommitResolved += trs.size();
if (debugID.present()) {
2020-09-11 08:44:15 +08:00
g_traceBatch.addEvent("CommitDebug", debugID.get().first(),
"CommitProxyServer.commitBatch.ProcessingMutations");
2020-07-29 15:42:52 +08:00
}
self->isMyFirstBatch = !pProxyCommitData->version;
self->oldCoordinators = pProxyCommitData->txnStateStore->readValue(coordinatorsKey).get();
assertResolutionStateMutationsSizeConsistent(self->resolution);
applyMetadataEffect(self);
determineCommittedTransactions(self);
if(self->forceRecovery) {
wait( Future<Void>(Never()) );
}
// First pass
wait(applyMetadataToCommittedTransactions(self));
// Second pass
wait(assignMutationsToStorageServers(self));
2017-05-26 04:48:44 +08:00
// Serialize and backup the mutations as a single mutation
2020-07-26 12:55:22 +08:00
if ((pProxyCommitData->vecBackupKeys.size() > 1) && self->logRangeMutations.size()) {
wait( addBackupMutations(pProxyCommitData, &self->logRangeMutations, &self->toCommit, self->commitVersion, &self->computeDuration, &self->computeStart) );
2017-05-26 04:48:44 +08:00
}
2020-07-26 12:55:22 +08:00
pProxyCommitData->stats.mutations += self->mutationCount;
pProxyCommitData->stats.mutationBytes += self->mutationBytes;
2017-05-26 04:48:44 +08:00
// 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
2020-07-26 12:55:22 +08:00
if (pProxyCommitData->committedVersion.get() < self->commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS) {
self->computeDuration += g_network->timer() - self->computeStart;
state Span waitVersionSpan;
2020-07-26 12:55:22 +08:00
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
2020-07-26 12:55:22 +08:00
//TraceEvent("ProxyWaitingForCommitted", pProxyCommitData->dbgid).detail("CommittedVersion", pProxyCommitData->committedVersion.get()).detail("NeedToCommit", commitVersion);
waitVersionSpan = Span(deterministicRandom()->randomUniqueID(), "MP:overMaxReadTransactionLifeVersions"_loc, {self->span.context});
choose{
2020-07-26 12:55:22 +08:00
when(wait(pProxyCommitData->committedVersion.whenAtLeast(self->commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS))) {
wait(yield());
2020-07-26 12:44:10 +08:00
break;
}
2020-08-12 10:22:10 +08:00
when(wait(pProxyCommitData->cx->onProxiesChanged())) {}
when(GetRawCommittedVersionReply v = wait(pProxyCommitData->master.getLiveCommittedVersion.getReply(
2020-07-29 22:26:55 +08:00
GetRawCommittedVersionRequest(waitVersionSpan.context, debugID), TaskPriority::GetLiveCommittedVersionReply))) {
2020-07-26 12:55:22 +08:00
if(v.version > pProxyCommitData->committedVersion.get()) {
pProxyCommitData->locked = v.locked;
pProxyCommitData->metadataVersion = v.metadataVersion;
pProxyCommitData->committedVersion.set(v.version);
}
2020-06-09 07:09:37 +08:00
2020-07-26 12:55:22 +08:00
if (pProxyCommitData->committedVersion.get() < self->commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS)
wait(delay(SERVER_KNOBS->PROXY_SPIN_DELAY));
2017-05-26 04:48:44 +08:00
}
}
}
waitVersionSpan = Span{};
2020-07-26 12:55:22 +08:00
self->computeStart = g_network->timer();
2017-05-26 04:48:44 +08:00
}
2020-07-26 12:55:22 +08:00
self->msg = self->storeCommits.back().first.get();
2017-05-26 04:48:44 +08:00
2020-07-26 12:55:22 +08:00
if (self->debugID.present())
2020-09-11 08:44:15 +08:00
g_traceBatch.addEvent("CommitDebug", self->debugID.get().first(),
"CommitProxyServer.commitBatch.AfterStoreCommits");
2017-05-26 04:48:44 +08:00
// txnState (transaction subsystem state) tag: message extracted from log adapter
bool firstMessage = true;
2020-07-26 12:55:22 +08:00
for(auto m : self->msg.messages) {
2017-07-16 06:15:03 +08:00
if(firstMessage) {
2020-07-26 12:55:22 +08:00
self->toCommit.addTxsTag();
2017-07-16 06:15:03 +08:00
}
2020-07-26 12:55:22 +08:00
self->toCommit.addMessage(StringRef(m.begin(), m.size()), !firstMessage);
2017-05-26 04:48:44 +08:00
firstMessage = false;
}
2020-07-26 12:55:22 +08:00
if ( self->prevVersion && self->commitVersion - self->prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT/2 )
debug_advanceMaxCommittedVersion( UID(), self->commitVersion ); //< Is this valid?
2017-05-26 04:48:44 +08:00
2020-07-26 12:55:22 +08:00
//TraceEvent("ProxyPush", pProxyCommitData->dbgid).detail("PrevVersion", prevVersion).detail("Version", commitVersion)
// .detail("TransactionsSubmitted", trs.size()).detail("TransactionsCommitted", commitCount).detail("TxsPopTo", msg.popTo);
2017-05-26 04:48:44 +08:00
2020-07-26 12:55:22 +08:00
if ( self->prevVersion && self->commitVersion - self->prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT/2 )
debug_advanceMaxCommittedVersion(UID(), self->commitVersion);
2017-05-26 04:48:44 +08:00
2020-07-26 12:55:22 +08:00
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 );
2017-05-26 04:48:44 +08:00
2020-07-26 12:55:22 +08:00
if (!self->forceRecovery) {
ASSERT(pProxyCommitData->latestLocalCommitBatchLogging.get() == self->localBatchNumber-1);
pProxyCommitData->latestLocalCommitBatchLogging.set(self->localBatchNumber);
2017-05-26 04:48:44 +08:00
}
2020-07-26 12:55:22 +08:00
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 {
2020-07-26 12:55:22 +08:00
pProxyCommitData->commitComputePerOperation[self->latencyBucket] = SERVER_KNOBS->PROXY_COMPUTE_GROWTH_RATE*computePerOperation + ((1.0-SERVER_KNOBS->PROXY_COMPUTE_GROWTH_RATE)*pProxyCommitData->commitComputePerOperation[self->latencyBucket]);
}
}
2020-07-26 12:55:22 +08:00
return Void();
}
ACTOR Future<Void> transactionLogging(CommitBatchContext* self) {
2020-07-26 09:11:25 +08:00
state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData;
try {
choose {
2020-07-26 09:11:25 +08:00
when(Version ver = wait(self->loggingComplete)) {
pProxyCommitData->minKnownCommittedVersion = std::max(pProxyCommitData->minKnownCommittedVersion, ver);
}
2020-07-26 09:11:25 +08:00
when(wait(pProxyCommitData->committedVersion.whenAtLeast( self->commitVersion + 1 ))) {}
}
} catch(Error &e) {
if(e.code() == error_code_broken_promise) {
throw master_tlog_failed();
}
throw;
}
2020-07-26 09:11:25 +08:00
pProxyCommitData->lastCommitLatency = now() - self->commitStartTime;
pProxyCommitData->lastCommitTime = std::max(pProxyCommitData->lastCommitTime.get(), self->commitStartTime);
2020-07-26 12:44:10 +08:00
wait(yield(TaskPriority::ProxyCommitYield2));
2017-05-26 04:48:44 +08:00
2020-07-26 09:11:25 +08:00
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);
2020-07-26 09:11:25 +08:00
pProxyCommitData->txsPopVersions.pop_front();
}
2020-07-26 09:11:25 +08:00
pProxyCommitData->txsPopVersions.emplace_back(self->commitVersion, self->msg.popTo);
}
2020-07-26 09:11:25 +08:00
pProxyCommitData->logSystem->popTxs(self->msg.popTo);
2020-07-26 09:11:25 +08:00
return Void();
}
ACTOR Future<Void> reply(CommitBatchContext* self) {
2020-07-26 09:11:25 +08:00
state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData;
2017-05-26 04:48:44 +08:00
2020-07-26 09:11:25 +08:00
const Optional<UID>& debugID = self->debugID;
2017-05-26 04:48:44 +08:00
2020-07-26 09:11:25 +08:00
if ( self->prevVersion && self->commitVersion - self->prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT/2 )
debug_advanceMinCommittedVersion(UID(), self->commitVersion);
2017-05-26 04:48:44 +08:00
2020-07-26 09:11:25 +08:00
//TraceEvent("ProxyPushed", pProxyCommitData->dbgid).detail("PrevVersion", prevVersion).detail("Version", commitVersion);
2017-05-26 04:48:44 +08:00
if (debugID.present())
2020-09-11 08:44:15 +08:00
g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "CommitProxyServer.commitBatch.AfterLogPush");
2017-05-26 04:48:44 +08:00
2020-07-26 09:11:25 +08:00
for (auto &p : self->storeCommits) {
2017-05-26 04:48:44 +08:00
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.
2020-07-26 09:11:25 +08:00
TEST(pProxyCommitData->committedVersion.get() > self->commitVersion); // A later version was reported committed first
2020-08-13 01:34:07 +08:00
if (self->commitVersion >= pProxyCommitData->committedVersion.get()) {
2020-07-26 09:11:25 +08:00
wait(pProxyCommitData->master.reportLiveCommittedVersion.getReply(
2020-08-13 01:34:07 +08:00
ReportRawCommittedVersionRequest(self->commitVersion, self->lockedAfter, self->metadataVersionAfter,
pProxyCommitData->minKnownCommittedVersion),
TaskPriority::ProxyMasterVersionReply));
}
2020-07-26 09:11:25 +08:00
if( self->commitVersion > pProxyCommitData->committedVersion.get() ) {
pProxyCommitData->locked = self->lockedAfter;
pProxyCommitData->metadataVersion = self->metadataVersionAfter;
pProxyCommitData->committedVersion.set(self->commitVersion);
}
2020-07-29 22:26:55 +08:00
2020-07-26 09:11:25 +08:00
if (self->forceRecovery) {
TraceEvent(SevWarn, "RestartingTxnSubsystem", pProxyCommitData->dbgid).detail("Stage", "ProxyShutdown");
2017-05-26 04:48:44 +08:00
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
2020-07-26 09:11:25 +08:00
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));
}
2020-07-26 09:11:25 +08:00
else if (self->committed[t] == ConflictBatch::TransactionTooOld) {
tr.reply.sendError(transaction_too_old());
}
2019-07-23 06:44:49 +08:00
else {
2020-03-27 06:52:30 +08:00
// If enable the option to report conflicting keys from resolvers, we send back all keyranges' indices
// through CommitID
2020-07-26 09:11:25 +08:00
if (tr.transaction.report_conflicting_keys) {
Standalone<VectorRef<int>> conflictingKRIndices;
2020-07-26 09:11:25 +08:00
for (int resolverInd : self->transactionResolverMap[t]) {
2020-05-22 20:24:15 +08:00
auto const& cKRs =
2020-07-26 09:11:25 +08:00
self->resolution[resolverInd]
.conflictingKeyRangeMap[self->nextTr[resolverInd]]; // nextTr[resolverInd] -> index of this trs[t]
2020-05-22 20:24:15 +08:00
// on the resolver
2020-03-27 06:52:30 +08:00
for (auto const& rCRIndex : cKRs)
2020-05-22 20:24:15 +08:00
// 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(),
2020-07-26 09:11:25 +08:00
self->txReadConflictRangeIndexMap[t][resolverInd][rCRIndex]);
}
// At least one keyRange index should be returned
ASSERT(conflictingKRIndices.size());
2020-07-26 09:11:25 +08:00
tr.reply.send(CommitID(invalidVersion, t, Optional<Value>(),
2020-03-27 06:52:30 +08:00
Optional<Standalone<VectorRef<int>>>(conflictingKRIndices)));
} else {
2020-07-26 09:11:25 +08:00
tr.reply.sendError(not_committed());
}
}
// Update corresponding transaction indices on each resolver
2020-07-26 09:11:25 +08:00
for (int resolverInd : self->transactionResolverMap[t]) self->nextTr[resolverInd]++;
// TODO: filter if pipelined with large commit
2020-08-04 12:19:14 +08:00
const double duration = endTime - tr.requestTime();
2020-07-26 09:11:25 +08:00
pProxyCommitData->stats.commitLatencySample.addMeasurement(duration);
if(pProxyCommitData->latencyBandConfig.present()) {
bool filter = self->maxTransactionBytes > pProxyCommitData->latencyBandConfig.get().commitConfig.maxCommitBytes.orDefault(std::numeric_limits<int>::max());
2020-08-04 12:19:14 +08:00
pProxyCommitData->stats.commitLatencyBands.addMeasurement(duration, filter);
}
2017-05-26 04:48:44 +08:00
}
2020-07-26 09:11:25 +08:00
++pProxyCommitData->stats.commitBatchOut;
pProxyCommitData->stats.txnCommitOut += self->trs.size();
pProxyCommitData->stats.txnConflicts += self->trs.size() - self->commitCount;
pProxyCommitData->stats.txnCommitOutSuccess += self->commitCount;
2017-05-26 04:48:44 +08:00
2020-07-26 09:11:25 +08:00
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;
2017-05-26 04:48:44 +08:00
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;
}
2020-07-26 09:11:25 +08:00
pProxyCommitData->keyResolvers.coalesce(allKeys);
if(pProxyCommitData->keyResolvers.size() != lastSize)
TraceEvent("KeyResolverSize", pProxyCommitData->dbgid).detail("Size", pProxyCommitData->keyResolvers.size());
2017-05-26 04:48:44 +08:00
}
// Dynamic batching for commits
2020-07-26 09:11:25 +08:00
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 +
2020-07-26 09:11:25 +08:00
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
ACTOR Future<Void> commitBatch(
ProxyCommitData* self,
vector<CommitTransactionRequest>* 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::CommitBatchContext context(self, trs, currentBatchMemBytesCount);
2020-07-26 09:11:25 +08:00
// 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;
2020-08-22 02:06:31 +08:00
context.setupTraceBatch();
2020-07-26 09:11:25 +08:00
/////// 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)
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)
wait(CommitBatch::postResolution(&context));
/////// Phase 4: Logging (network bound; pipelined up to MAX_READ_TRANSACTION_LIFE_VERSIONS (limited by loop above))
wait(CommitBatch::transactionLogging(&context));
/////// Phase 5: Replies (CPU bound; no particular order required, though ordered execution would be best for latency)
wait(CommitBatch::reply(&context));
2017-05-26 04:48:44 +08:00
return Void();
}
ACTOR static Future<Void> doKeyServerLocationRequest( GetKeyServerLocationsRequest req, ProxyCommitData* commitData ) {
2017-05-26 04:48:44 +08:00
// We can't respond to these requests until we have valid txnStateStore
wait(commitData->validState.getFuture());
wait(delay(0, TaskPriority::DefaultEndpoint));
GetKeyServerLocationsReply rep;
if(!req.end.present()) {
auto r = req.reverse ? commitData->keyInfo.rangeContainingKeyBefore(req.begin) : commitData->keyInfo.rangeContaining(req.begin);
vector<StorageServerInterface> ssis;
ssis.reserve(r.value().src_info.size());
for(auto& it : r.value().src_info) {
ssis.push_back(it->interf);
}
rep.results.push_back(std::make_pair(r.range(), ssis));
} else if(!req.reverse) {
int count = 0;
for(auto r = commitData->keyInfo.rangeContaining(req.begin); r != commitData->keyInfo.ranges().end() && count < req.limit && r.begin() < req.end.get(); ++r) {
vector<StorageServerInterface> ssis;
ssis.reserve(r.value().src_info.size());
for(auto& it : r.value().src_info) {
ssis.push_back(it->interf);
}
rep.results.push_back(std::make_pair(r.range(), ssis));
count++;
}
} else {
int count = 0;
auto r = commitData->keyInfo.rangeContainingKeyBefore(req.end.get());
while( count < req.limit && req.begin < r.end() ) {
vector<StorageServerInterface> ssis;
ssis.reserve(r.value().src_info.size());
for(auto& it : r.value().src_info) {
ssis.push_back(it->interf);
}
rep.results.push_back(std::make_pair(r.range(), ssis));
if(r == commitData->keyInfo.ranges().begin()) {
break;
}
count++;
--r;
}
}
req.reply.send(rep);
++commitData->stats.keyServerLocationOut;
return Void();
}
2020-09-11 08:44:15 +08:00
ACTOR static Future<Void> readRequestServer(CommitProxyInterface proxy, PromiseStream<Future<Void>> addActor,
ProxyCommitData* commitData) {
loop {
GetKeyServerLocationsRequest req = waitNext(proxy.getKeyServersLocations.getFuture());
//WARNING: this code is run at a high priority, so it needs to do as little work as possible
if(req.limit != CLIENT_KNOBS->STORAGE_METRICS_SHARD_LIMIT && //Always do data distribution requests
commitData->stats.keyServerLocationIn.getValue() - commitData->stats.keyServerLocationOut.getValue() > SERVER_KNOBS->KEY_LOCATION_MAX_QUEUE_SIZE) {
++commitData->stats.keyServerLocationErrors;
req.reply.sendError(proxy_memory_limit_exceeded());
TraceEvent(SevWarnAlways, "ProxyLocationRequestThresholdExceeded").suppressFor(60);
} else {
++commitData->stats.keyServerLocationIn;
addActor.send(doKeyServerLocationRequest(req, commitData));
}
}
}
2020-09-11 08:44:15 +08:00
ACTOR static Future<Void> rejoinServer(CommitProxyInterface proxy, ProxyCommitData* commitData) {
// We can't respond to these requests until we have valid txnStateStore
wait(commitData->validState.getFuture());
TraceEvent("ProxyReadyForReads", proxy.id());
loop {
GetStorageServerRejoinInfoRequest req = waitNext(proxy.getStorageServerRejoinInfo.getFuture());
if (commitData->txnStateStore->readValue(serverListKeyFor(req.id)).get().present()) {
GetStorageServerRejoinInfoReply rep;
rep.version = commitData->version;
rep.tag = decodeServerTagValue( commitData->txnStateStore->readValue(serverTagKeyFor(req.id)).get().get() );
Standalone<RangeResultRef> history = commitData->txnStateStore->readRange(serverTagHistoryRangeFor(req.id)).get();
for(int i = history.size()-1; i >= 0; i-- ) {
rep.history.push_back(std::make_pair(decodeServerTagHistoryKey(history[i].key), decodeServerTagValue(history[i].value)));
}
auto localityKey = commitData->txnStateStore->readValue(tagLocalityListKeyFor(req.dcId)).get();
rep.newLocality = false;
if( localityKey.present() ) {
int8_t locality = decodeTagLocalityListValue(localityKey.get());
if(rep.tag.locality != tagLocalityUpgraded && locality != rep.tag.locality) {
TraceEvent(SevWarnAlways, "SSRejoinedWithChangedLocality").detail("Tag", rep.tag.toString()).detail("DcId", req.dcId).detail("NewLocality", locality);
} else if(locality != rep.tag.locality) {
uint16_t tagId = 0;
std::vector<uint16_t> usedTags;
auto tagKeys = commitData->txnStateStore->readRange(serverTagKeys).get();
for( auto& kv : tagKeys ) {
Tag t = decodeServerTagValue( kv.value );
if(t.locality == locality) {
usedTags.push_back(t.id);
}
}
auto historyKeys = commitData->txnStateStore->readRange(serverTagHistoryKeys).get();
for( auto& kv : historyKeys ) {
Tag t = decodeServerTagValue( kv.value );
if(t.locality == locality) {
usedTags.push_back(t.id);
}
}
std::sort(usedTags.begin(), usedTags.end());
int usedIdx = 0;
for(; usedTags.size() > 0 && tagId <= usedTags.end()[-1]; tagId++) {
if(tagId < usedTags[usedIdx]) {
break;
} else {
usedIdx++;
}
}
rep.newTag = Tag(locality, tagId);
2017-07-27 04:45:11 +08:00
}
} else if(rep.tag.locality != tagLocalityUpgraded) {
TraceEvent(SevWarnAlways, "SSRejoinedWithUnknownLocality").detail("Tag", rep.tag.toString()).detail("DcId", req.dcId);
} else {
rep.newLocality = true;
int8_t maxTagLocality = -1;
auto localityKeys = commitData->txnStateStore->readRange(tagLocalityListKeys).get();
for( auto& kv : localityKeys ) {
maxTagLocality = std::max(maxTagLocality, decodeTagLocalityListValue( kv.value ));
}
rep.newTag = Tag(maxTagLocality+1,0);
}
req.reply.send(rep);
} else {
req.reply.sendError(worker_removed());
2017-05-26 04:48:44 +08:00
}
}
}
2020-09-11 08:44:15 +08:00
ACTOR Future<Void> ddMetricsRequestServer(CommitProxyInterface proxy, Reference<AsyncVar<ServerDBInfo>> db) {
loop {
choose {
when(state GetDDMetricsRequest req = waitNext(proxy.getDDMetrics.getFuture()))
{
if (!db->get().distributor.present()) {
2020-08-15 00:40:16 +08:00
req.reply.sendError(dd_not_found());
2020-08-03 08:11:31 +08:00
continue;
}
2020-08-01 02:13:05 +08:00
ErrorOr<GetDataDistributorMetricsReply> reply =
wait(errorOr(db->get().distributor.get().dataDistributorMetrics.getReply(
2020-08-10 14:42:30 +08:00
GetDataDistributorMetricsRequest(req.keys, req.shardLimit))));
2020-08-01 02:13:05 +08:00
if (reply.isError()) {
req.reply.sendError(reply.getError());
} else {
GetDDMetricsReply newReply;
newReply.storageMetricsList = reply.get().storageMetricsList;
req.reply.send(newReply);
}
}
}
}
}
ACTOR Future<Void> monitorRemoteCommitted(ProxyCommitData* self) {
loop {
wait(delay(0)); //allow this actor to be cancelled if we are removed after db changes.
state Optional<std::vector<OptionalInterface<TLogInterface>>> remoteLogs;
if(self->db->get().recoveryState >= RecoveryState::ALL_LOGS_RECRUITED) {
for(auto& logSet : self->db->get().logSystemConfig.tLogs) {
if(!logSet.isLocal) {
remoteLogs = logSet.tLogs;
for(auto& tLog : logSet.tLogs) {
if(!tLog.present()) {
remoteLogs = Optional<std::vector<OptionalInterface<TLogInterface>>>();
break;
}
}
break;
}
}
}
if(!remoteLogs.present()) {
wait(self->db->onChange());
continue;
}
self->popRemoteTxs = true;
state Future<Void> onChange = self->db->onChange();
loop {
state std::vector<Future<TLogQueuingMetricsReply>> replies;
for(auto &it : remoteLogs.get()) {
replies.push_back(brokenPromiseToNever( it.interf().getQueuingMetrics.getReply( TLogQueuingMetricsRequest() ) ));
}
wait( waitForAll(replies) || onChange );
if(onChange.isReady()) {
break;
}
//FIXME: use the configuration to calculate a more precise minimum recovery version.
Version minVersion = std::numeric_limits<Version>::max();
for(auto& it : replies) {
minVersion = std::min(minVersion, it.get().v);
}
while(self->txsPopVersions.size() && self->txsPopVersions.front().first <= minVersion) {
self->lastTxsPop = self->txsPopVersions.front().second;
self->logSystem->popTxs(self->txsPopVersions.front().second, tagLocalityRemoteLog);
self->txsPopVersions.pop_front();
}
wait( delay(SERVER_KNOBS->UPDATE_REMOTE_LOG_VERSION_INTERVAL) || onChange );
if(onChange.isReady()) {
break;
}
}
}
}
ACTOR Future<Void> proxySnapCreate(ProxySnapRequest snapReq, ProxyCommitData* commitData) {
2020-09-11 08:44:15 +08:00
TraceEvent("SnapCommitProxy_SnapReqEnter")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID);
2019-06-20 02:11:50 +08:00
try {
// whitelist check
ExecCmdValueString execArg(snapReq.snapPayload);
StringRef binPath = execArg.getBinaryPath();
2019-06-20 02:11:50 +08:00
if (!isWhitelisted(commitData->whitelistedBinPathVec, binPath)) {
2020-09-11 08:44:15 +08:00
TraceEvent("SnapCommitProxy_WhiteListCheckFailed")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID);
2019-08-24 02:56:06 +08:00
throw snap_path_not_whitelisted();
2019-06-20 02:11:50 +08:00
}
// db fully recovered check
if (commitData->db->get().recoveryState != RecoveryState::FULLY_RECOVERED) {
// Cluster is not fully recovered and needs TLogs
// from previous generation for full recovery.
// Currently, snapshot of old tlog generation is not
// supported and hence failing the snapshot request until
// cluster is fully_recovered.
2020-09-11 08:44:15 +08:00
TraceEvent("SnapCommitProxy_ClusterNotFullyRecovered")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID);
2019-08-24 02:56:06 +08:00
throw snap_not_fully_recovered_unsupported();
2019-06-20 02:11:50 +08:00
}
2019-06-20 02:11:50 +08:00
auto result =
commitData->txnStateStore->readValue(LiteralStringRef("log_anti_quorum").withPrefix(configKeysPrefix)).get();
int logAntiQuorum = 0;
if (result.present()) {
logAntiQuorum = atoi(result.get().toString().c_str());
}
// FIXME: logAntiQuorum not supported, remove it later,
2019-07-16 02:33:40 +08:00
// In version2, we probably don't need this limtiation, but this needs to be tested.
2019-06-20 02:11:50 +08:00
if (logAntiQuorum > 0) {
2020-09-11 08:44:15 +08:00
TraceEvent("SnapCommitProxy_LogAnitQuorumNotSupported")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID);
2019-08-24 02:56:06 +08:00
throw snap_log_anti_quorum_unsupported();
2019-06-20 02:11:50 +08:00
}
// send a snap request to DD
if (!commitData->db->get().distributor.present()) {
TraceEvent(SevWarnAlways, "DataDistributorNotPresent").detail("Operation", "SnapRequest");
2020-08-15 00:40:16 +08:00
throw dd_not_found();
2019-06-20 02:11:50 +08:00
}
state Future<ErrorOr<Void>> ddSnapReq =
commitData->db->get().distributor.get().distributorSnapReq.tryGetReply(DistributorSnapRequest(snapReq.snapPayload, snapReq.snapUID));
2019-06-20 02:11:50 +08:00
try {
wait(throwErrorOr(ddSnapReq));
2019-06-20 02:11:50 +08:00
} catch (Error& e) {
2020-09-11 08:44:15 +08:00
TraceEvent("SnapCommitProxy_DDSnapResponseError")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID)
.error(e, true /*includeCancelled*/);
throw e;
2019-06-20 02:11:50 +08:00
}
snapReq.reply.send(Void());
} catch (Error& e) {
2020-09-11 08:44:15 +08:00
TraceEvent("SnapCommitProxy_SnapReqError")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID)
.error(e, true /*includeCancelled*/);
if (e.code() != error_code_operation_cancelled) {
2019-06-20 02:11:50 +08:00
snapReq.reply.sendError(e);
} else {
throw e;
2019-06-20 02:11:50 +08:00
}
}
2020-09-11 08:44:15 +08:00
TraceEvent("SnapCommitProxy_SnapReqExit")
.detail("SnapPayload", snapReq.snapPayload)
.detail("SnapUID", snapReq.snapUID);
2019-06-20 02:11:50 +08:00
return Void();
}
ACTOR Future<Void> proxyCheckSafeExclusion(Reference<AsyncVar<ServerDBInfo>> db, ExclusionSafetyCheckRequest req) {
2020-09-11 08:44:15 +08:00
TraceEvent("SafetyCheckCommitProxyBegin");
state ExclusionSafetyCheckReply reply(false);
if (!db->get().distributor.present()) {
TraceEvent(SevWarnAlways, "DataDistributorNotPresent").detail("Operation", "ExclusionSafetyCheck");
req.reply.send(reply);
return Void();
}
try {
state Future<ErrorOr<DistributorExclusionSafetyCheckReply>> safeFuture =
db->get().distributor.get().distributorExclCheckReq.tryGetReply(
DistributorExclusionSafetyCheckRequest(req.exclusions));
DistributorExclusionSafetyCheckReply _reply = wait(throwErrorOr(safeFuture));
reply.safe = _reply.safe;
} catch (Error& e) {
2020-09-11 08:44:15 +08:00
TraceEvent("SafetyCheckCommitProxyResponseError").error(e);
if (e.code() != error_code_operation_cancelled) {
req.reply.sendError(e);
return Void();
} else {
throw e;
}
}
2020-09-11 08:44:15 +08:00
TraceEvent("SafetyCheckCommitProxyFinish");
req.reply.send(reply);
return Void();
}
ACTOR Future<Void> reportTxnTagCommitCost(UID myID, Reference<AsyncVar<ServerDBInfo>> db,
UIDTransactionTagMap<TransactionCommitCostEstimation>* ssTrTagCommitCost) {
state Future<Void> nextRequestTimer = Never();
state Future<Void> nextReply = Never();
if (db->get().ratekeeper.present()) nextRequestTimer = Void();
loop choose {
when(wait(db->onChange())) {
if (db->get().ratekeeper.present()) {
TraceEvent("ProxyRatekeeperChanged", myID).detail("RKID", db->get().ratekeeper.get().id());
nextRequestTimer = Void();
} else {
TraceEvent("ProxyRatekeeperDied", myID);
nextRequestTimer = Never();
}
}
when(wait(nextRequestTimer)) {
nextRequestTimer = Never();
if (db->get().ratekeeper.present()) {
nextReply = brokenPromiseToNever(db->get().ratekeeper.get().reportCommitCostEstimation.getReply(
ReportCommitCostEstimationRequest(*ssTrTagCommitCost)));
} else {
nextReply = Never();
}
}
when(wait(nextReply)) {
nextReply = Never();
ssTrTagCommitCost->clear();
nextRequestTimer = delay(SERVER_KNOBS->REPORT_TRANSACTION_COST_ESTIMATION_DELAY);
}
}
}
2020-09-11 08:44:15 +08:00
ACTOR Future<Void> commitProxyServerCore(CommitProxyInterface proxy, MasterInterface master,
Reference<AsyncVar<ServerDBInfo>> db, LogEpoch epoch,
Version recoveryTransactionVersion, bool firstProxy,
std::string whitelistBinPaths) {
2017-05-26 04:48:44 +08:00
state ProxyCommitData commitData(proxy.id(), master, proxy.getConsistentReadVersion, recoveryTransactionVersion, proxy.commit, db, firstProxy);
state Future<Sequence> sequenceFuture = (Sequence)0;
state PromiseStream< std::pair<vector<CommitTransactionRequest>, int> > batchedCommits;
state Future<Void> commitBatcherActor;
2017-05-26 04:48:44 +08:00
state Future<Void> lastCommitComplete = Void();
state PromiseStream<Future<Void>> addActor;
state Future<Void> onError = transformError( actorCollection(addActor.getFuture()), broken_promise(), master_tlog_failed() );
2017-05-26 04:48:44 +08:00
state double lastCommit = 0;
state std::set<Sequence> txnSequences;
state Sequence maxSequence = std::numeric_limits<Sequence>::max();
state GetHealthMetricsReply healthMetricsReply;
state GetHealthMetricsReply detailedHealthMetricsReply;
2017-05-26 04:48:44 +08:00
addActor.send( waitFailureServer(proxy.waitFailure.getFuture()) );
2020-09-11 08:44:15 +08:00
addActor.send(traceRole(Role::COMMIT_PROXY, proxy.id()));
2017-05-26 04:48:44 +08:00
2020-09-11 08:44:15 +08:00
//TraceEvent("CommitProxyInit1", proxy.id());
2017-05-26 04:48:44 +08:00
// Wait until we can load the "real" logsystem, since we don't support switching them currently
while (!(commitData.db->get().master.id() == master.id() && commitData.db->get().recoveryState >= RecoveryState::RECOVERY_TRANSACTION)) {
2017-05-26 04:48:44 +08:00
//TraceEvent("ProxyInit2", proxy.id()).detail("LSEpoch", db->get().logSystemConfig.epoch).detail("Need", epoch);
wait(commitData.db->onChange());
2017-05-26 04:48:44 +08:00
}
state Future<Void> dbInfoChange = commitData.db->onChange();
2017-05-26 04:48:44 +08:00
//TraceEvent("ProxyInit3", proxy.id());
commitData.resolvers = commitData.db->get().resolvers;
2017-05-26 04:48:44 +08:00
ASSERT(commitData.resolvers.size() != 0);
auto rs = commitData.keyResolvers.modify(allKeys);
for(auto r = rs.begin(); r != rs.end(); ++r)
r->value().emplace_back(0,0);
2017-05-26 04:48:44 +08:00
commitData.logSystem = ILogSystem::fromServerDBInfo(proxy.id(), commitData.db->get(), false, addActor);
commitData.logAdapter = new LogSystemDiskQueueAdapter(commitData.logSystem, Reference<AsyncVar<PeekTxsInfo>>(), 1, false);
commitData.txnStateStore = keyValueStoreLogSystem(commitData.logAdapter, proxy.id(), 2e9, true, true, true);
createWhitelistBinPathVec(whitelistBinPaths, commitData.whitelistedBinPathVec);
2017-05-26 04:48:44 +08:00
commitData.updateLatencyBandConfig(commitData.db->get().latencyBandConfig);
// ((SERVER_MEM_LIMIT * COMMIT_BATCHES_MEM_FRACTION_OF_TOTAL) / COMMIT_BATCHES_MEM_TO_TOTAL_MEM_SCALE_FACTOR) is only a approximate formula for limiting the memory used.
// COMMIT_BATCHES_MEM_TO_TOTAL_MEM_SCALE_FACTOR is an estimate based on experiments and not an accurate one.
state int64_t commitBatchesMemoryLimit = std::min(SERVER_KNOBS->COMMIT_BATCHES_MEM_BYTES_HARD_LIMIT, static_cast<int64_t>((SERVER_KNOBS->SERVER_MEM_LIMIT * SERVER_KNOBS->COMMIT_BATCHES_MEM_FRACTION_OF_TOTAL) / SERVER_KNOBS->COMMIT_BATCHES_MEM_TO_TOTAL_MEM_SCALE_FACTOR));
TraceEvent(SevInfo, "CommitBatchesMemoryLimit").detail("BytesLimit", commitBatchesMemoryLimit);
2017-05-26 04:48:44 +08:00
addActor.send(monitorRemoteCommitted(&commitData));
addActor.send(readRequestServer(proxy, addActor, &commitData));
addActor.send(rejoinServer(proxy, &commitData));
addActor.send(ddMetricsRequestServer(proxy, db));
addActor.send(reportTxnTagCommitCost(proxy.id(), db, &commitData.ssTrTagCommitCost));
2017-05-26 04:48:44 +08:00
// wait for txnStateStore recovery
wait(success(commitData.txnStateStore->readValue(StringRef())));
2017-05-26 04:48:44 +08:00
int commitBatchByteLimit =
(int)std::min<double>(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_BYTES_MAX,
std::max<double>(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_BYTES_MIN,
SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_BYTES_SCALE_BASE *
2020-09-11 08:44:15 +08:00
pow(commitData.db->get().client.commitProxies.size(),
SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_BYTES_SCALE_POWER)));
commitBatcherActor = commitBatcher(&commitData, batchedCommits, proxy.commit.getFuture(), commitBatchByteLimit, commitBatchesMemoryLimit);
2017-05-26 04:48:44 +08:00
loop choose{
when( wait( dbInfoChange ) ) {
dbInfoChange = commitData.db->onChange();
if(commitData.db->get().master.id() == master.id() && commitData.db->get().recoveryState >= RecoveryState::RECOVERY_TRANSACTION) {
commitData.logSystem = ILogSystem::fromServerDBInfo(proxy.id(), commitData.db->get(), false, addActor);
for(auto it : commitData.tag_popped) {
commitData.logSystem->pop(it.second, it.first);
}
commitData.logSystem->popTxs(commitData.lastTxsPop, tagLocalityRemoteLog);
}
commitData.updateLatencyBandConfig(commitData.db->get().latencyBandConfig);
}
when(wait(onError)) {}
when(std::pair<vector<CommitTransactionRequest>, int> batchedRequests = waitNext(batchedCommits.getFuture())) {
//WARNING: this code is run at a high priority, so it needs to do as little work as possible
const vector<CommitTransactionRequest> &trs = batchedRequests.first;
int batchBytes = batchedRequests.second;
2020-09-11 08:44:15 +08:00
//TraceEvent("CommitProxyCTR", proxy.id()).detail("CommitTransactions", trs.size()).detail("TransactionRate", transactionRate).detail("TransactionQueue", transactionQueue.size()).detail("ReleasedTransactionCount", transactionCount);
if (trs.size() || (commitData.db->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS && now() - lastCommit >= SERVER_KNOBS->MAX_COMMIT_BATCH_INTERVAL)) {
2017-05-26 04:48:44 +08:00
lastCommit = now();
if (trs.size() || lastCommitComplete.isReady()) {
lastCommitComplete = commitBatch(
&commitData,
const_cast<std::vector<CommitTransactionRequest>*>(&batchedRequests.first),
batchBytes
);
2017-05-26 04:48:44 +08:00
addActor.send(lastCommitComplete);
}
}
}
2019-06-20 02:11:50 +08:00
when(ProxySnapRequest snapReq = waitNext(proxy.proxySnapReq.getFuture())) {
TraceEvent(SevDebug, "SnapMasterEnqueue");
2019-06-20 02:11:50 +08:00
addActor.send(proxySnapCreate(snapReq, &commitData));
}
when(ExclusionSafetyCheckRequest exclCheckReq = waitNext(proxy.exclusionSafetyCheckReq.getFuture())) {
addActor.send(proxyCheckSafeExclusion(db, exclCheckReq));
}
when(state TxnStateRequest req = waitNext(proxy.txnState.getFuture())) {
2017-05-26 04:48:44 +08:00
state ReplyPromise<Void> reply = req.reply;
if(req.last) maxSequence = req.sequence + 1;
if (!txnSequences.count(req.sequence)) {
txnSequences.insert(req.sequence);
2017-05-26 04:48:44 +08:00
ASSERT(!commitData.validState.isSet()); // Although we may receive the CommitTransactionRequest for the recovery transaction before all of the TxnStateRequest, we will not get a resolution result from any resolver until the master has submitted its initial (sequence 0) resolution request, which it doesn't do until we have acknowledged all TxnStateRequests
for(auto& kv : req.data)
commitData.txnStateStore->set(kv, &req.arena);
commitData.txnStateStore->commit(true);
if(txnSequences.size() == maxSequence) {
state KeyRange txnKeys = allKeys;
Standalone<RangeResultRef> UIDtoTagMap = commitData.txnStateStore->readRange( serverTagKeys ).get();
state std::map<Tag, UID> tag_uid;
for (const KeyValueRef kv : UIDtoTagMap) {
tag_uid[decodeServerTagValue(kv.value)] = decodeServerTagKey(kv.key);
}
2017-05-26 04:48:44 +08:00
loop {
wait(yield());
Standalone<RangeResultRef> data = commitData.txnStateStore->readRange(txnKeys, SERVER_KNOBS->BUGGIFIED_ROW_LIMIT, SERVER_KNOBS->APPLY_MUTATION_BYTES).get();
2017-05-26 04:48:44 +08:00
if(!data.size()) break;
((KeyRangeRef&)txnKeys) = KeyRangeRef( keyAfter(data.back().key, txnKeys.arena()), txnKeys.end );
MutationsVec mutations;
std::vector<std::pair<MapPair<Key,ServerCacheInfo>,int>> keyInfoData;
2017-05-26 04:48:44 +08:00
vector<UID> src, dest;
ServerCacheInfo info;
2017-05-26 04:48:44 +08:00
for(auto &kv : data) {
if( kv.key.startsWith(keyServersPrefix) ) {
KeyRef k = kv.key.removePrefix(keyServersPrefix);
if(k != allKeys.end) {
decodeKeyServersValue(tag_uid, kv.value, src, dest);
info.tags.clear();
info.src_info.clear();
info.dest_info.clear();
2019-04-13 02:11:11 +08:00
for (const auto& id : src) {
auto storageInfo = getStorageInfo(id, &commitData.storageCache, commitData.txnStateStore);
ASSERT(storageInfo->tag != invalidTag);
info.tags.push_back( storageInfo->tag );
info.src_info.push_back( storageInfo );
2017-05-26 04:48:44 +08:00
}
2019-04-13 02:11:11 +08:00
for (const auto& id : dest) {
auto storageInfo = getStorageInfo(id, &commitData.storageCache, commitData.txnStateStore);
ASSERT(storageInfo->tag != invalidTag);
info.tags.push_back( storageInfo->tag );
info.dest_info.push_back( storageInfo );
2017-05-26 04:48:44 +08:00
}
uniquify(info.tags);
keyInfoData.emplace_back(MapPair<Key,ServerCacheInfo>(k, info), 1);
2017-05-26 04:48:44 +08:00
}
} else {
mutations.emplace_back(mutations.arena(), MutationRef::SetValue, kv.key, kv.value);
2017-05-26 04:48:44 +08:00
}
}
2017-05-26 04:48:44 +08:00
//insert keyTag data separately from metadata mutations so that we can do one bulk insert which avoids a lot of map lookups.
commitData.keyInfo.rawInsert(keyInfoData);
2017-05-26 04:48:44 +08:00
Arena arena;
bool confChanges;
applyMetadataMutations(commitData, arena, Reference<ILogSystem>(), mutations,
/* pToCommit= */ nullptr, confChanges,
/* popVersion= */ 0, /* initialCommit= */ true);
2017-05-26 04:48:44 +08:00
}
auto lockedKey = commitData.txnStateStore->readValue(databaseLockedKey).get();
commitData.locked = lockedKey.present() && lockedKey.get().size();
commitData.metadataVersion = commitData.txnStateStore->readValue(metadataVersionKey).get();
2017-05-26 04:48:44 +08:00
commitData.txnStateStore->enableSnapshot();
}
}
2020-04-11 08:02:11 +08:00
addActor.send(broadcastTxnRequest(req, SERVER_KNOBS->TXN_STATE_SEND_AMOUNT, true));
wait(yield());
2017-05-26 04:48:44 +08:00
}
}
}
2020-09-11 08:44:15 +08:00
ACTOR Future<Void> checkRemoved(Reference<AsyncVar<ServerDBInfo>> db, uint64_t recoveryCount,
CommitProxyInterface myInterface) {
2017-05-26 04:48:44 +08:00
loop{
2020-09-11 08:44:15 +08:00
if (db->get().recoveryCount >= recoveryCount &&
!std::count(db->get().client.commitProxies.begin(), db->get().client.commitProxies.end(), myInterface)) {
throw worker_removed();
}
wait(db->onChange());
2017-05-26 04:48:44 +08:00
}
}
2020-09-11 08:44:15 +08:00
ACTOR Future<Void> commitProxyServer(CommitProxyInterface proxy, InitializeCommitProxyRequest req,
Reference<AsyncVar<ServerDBInfo>> db, std::string whitelistBinPaths) {
2017-05-26 04:48:44 +08:00
try {
2020-09-11 08:44:15 +08:00
state Future<Void> core =
commitProxyServerCore(proxy, req.master, db, req.recoveryCount, req.recoveryTransactionVersion,
req.firstProxy, whitelistBinPaths);
wait(core || checkRemoved(db, req.recoveryCount, proxy));
2017-05-26 04:48:44 +08:00
}
catch (Error& e) {
2020-09-11 08:44:15 +08:00
TraceEvent("CommitProxyTerminated", proxy.id()).error(e, true);
if (e.code() != error_code_worker_removed && e.code() != error_code_tlog_stopped &&
e.code() != error_code_master_tlog_failed && e.code() != error_code_coordinators_changed &&
e.code() != error_code_coordinated_state_conflict && e.code() != error_code_new_coordinators_timed_out) {
throw;
2017-05-26 04:48:44 +08:00
}
}
return Void();
2017-05-26 04:48:44 +08:00
}