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
|
|
|
|
*
|
2022-03-22 04:36:23 +08:00
|
|
|
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
|
2018-02-22 02:25:11 +08:00
|
|
|
*
|
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
|
2018-02-22 02:25:11 +08:00
|
|
|
*
|
2017-05-26 04:48:44 +08:00
|
|
|
* http://www.apache.org/licenses/LICENSE-2.0
|
2018-02-22 02:25:11 +08:00
|
|
|
*
|
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"
|
2019-02-18 07:41:16 +08:00
|
|
|
#include "fdbclient/NativeAPI.actor.h"
|
2019-02-28 07:40:33 +08:00
|
|
|
#include "fdbclient/SystemData.h"
|
2021-06-04 06:10:04 +08:00
|
|
|
#include "fdbclient/TransactionLineage.h"
|
2019-07-20 02:24:26 +08:00
|
|
|
#include "fdbrpc/sim_validation.h"
|
2018-10-20 01:30:13 +08:00
|
|
|
#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"
|
Clean up and rework the debugMutation API.
As a relatively unknown debugging tool for simulation tests, one could
have simulation print when a particular key is handled in various stages
of the commit process. This functionality was enabled by changing a 0
to a 1 in an #if, and changing a constant to the key in question.
As a proxy and storage server handle mutations, they call debugMutation
or debugKeyRange, which then checks against the mutation against the key
in question, and logs if they match. A mixture of printfs and
TraceEvents would then be emitted, and for this to actually be usable,
one also needs to comment out some particularly spammy debugKeyRange()
calls.
This PR reworks the API of debugMutation/debugKeyRange, pulls it out
into its own file, and trims what is logged by default into something
useful and understandable:
* debugMutation() now returns a TraceEvent, that one can add more details to before it is logged.
* Data distribution and storage server cleanup operations are no longer logged by default
2020-03-27 16:35:26 +08:00
|
|
|
#include "fdbserver/MutationTracking.h"
|
2020-08-06 03:20:52 +08:00
|
|
|
#include "fdbserver/ProxyCommitData.actor.h"
|
2020-08-30 03:35:31 +08:00
|
|
|
#include "fdbserver/RatekeeperInterface.h"
|
2019-07-20 02:24:26 +08:00
|
|
|
#include "fdbserver/RecoveryState.h"
|
2021-05-31 02:51:47 +08:00
|
|
|
#include "fdbserver/RestoreUtil.h"
|
2019-07-20 02:24:26 +08:00
|
|
|
#include "fdbserver/WaitFailure.h"
|
|
|
|
#include "fdbserver/WorkerInterface.actor.h"
|
|
|
|
#include "flow/ActorCollection.h"
|
2020-07-08 00:06:13 +08:00
|
|
|
#include "flow/IRandom.h"
|
2019-07-20 02:24:26 +08:00
|
|
|
#include "flow/Knobs.h"
|
2020-08-30 03:35:31 +08:00
|
|
|
#include "flow/Trace.h"
|
2020-07-08 00:06:13 +08:00
|
|
|
#include "flow/Tracing.h"
|
2020-07-26 12:44:10 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
#include "flow/actorcompiler.h" // This must be the last #include.
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-04-06 14:09:36 +08:00
|
|
|
ACTOR Future<Void> broadcastTxnRequest(TxnStateRequest req, int sendAmount, bool sendReply) {
|
|
|
|
state ReplyPromise<Void> reply = req.reply;
|
2021-03-11 02:06:03 +08:00
|
|
|
resetReply(req);
|
2020-04-06 14:09:36 +08:00
|
|
|
std::vector<Future<Void>> replies;
|
|
|
|
int currentStream = 0;
|
2020-04-11 04:45:16 +08:00
|
|
|
std::vector<Endpoint> broadcastEndpoints = req.broadcastInfo;
|
2021-03-11 02:06:03 +08:00
|
|
|
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++]);
|
2021-03-11 02:06:03 +08:00
|
|
|
while (currentStream < broadcastEndpoints.size() * (i + 1) / sendAmount) {
|
2020-04-11 04:45:16 +08:00
|
|
|
endpoints.push_back(broadcastEndpoints[currentStream++]);
|
2020-04-06 14:09:36 +08:00
|
|
|
}
|
2020-04-07 11:58:43 +08:00
|
|
|
req.broadcastInfo = endpoints;
|
2021-03-11 02:06:03 +08:00
|
|
|
replies.push_back(brokenPromiseToNever(cur.getReply(req)));
|
|
|
|
resetReply(req);
|
2020-04-06 14:09:36 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(waitForAll(replies));
|
|
|
|
if (sendReply) {
|
2020-04-06 14:09:36 +08:00
|
|
|
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);
|
2018-06-09 02:11:08 +08:00
|
|
|
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;
|
2021-09-17 08:42:34 +08:00
|
|
|
std::vector<ResolveTransactionBatchRequest> requests;
|
|
|
|
std::vector<std::vector<int>> transactionResolverMap;
|
|
|
|
std::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
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ResolutionRequestBuilder(ProxyCommitData* self,
|
|
|
|
Version version,
|
|
|
|
Version prevVersion,
|
|
|
|
Version lastReceivedVersion,
|
2020-07-08 00:06:13 +08:00
|
|
|
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) {
|
2021-03-11 02:06:03 +08:00
|
|
|
CommitTransactionRef*& out = outTr[resolver];
|
2017-05-26 04:48:44 +08:00
|
|
|
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;
|
|
|
|
}
|
|
|
|
|
2020-08-20 10:52:37 +08:00
|
|
|
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);
|
2021-03-11 02:06:03 +08:00
|
|
|
ASSERT(transactionNumberInBatch >= 0 && transactionNumberInBatch < 32768);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
bool isTXNStateTransaction = false;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& m : trIn.mutations) {
|
2017-05-26 04:48:44 +08:00
|
|
|
if (m.type == MutationRef::SetVersionstampedKey) {
|
2021-03-11 02:06:03 +08:00
|
|
|
transformVersionstampMutation(m, &MutationRef::param1, requests[0].version, transactionNumberInBatch);
|
|
|
|
trIn.write_conflict_ranges.push_back(requests[0].arena, singleKeyRange(m.param1, requests[0].arena));
|
2018-03-22 09:58:19 +08:00
|
|
|
} else if (m.type == MutationRef::SetVersionstampedValue) {
|
2021-03-11 02:06:03 +08:00
|
|
|
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);
|
|
|
|
}
|
|
|
|
}
|
2020-08-20 10:52:37 +08:00
|
|
|
if (isTXNStateTransaction && !trRequest.isLockAware()) {
|
2020-08-21 02:05:03 +08:00
|
|
|
// 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-08-20 10:52:37 +08:00
|
|
|
}
|
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) {
|
2020-05-22 19:44:09 +08:00
|
|
|
const auto& r = trIn.read_conflict_ranges[idx];
|
2021-03-11 02:06:03 +08:00
|
|
|
auto ranges = self->keyResolvers.intersectingRanges(r);
|
2017-05-26 04:48:44 +08:00
|
|
|
std::set<int> resolvers;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& ir : ranges) {
|
2017-05-26 04:48:44 +08:00
|
|
|
auto& version_resolver = ir.value();
|
2021-03-11 02:06:03 +08:00
|
|
|
for (int i = version_resolver.size() - 1; i >= 0; i--) {
|
2017-05-26 04:48:44 +08:00
|
|
|
resolvers.insert(version_resolver[i].second);
|
2021-03-11 02:06:03 +08:00
|
|
|
if (version_resolver[i].first < trIn.read_snapshot)
|
2017-05-26 04:48:44 +08:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
ASSERT(resolvers.size());
|
2020-05-22 20:24:15 +08:00
|
|
|
for (int resolver : resolvers) {
|
2021-03-11 02:06:03 +08:00
|
|
|
getOutTransaction(resolver, trIn.read_snapshot)
|
|
|
|
.read_conflict_ranges.push_back(requests[resolver].arena, r);
|
2020-05-22 19:44:09 +08:00
|
|
|
rCRIndexMap[resolver].push_back(idx);
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2020-05-22 19:44:09 +08:00
|
|
|
txReadConflictRangeIndexMap.push_back(std::move(rCRIndexMap));
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& r : trIn.write_conflict_ranges) {
|
|
|
|
auto ranges = self->keyResolvers.intersectingRanges(r);
|
2017-05-26 04:48:44 +08:00
|
|
|
std::set<int> resolvers;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& ir : ranges)
|
2017-05-26 04:48:44 +08:00
|
|
|
resolvers.insert(ir.value().back().second);
|
|
|
|
ASSERT(resolvers.size());
|
2021-03-11 02:06:03 +08:00
|
|
|
for (int resolver : resolvers)
|
|
|
|
getOutTransaction(resolver, trIn.read_snapshot)
|
|
|
|
.write_conflict_ranges.push_back(requests[resolver].arena, r);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
if (isTXNStateTransaction)
|
2021-03-11 02:06:03 +08:00
|
|
|
for (int r = 0; r < requests.size(); r++) {
|
|
|
|
int transactionNumberInRequest =
|
|
|
|
&getOutTransaction(r, trIn.read_snapshot) - requests[r].transactions.begin();
|
2017-05-26 04:48:44 +08:00
|
|
|
requests[r].txnStateTransactions.push_back(requests[r].arena, transactionNumberInRequest);
|
|
|
|
}
|
|
|
|
|
2021-09-17 08:42:34 +08:00
|
|
|
std::vector<int> resolversUsed;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (int r = 0; r < outTr.size(); r++)
|
2020-03-25 00:48:03 +08:00
|
|
|
if (outTr[r]) {
|
2017-05-26 04:48:44 +08:00
|
|
|
resolversUsed.push_back(r);
|
2020-03-25 00:48:03 +08:00
|
|
|
outTr[r]->report_conflicting_keys = trIn.report_conflicting_keys;
|
|
|
|
}
|
2020-06-20 10:32:30 +08:00
|
|
|
transactionResolverMap.emplace_back(std::move(resolversUsed));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> commitBatcher(ProxyCommitData* commitData,
|
|
|
|
PromiseStream<std::pair<std::vector<CommitTransactionRequest>, int>> out,
|
|
|
|
FutureStream<CommitTransactionRequest> in,
|
|
|
|
int desiredBytes,
|
|
|
|
int64_t memBytesLimit) {
|
2019-08-09 06:00:33 +08:00
|
|
|
wait(delayJittered(commitData->commitBatchInterval, TaskPriority::ProxyCommitBatcher));
|
2018-11-10 06:19:18 +08:00
|
|
|
|
|
|
|
state double lastBatch = 0;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
loop {
|
2018-11-10 06:19:18 +08:00
|
|
|
state Future<Void> timeout;
|
|
|
|
state std::vector<CommitTransactionRequest> batch;
|
|
|
|
state int batchBytes = 0;
|
2022-01-04 14:51:00 +08:00
|
|
|
// TODO: Enable this assertion (currently failing with gcc)
|
|
|
|
// static_assert(std::is_nothrow_move_constructible_v<CommitTransactionRequest>);
|
2018-11-10 06:19:18 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (SERVER_KNOBS->MAX_COMMIT_BATCH_INTERVAL <= 0) {
|
2018-11-10 06:19:18 +08:00
|
|
|
timeout = Never();
|
2021-03-11 02:06:03 +08:00
|
|
|
} else {
|
2019-06-25 17:47:35 +08:00
|
|
|
timeout = delayJittered(SERVER_KNOBS->MAX_COMMIT_BATCH_INTERVAL, TaskPriority::ProxyCommitBatcher);
|
2018-11-10 06:19:18 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
while (!timeout.isReady() &&
|
|
|
|
!(batch.size() == SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_COUNT_MAX || batchBytes >= desiredBytes)) {
|
|
|
|
choose {
|
2018-11-10 06:19:18 +08:00
|
|
|
when(CommitTransactionRequest req = waitNext(in)) {
|
2021-03-11 02:06:03 +08:00
|
|
|
// 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:19:18 +08:00
|
|
|
|
2018-11-10 06:25:40 +08:00
|
|
|
// Drop requests if memory is under severe pressure
|
2021-03-11 02:06:03 +08:00
|
|
|
if (commitData->commitBatchesMemBytesCount + bytes > memBytesLimit) {
|
2020-03-14 09:31:22 +08:00
|
|
|
++commitData->stats.txnCommitErrors;
|
2018-11-10 06:25:40 +08:00
|
|
|
req.reply.sendError(proxy_memory_limit_exceeded());
|
2021-03-11 02:06:03 +08:00
|
|
|
TraceEvent(SevWarnAlways, "ProxyCommitBatchMemoryThresholdExceeded")
|
|
|
|
.suppressFor(60)
|
|
|
|
.detail("MemBytesCount", commitData->commitBatchesMemBytesCount)
|
|
|
|
.detail("MemLimit", memBytesLimit);
|
2018-11-10 06:25:40 +08:00
|
|
|
continue;
|
|
|
|
}
|
2018-11-10 06:19:18 +08:00
|
|
|
|
2019-07-20 02:10:48 +08:00
|
|
|
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)
|
2019-07-20 02:10:48 +08:00
|
|
|
.detail("Client", req.reply.getEndpoint().getPrimaryAddress());
|
|
|
|
}
|
2018-11-10 06:25:40 +08:00
|
|
|
++commitData->stats.txnCommitIn;
|
2018-11-10 06:19:18 +08:00
|
|
|
|
2021-03-11 02:06:03 +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:19:18 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!batch.size()) {
|
|
|
|
if (now() - lastBatch > commitData->commitBatchInterval) {
|
|
|
|
timeout = delayJittered(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_FROM_IDLE,
|
|
|
|
TaskPriority::ProxyCommitBatcher);
|
|
|
|
} else {
|
|
|
|
timeout = delayJittered(commitData->commitBatchInterval - (now() - lastBatch),
|
|
|
|
TaskPriority::ProxyCommitBatcher);
|
2018-11-10 06:25:40 +08:00
|
|
|
}
|
2018-11-10 06:19:18 +08:00
|
|
|
}
|
2018-11-10 06:25:40 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if ((batchBytes + bytes > CLIENT_KNOBS->TRANSACTION_SIZE_LIMIT || req.firstInBatch()) &&
|
|
|
|
batch.size()) {
|
2020-04-07 04:45:13 +08:00
|
|
|
out.send({ std::move(batch), batchBytes });
|
2018-11-10 06:25:40 +08:00
|
|
|
lastBatch = now();
|
2019-06-25 17:47:35 +08:00
|
|
|
timeout = delayJittered(commitData->commitBatchInterval, TaskPriority::ProxyCommitBatcher);
|
2020-06-20 13:55:33 +08:00
|
|
|
batch.clear();
|
2018-11-10 06:25:40 +08:00
|
|
|
batchBytes = 0;
|
2018-11-10 06:19:18 +08:00
|
|
|
}
|
|
|
|
|
2018-11-10 06:25:40 +08:00
|
|
|
batch.push_back(req);
|
|
|
|
batchBytes += bytes;
|
|
|
|
commitData->commitBatchesMemBytesCount += bytes;
|
2018-11-10 06:19:18 +08:00
|
|
|
}
|
2018-11-11 05:04:24 +08:00
|
|
|
when(wait(timeout)) {}
|
2018-11-10 06:19:18 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
out.send({ std::move(batch), batchBytes });
|
|
|
|
lastBatch = now();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-09-17 08:42:34 +08:00
|
|
|
void createWhitelistBinPathVec(const std::string& binPath, std::vector<Standalone<StringRef>>& binPathVec) {
|
2019-04-03 20:27:11 +08:00
|
|
|
TraceEvent(SevDebug, "BinPathConverter").detail("Input", binPath);
|
2019-04-21 03:58:24 +08:00
|
|
|
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);
|
2019-04-21 03:58:24 +08:00
|
|
|
binPathVec.push_back(newElement);
|
|
|
|
}
|
2019-04-03 20:27:11 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2021-09-17 08:42:34 +08:00
|
|
|
bool isWhitelisted(const std::vector<Standalone<StringRef>>& binPathVec, StringRef binPath) {
|
2019-04-23 21:55:55 +08:00
|
|
|
TraceEvent("BinPath").detail("Value", binPath);
|
2019-04-21 03:58:24 +08:00
|
|
|
for (const auto& item : binPathVec) {
|
2019-04-23 21:55:55 +08:00
|
|
|
TraceEvent("Element").detail("Value", item);
|
2019-04-03 20:27:11 +08:00
|
|
|
}
|
2019-04-13 04:23:02 +08:00
|
|
|
return std::find(binPathVec.begin(), binPathVec.end(), binPath) != binPathVec.end();
|
2019-04-03 20:27:11 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> addBackupMutations(ProxyCommitData* self,
|
|
|
|
const std::map<Key, MutationListRef>* logRangeMutations,
|
|
|
|
LogPushData* toCommit,
|
|
|
|
Version commitVersion,
|
|
|
|
double* computeDuration,
|
2021-01-22 08:27:21 +08:00
|
|
|
double* computeStart) {
|
|
|
|
state std::map<Key, MutationListRef>::const_iterator logRangeMutation = logRangeMutations->cbegin();
|
2019-10-22 02:43:46 +08:00
|
|
|
state int32_t version = commitVersion / CLIENT_KNOBS->LOG_RANGE_BLOCK_SIZE;
|
|
|
|
state int yieldBytes = 0;
|
2019-11-15 07:49:56 +08:00
|
|
|
state BinaryWriter valueWriter(Unversioned());
|
2020-09-01 01:39:07 +08:00
|
|
|
|
2020-08-28 07:16:05 +08:00
|
|
|
toCommit->addTransactionInfo(SpanID());
|
|
|
|
|
2019-08-31 07:16:19 +08:00
|
|
|
// Serialize the log range mutations within the map
|
2021-01-22 08:27:21 +08:00
|
|
|
for (; logRangeMutation != logRangeMutations->cend(); ++logRangeMutation) {
|
2021-03-11 02:06:03 +08:00
|
|
|
// FIXME: this is re-implementing the serialize function of MutationListRef in order to have a yield
|
2020-05-23 08:14:21 +08:00
|
|
|
valueWriter = BinaryWriter(IncludeVersion(ProtocolVersion::withBackupMutations()));
|
2019-11-15 07:49:56 +08:00
|
|
|
valueWriter << logRangeMutation->second.totalSize();
|
|
|
|
|
|
|
|
state MutationListRef::Blob* blobIter = logRangeMutation->second.blob_begin;
|
2021-03-11 02:06:03 +08:00
|
|
|
while (blobIter) {
|
|
|
|
if (yieldBytes > SERVER_KNOBS->DESIRED_TOTAL_BYTES) {
|
2019-11-15 07:49:56 +08:00
|
|
|
yieldBytes = 0;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (g_network->check_yield(TaskPriority::ProxyCommitYield1)) {
|
2020-02-22 11:09:16 +08:00
|
|
|
*computeDuration += g_network->timer() - *computeStart;
|
|
|
|
wait(delay(0, TaskPriority::ProxyCommitYield1));
|
|
|
|
*computeStart = g_network->timer();
|
|
|
|
}
|
2019-11-15 07:49:56 +08:00
|
|
|
}
|
|
|
|
valueWriter.serializeBytes(blobIter->data);
|
|
|
|
yieldBytes += blobIter->data.size();
|
|
|
|
blobIter = blobIter->next;
|
2019-10-22 02:43:46 +08:00
|
|
|
}
|
|
|
|
|
2019-11-15 07:49:56 +08:00
|
|
|
Key val = valueWriter.toValue();
|
2020-07-26 12:44:10 +08:00
|
|
|
|
2019-08-31 07:16:19 +08:00
|
|
|
BinaryWriter wr(Unversioned());
|
|
|
|
|
|
|
|
// Serialize the log destination
|
2021-03-11 02:06:03 +08:00
|
|
|
wr.serializeBytes(logRangeMutation->first);
|
2019-08-31 07:16:19 +08:00
|
|
|
|
|
|
|
// 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;
|
2019-08-31 07:16:19 +08:00
|
|
|
backupMutation.type = MutationRef::SetValue;
|
2020-08-19 05:18:50 +08:00
|
|
|
uint32_t* partBuffer = nullptr;
|
2019-08-31 07:16:19 +08:00
|
|
|
|
|
|
|
for (int part = 0; part * CLIENT_KNOBS->MUTATION_BLOCK_SIZE < val.size(); part++) {
|
2019-10-22 02:43:46 +08:00
|
|
|
|
2019-08-31 07:16:19 +08:00
|
|
|
// Assign the second parameter as the part
|
2021-03-11 02:06:03 +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));
|
2019-08-31 07:16:19 +08:00
|
|
|
|
|
|
|
// 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
|
2021-03-11 02:06:03 +08:00
|
|
|
partBuffer = (uint32_t*)((char*)wr.getData() + wr.getLength() - sizeof(uint32_t));
|
|
|
|
} else {
|
2019-08-31 07:16:19 +08:00
|
|
|
*partBuffer = bigEndian32(part);
|
|
|
|
}
|
|
|
|
|
|
|
|
// Define the mutation type and and location
|
|
|
|
backupMutation.param1 = wr.toValue();
|
2021-03-11 02:06:03 +08:00
|
|
|
ASSERT(backupMutation.param1.startsWith(
|
|
|
|
logRangeMutation->first)); // We are writing into the configured destination
|
2020-07-26 12:44:10 +08:00
|
|
|
|
2019-08-31 07:16:19 +08:00
|
|
|
auto& tags = self->tagsForKey(backupMutation.param1);
|
|
|
|
toCommit->addTags(tags);
|
2020-08-28 07:16:05 +08:00
|
|
|
toCommit->writeTypedMessage(backupMutation);
|
2019-08-31 07:16:19 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// if (DEBUG_MUTATION("BackupProxyCommit", commitVersion, backupMutation)) {
|
|
|
|
// 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",
|
2021-03-11 06:42:17 +08:00
|
|
|
// bigEndian32(part)).detail("PartData", backupMutation.param1);
|
2021-03-11 02:06:03 +08:00
|
|
|
// }
|
2019-08-31 07:16:19 +08:00
|
|
|
}
|
|
|
|
}
|
2019-10-22 02:43:46 +08:00
|
|
|
return Void();
|
2019-08-31 07:16:19 +08:00
|
|
|
}
|
|
|
|
|
2020-01-24 10:31:51 +08:00
|
|
|
ACTOR Future<Void> releaseResolvingAfter(ProxyCommitData* self, Future<Void> releaseDelay, int64_t localBatchNumber) {
|
|
|
|
wait(releaseDelay);
|
2021-03-11 02:06:03 +08:00
|
|
|
ASSERT(self->latestLocalCommitBatchResolving.get() == localBatchNumber - 1);
|
2020-01-24 10:31:51 +08:00
|
|
|
self->latestLocalCommitBatchResolving.set(localBatchNumber);
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-08-06 00:17:37 +08:00
|
|
|
ACTOR static Future<ResolveTransactionBatchReply> trackResolutionMetrics(Reference<Histogram> dist,
|
|
|
|
Future<ResolveTransactionBatchReply> in) {
|
|
|
|
state double startTime = now();
|
|
|
|
ResolveTransactionBatchReply reply = wait(in);
|
|
|
|
dist->sampleSeconds(now() - startTime);
|
|
|
|
return reply;
|
|
|
|
}
|
|
|
|
|
2022-02-26 03:18:42 +08:00
|
|
|
ErrorOr<Optional<TenantMapEntry>> getTenantEntry(ProxyCommitData* commitData,
|
|
|
|
Optional<TenantNameRef> tenant,
|
2022-03-04 07:40:38 +08:00
|
|
|
Optional<int64_t> tenantId,
|
2022-03-09 05:33:56 +08:00
|
|
|
bool logOnFailure) {
|
2022-02-26 03:18:42 +08:00
|
|
|
if (tenant.present()) {
|
2022-03-09 05:33:56 +08:00
|
|
|
auto itr = commitData->tenantMap.find(tenant.get());
|
|
|
|
if (itr == commitData->tenantMap.end()) {
|
|
|
|
if (logOnFailure) {
|
|
|
|
TraceEvent(SevWarn, "CommitProxyUnknownTenant", commitData->dbgid).detail("Tenant", tenant.get());
|
2022-02-26 03:18:42 +08:00
|
|
|
}
|
|
|
|
|
2022-03-05 04:22:05 +08:00
|
|
|
return unknown_tenant();
|
2022-03-09 05:33:56 +08:00
|
|
|
} else if (tenantId.present() && tenantId.get() != itr->second.id) {
|
|
|
|
if (logOnFailure) {
|
2022-03-04 07:40:38 +08:00
|
|
|
TraceEvent(SevWarn, "CommitProxyTenantIdMismatch", commitData->dbgid)
|
|
|
|
.detail("Tenant", tenant.get())
|
|
|
|
.detail("TenantId", tenantId)
|
2022-03-09 05:33:56 +08:00
|
|
|
.detail("ExistingId", itr->second.id);
|
2022-03-04 07:40:38 +08:00
|
|
|
}
|
|
|
|
|
2022-03-05 04:22:05 +08:00
|
|
|
return unknown_tenant();
|
2022-02-26 03:18:42 +08:00
|
|
|
}
|
|
|
|
|
2022-03-09 05:33:56 +08:00
|
|
|
return ErrorOr<Optional<TenantMapEntry>>(Optional<TenantMapEntry>(itr->second));
|
2022-02-26 03:18:42 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
return Optional<TenantMapEntry>();
|
|
|
|
}
|
|
|
|
|
2020-07-26 12:44:10 +08:00
|
|
|
namespace CommitBatch {
|
|
|
|
|
2020-08-06 05:11:43 +08:00
|
|
|
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;
|
2021-01-22 06:10:48 +08:00
|
|
|
bool rejected = false; // If rejected due to long queue length
|
2020-07-26 12:44:10 +08:00
|
|
|
|
|
|
|
int64_t localBatchNumber;
|
|
|
|
LogPushData toCommit;
|
|
|
|
|
|
|
|
int batchOperations = 0;
|
|
|
|
|
2020-08-28 07:16:05 +08:00
|
|
|
Span span;
|
2020-07-26 12:44:10 +08:00
|
|
|
|
2020-01-27 11:47:12 +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;
|
2018-06-02 06:21:40 +08:00
|
|
|
|
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;
|
2018-06-02 06:21:40 +08:00
|
|
|
|
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;
|
|
|
|
|
2020-08-06 05:11:43 +08:00
|
|
|
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();
|
|
|
|
};
|
2018-06-02 06:21:40 +08:00
|
|
|
|
2020-08-06 05:11:43 +08:00
|
|
|
CommitBatchContext::CommitBatchContext(ProxyCommitData* const pProxyCommitData_,
|
|
|
|
const std::vector<CommitTransactionRequest>* trs_,
|
|
|
|
const int currentBatchMemBytesCount)
|
|
|
|
:
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-08-06 05:11:43 +08:00
|
|
|
pProxyCommitData(pProxyCommitData_), trs(std::move(*const_cast<std::vector<CommitTransactionRequest>*>(trs_))),
|
|
|
|
currentBatchMemBytesCount(currentBatchMemBytesCount),
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-08-06 05:11:43 +08:00
|
|
|
startTime(g_network->now()),
|
2020-07-26 12:44:10 +08:00
|
|
|
|
2020-08-06 05:11:43 +08:00
|
|
|
localBatchNumber(++pProxyCommitData->localCommitBatchesStarted), toCommit(pProxyCommitData->logSystem),
|
|
|
|
|
2021-07-23 13:48:27 +08:00
|
|
|
span("MP:commitBatch"_loc), committed(trs.size()) {
|
2020-07-26 12:44:10 +08:00
|
|
|
|
|
|
|
evaluateBatchSize();
|
|
|
|
|
|
|
|
if (batchOperations != 0) {
|
2021-03-11 02:06:03 +08:00
|
|
|
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)));
|
2020-07-26 12:44:10 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// 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);
|
|
|
|
}
|
|
|
|
|
2020-08-06 05:11:43 +08:00
|
|
|
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()) {
|
2019-05-11 05:01:52 +08:00
|
|
|
debugID = nondeterministicRandom()->randomUniqueID();
|
2020-07-26 12:44:10 +08:00
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +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
|
|
|
|
2020-08-06 05:11:43 +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
|
|
|
|
2021-01-22 02:19:53 +08:00
|
|
|
// Try to identify recovery transaction and backup's apply mutations (blind writes).
|
|
|
|
// Both cannot be rejected and are approximated by looking at first mutation
|
|
|
|
// starting with 0xff.
|
|
|
|
bool canReject(const std::vector<CommitTransactionRequest>& trs) {
|
|
|
|
for (const auto& tr : trs) {
|
2021-03-11 02:06:03 +08:00
|
|
|
if (tr.transaction.mutations.empty())
|
|
|
|
continue;
|
2022-02-26 03:18:42 +08:00
|
|
|
if (!tr.tenantInfo.name.present() && (tr.transaction.mutations[0].param1.startsWith(LiteralStringRef("\xff")) ||
|
|
|
|
tr.transaction.read_conflict_ranges.empty())) {
|
2021-01-22 02:19:53 +08:00
|
|
|
return false;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2020-08-06 05:11:43 +08:00
|
|
|
ACTOR Future<Void> preresolutionProcessing(CommitBatchContext* self) {
|
2019-10-05 04:23:38 +08:00
|
|
|
|
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;
|
2020-08-29 03:02:51 +08:00
|
|
|
state Span span("MP:preresolutionProcessing"_loc, self->span.context);
|
2021-01-22 02:19:53 +08:00
|
|
|
state double timeStart = now();
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-10-20 13:51:56 +08:00
|
|
|
if (self->localBatchNumber - self->pProxyCommitData->latestLocalCommitBatchResolving.get() >
|
|
|
|
SERVER_KNOBS->RESET_MASTER_BATCHES &&
|
|
|
|
now() - self->pProxyCommitData->lastMasterReset > SERVER_KNOBS->RESET_MASTER_DELAY) {
|
2021-03-17 05:00:11 +08:00
|
|
|
TraceEvent(SevWarnAlways, "ResetMasterNetwork", self->pProxyCommitData->dbgid)
|
2020-10-20 13:51:56 +08:00
|
|
|
.detail("CurrentBatch", self->localBatchNumber)
|
|
|
|
.detail("InProcessBatch", self->pProxyCommitData->latestLocalCommitBatchResolving.get());
|
|
|
|
FlowTransport::transport().resetConnection(self->pProxyCommitData->master.address());
|
|
|
|
self->pProxyCommitData->lastMasterReset = now();
|
|
|
|
}
|
|
|
|
|
2020-07-26 12:44:10 +08:00
|
|
|
// Pre-resolution the commits
|
2020-11-12 05:07:54 +08:00
|
|
|
TEST(pProxyCommitData->latestLocalCommitBatchResolving.get() < localBatchNumber - 1); // Wait for local batch
|
2020-07-26 12:44:10 +08:00
|
|
|
wait(pProxyCommitData->latestLocalCommitBatchResolving.whenAtLeast(localBatchNumber - 1));
|
2021-01-22 02:19:53 +08:00
|
|
|
double queuingDelay = g_network->now() - timeStart;
|
2021-08-06 00:17:37 +08:00
|
|
|
pProxyCommitData->stats.commitBatchQueuingDist->sampleSeconds(queuingDelay);
|
2021-01-22 02:19:53 +08:00
|
|
|
if ((queuingDelay > (double)SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS / SERVER_KNOBS->VERSIONS_PER_SECOND ||
|
|
|
|
(g_network->isSimulated() && BUGGIFY_WITH_PROB(0.01))) &&
|
|
|
|
SERVER_KNOBS->PROXY_REJECT_BATCH_QUEUED_TOO_LONG && canReject(trs)) {
|
|
|
|
// Disabled for the recovery transaction. otherwise, recovery can't finish and keeps doing more recoveries.
|
|
|
|
TEST(true); // Reject transactions in the batch
|
2021-01-22 03:11:21 +08:00
|
|
|
TraceEvent(SevWarnAlways, "ProxyReject", pProxyCommitData->dbgid)
|
2021-01-22 02:19:53 +08:00
|
|
|
.suppressFor(0.1)
|
|
|
|
.detail("QDelay", queuingDelay)
|
|
|
|
.detail("Transactions", trs.size())
|
|
|
|
.detail("BatchNumber", localBatchNumber);
|
2021-01-22 03:11:21 +08:00
|
|
|
ASSERT(pProxyCommitData->latestLocalCommitBatchResolving.get() == localBatchNumber - 1);
|
|
|
|
pProxyCommitData->latestLocalCommitBatchResolving.set(localBatchNumber);
|
2021-01-22 02:19:53 +08:00
|
|
|
|
2021-01-22 03:11:21 +08:00
|
|
|
wait(pProxyCommitData->latestLocalCommitBatchLogging.whenAtLeast(localBatchNumber - 1));
|
|
|
|
ASSERT(pProxyCommitData->latestLocalCommitBatchLogging.get() == localBatchNumber - 1);
|
|
|
|
pProxyCommitData->latestLocalCommitBatchLogging.set(localBatchNumber);
|
2021-01-22 02:19:53 +08:00
|
|
|
for (const auto& tr : trs) {
|
|
|
|
tr.reply.sendError(transaction_too_old());
|
|
|
|
}
|
2021-01-22 03:11:21 +08:00
|
|
|
++pProxyCommitData->stats.commitBatchOut;
|
|
|
|
pProxyCommitData->stats.txnCommitOut += trs.size();
|
2021-02-08 06:56:34 +08:00
|
|
|
pProxyCommitData->stats.txnRejectedForQueuedTooLong += trs.size();
|
2021-01-22 06:10:48 +08:00
|
|
|
self->rejected = true;
|
2021-01-22 02:19:53 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
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()) {
|
2021-03-11 02:06:03 +08:00
|
|
|
g_traceBatch.addEvent(
|
|
|
|
"CommitDebug", debugID.get().first(), "CommitProxyServer.commitBatch.GettingCommitVersion");
|
2020-07-26 12:44:10 +08:00
|
|
|
}
|
2019-10-05 04:23:38 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
GetCommitVersionRequest req(span.context,
|
|
|
|
pProxyCommitData->commitVersionRequestNumber++,
|
|
|
|
pProxyCommitData->mostRecentProcessedRequestNumber,
|
|
|
|
pProxyCommitData->dbgid);
|
2021-08-06 00:17:37 +08:00
|
|
|
state double beforeGettingCommitVersion = now();
|
2020-07-26 12:44:10 +08:00
|
|
|
GetCommitVersionReply versionReply = wait(brokenPromiseToNever(
|
2021-03-11 02:06:03 +08:00
|
|
|
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;
|
2021-08-06 00:17:37 +08:00
|
|
|
pProxyCommitData->stats.getCommitVersionDist->sampleSeconds(now() - beforeGettingCommitVersion);
|
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
|
|
|
|
2022-01-07 04:15:51 +08:00
|
|
|
//TraceEvent("CPGetVersion", pProxyCommitData->dbgid).detail("Master", pProxyCommitData->master.id().toString()).detail("CommitVersion", self->commitVersion).detail("PrvVersion", self->prevVersion);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto it : versionReply.resolverChanges) {
|
2020-07-26 12:44:10 +08:00
|
|
|
auto rs = pProxyCommitData->keyResolvers.modify(it.range);
|
2021-03-11 02:06:03 +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
|
|
|
|
2020-08-06 05:11:43 +08:00
|
|
|
ACTOR Future<Void> getResolution(CommitBatchContext* self) {
|
2021-08-06 00:17:37 +08:00
|
|
|
state double resolutionStart = now();
|
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;
|
2020-08-29 03:02:51 +08:00
|
|
|
state Span span("MP:getResolution"_loc, self->span.context);
|
2020-07-26 12:44:10 +08:00
|
|
|
|
|
|
|
ResolutionRequestBuilder requests(
|
2022-02-26 03:18:42 +08:00
|
|
|
pProxyCommitData, self->commitVersion, self->prevVersion, pProxyCommitData->version.get(), 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++) {
|
2020-08-20 10:52:37 +08:00
|
|
|
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);
|
2021-03-11 02:06:03 +08:00
|
|
|
// for(auto& m : trs[t].transaction.mutations)
|
|
|
|
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++)
|
2021-03-11 02:06:03 +08:00
|
|
|
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;
|
2021-08-06 00:17:37 +08:00
|
|
|
replies.push_back(trackResolutionMetrics(pProxyCommitData->stats.resolverDist[r],
|
|
|
|
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);
|
2021-03-11 02:06:03 +08:00
|
|
|
self->releaseFuture = releaseResolvingAfter(pProxyCommitData, self->releaseDelay, self->localBatchNumber);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-10-20 13:51:56 +08:00
|
|
|
if (self->localBatchNumber - self->pProxyCommitData->latestLocalCommitBatchLogging.get() >
|
|
|
|
SERVER_KNOBS->RESET_RESOLVER_BATCHES &&
|
|
|
|
now() - self->pProxyCommitData->lastResolverReset > SERVER_KNOBS->RESET_RESOLVER_DELAY) {
|
|
|
|
for (int r = 0; r < self->pProxyCommitData->resolvers.size(); r++) {
|
2021-03-17 05:00:11 +08:00
|
|
|
TraceEvent(SevWarnAlways, "ResetResolverNetwork", self->pProxyCommitData->dbgid)
|
|
|
|
.detail("PeerAddr", self->pProxyCommitData->resolvers[r].address())
|
|
|
|
.detail("CurrentBatch", self->localBatchNumber)
|
|
|
|
.detail("InProcessBatch", self->pProxyCommitData->latestLocalCommitBatchLogging.get());
|
2020-10-20 13:51:56 +08:00
|
|
|
FlowTransport::transport().resetConnection(self->pProxyCommitData->resolvers[r].address());
|
|
|
|
}
|
|
|
|
self->pProxyCommitData->lastResolverReset = now();
|
|
|
|
}
|
|
|
|
|
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
|
|
|
|
2021-08-06 00:17:37 +08:00
|
|
|
self->pProxyCommitData->stats.resolutionDist->sampleSeconds(now() - resolutionStart);
|
2020-07-26 12:44:10 +08:00
|
|
|
if (self->debugID.present()) {
|
2021-03-11 02:06:03 +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
|
|
|
|
2021-03-11 02:06:03 +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());
|
2021-03-11 02:06:03 +08:00
|
|
|
for (int s = 0; s < resolution[r].stateMutations.size(); s++) {
|
2020-07-29 15:42:52 +08:00
|
|
|
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
|
2020-08-06 05:11:43 +08:00
|
|
|
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++) {
|
2021-03-11 02:06:03 +08:00
|
|
|
// 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++) {
|
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++)
|
2021-03-11 02:06:03 +08:00
|
|
|
committed =
|
|
|
|
committed && self->resolution[resolver].stateMutations[versionIndex][transactionIndex].committed;
|
2020-08-04 12:43:16 +08:00
|
|
|
if (committed) {
|
2021-03-11 02:06:03 +08:00
|
|
|
applyMetadataMutations(SpanID(),
|
|
|
|
*self->pProxyCommitData,
|
|
|
|
self->arena,
|
|
|
|
self->pProxyCommitData->logSystem,
|
2020-08-04 12:43:16 +08:00
|
|
|
self->resolution[0].stateMutations[versionIndex][transactionIndex].mutations,
|
2021-03-11 02:06:03 +08:00
|
|
|
/* pToCommit= */ nullptr,
|
|
|
|
self->forceRecovery,
|
2022-02-19 13:22:31 +08:00
|
|
|
/* version= */ self->commitVersion,
|
2021-03-11 02:06:03 +08:00
|
|
|
/* popVersion= */ 0,
|
|
|
|
/* initialCommit */ false);
|
2020-08-04 12:43:16 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +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
|
|
|
|
2021-03-11 02:06:03 +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
|
2020-08-06 05:11:43 +08:00
|
|
|
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());
|
2020-03-25 00:48:03 +08:00
|
|
|
// 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
|
|
|
|
2021-03-11 02:06:03 +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++) {
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->committed[t] == ConflictBatch::TransactionCommitted) {
|
2018-08-22 12:09:50 +08:00
|
|
|
bool foundSystem = false;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& m : trs[t].transaction.mutations) {
|
|
|
|
if ((m.type == MutationRef::ClearRange ? m.param2 : m.param1) >= nonMetadataSystemKeys.end) {
|
2018-08-22 12:09:50 +08:00
|
|
|
foundSystem = true;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!foundSystem) {
|
2020-07-26 12:55:22 +08:00
|
|
|
self->committed[t] = ConflictBatch::TransactionConflict;
|
2018-08-22 12:09:50 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2020-07-29 15:42:52 +08:00
|
|
|
}
|
2018-08-22 12:09:50 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// This first pass through committed transactions deals with "metadata" effects (modifications of txnStateStore, changes
|
|
|
|
// to storage servers' responsibilities)
|
2020-08-06 05:11:43 +08:00
|
|
|
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())) {
|
2022-03-09 05:33:56 +08:00
|
|
|
ErrorOr<Optional<TenantMapEntry>> result = getTenantEntry(
|
|
|
|
pProxyCommitData, trs[t].tenantInfo.name.castTo<TenantNameRef>(), trs[t].tenantInfo.tenantId, true);
|
2022-02-19 13:22:31 +08:00
|
|
|
|
|
|
|
if (result.isError()) {
|
|
|
|
self->committed[t] = ConflictBatch::TransactionTenantFailure;
|
|
|
|
trs[t].reply.sendError(result.getError());
|
|
|
|
} else {
|
|
|
|
self->commitCount++;
|
|
|
|
applyMetadataMutations(trs[t].spanContext,
|
|
|
|
*pProxyCommitData,
|
|
|
|
self->arena,
|
|
|
|
pProxyCommitData->logSystem,
|
|
|
|
trs[t].transaction.mutations,
|
|
|
|
&self->toCommit,
|
|
|
|
self->forceRecovery,
|
|
|
|
self->commitVersion,
|
|
|
|
self->commitVersion + 1,
|
|
|
|
/* initialCommit= */ false);
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
if (self->firstStateMutations) {
|
2020-07-26 12:55:22 +08:00
|
|
|
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();
|
2019-03-01 09:45:00 +08:00
|
|
|
|
2020-07-26 12:55:22 +08:00
|
|
|
auto fcm = pProxyCommitData->logAdapter->getCommitMessage();
|
|
|
|
self->storeCommits.emplace_back(fcm, pProxyCommitData->txnStateStore->commit());
|
2022-02-26 03:18:42 +08:00
|
|
|
pProxyCommitData->version.set(self->commitVersion);
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!pProxyCommitData->validState.isSet())
|
|
|
|
pProxyCommitData->validState.send(Void());
|
2020-07-26 12:55:22 +08:00
|
|
|
ASSERT(self->commitVersion);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!self->isMyFirstBatch &&
|
|
|
|
pProxyCommitData->txnStateStore->readValue(coordinatorsKey).get().get() != self->oldCoordinators.get()) {
|
2022-01-07 04:15:51 +08:00
|
|
|
wait(brokenPromiseToNever(pProxyCommitData->db->get().clusterInterface.changeCoordinators.getReply(
|
2021-03-11 02:06:03 +08:00
|
|
|
ChangeCoordinatorsRequest(pProxyCommitData->txnStateStore->readValue(coordinatorsKey).get().get()))));
|
|
|
|
ASSERT(false); // ChangeCoordinatorsRequest should always throw
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
2020-07-29 15:42:52 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
/// This second pass through committed transactions assigns the actual mutations to the appropriate storage servers'
|
|
|
|
/// tags
|
2020-08-06 05:11:43 +08:00
|
|
|
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;
|
2019-08-09 06:00:33 +08:00
|
|
|
|
2020-07-26 12:55:22 +08:00
|
|
|
for (; self->transactionNum < trs.size(); self->transactionNum++) {
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!(self->committed[self->transactionNum] == ConflictBatch::TransactionCommitted &&
|
|
|
|
(!self->locked || trs[self->transactionNum].isLockAware()))) {
|
2020-07-29 15:42:52 +08:00
|
|
|
continue;
|
|
|
|
}
|
2018-10-19 06:26:00 +08:00
|
|
|
|
2020-08-19 12:23:23 +08:00
|
|
|
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;
|
2020-08-28 07:16:05 +08:00
|
|
|
|
|
|
|
self->toCommit.addTransactionInfo(trs[self->transactionNum].spanContext);
|
|
|
|
|
2020-07-29 15:42:52 +08:00
|
|
|
for (; mutationNum < pMutations->size(); mutationNum++) {
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->yieldBytes > SERVER_KNOBS->DESIRED_TOTAL_BYTES) {
|
2020-07-29 15:42:52 +08:00
|
|
|
self->yieldBytes = 0;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (g_network->check_yield(TaskPriority::ProxyCommitYield1)) {
|
2020-07-29 15:42:52 +08:00
|
|
|
self->computeDuration += g_network->timer() - self->computeStart;
|
|
|
|
wait(delay(0, TaskPriority::ProxyCommitYield1));
|
|
|
|
self->computeStart = g_network->timer();
|
2018-10-19 06:26:00 +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
|
|
|
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.
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (isSingleKeyMutation((MutationRef::Type)m.type)) {
|
2020-07-29 15:42:52 +08:00
|
|
|
auto& tags = pProxyCommitData->tagsForKey(m.param1);
|
|
|
|
|
2020-08-21 07:00:45 +08:00
|
|
|
// 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;
|
2020-08-15 02:18:54 +08:00
|
|
|
|
2020-08-30 03:35:31 +08:00
|
|
|
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
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (pProxyCommitData->singleKeyMutationEvent->enabled) {
|
2020-07-29 15:42:52 +08:00
|
|
|
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
|
|
|
|
2021-08-28 08:07:47 +08:00
|
|
|
DEBUG_MUTATION("ProxyCommit", self->commitVersion, m, pProxyCommitData->dbgid).detail("To", tags);
|
2020-07-29 15:42:52 +08:00
|
|
|
self->toCommit.addTags(tags);
|
2021-03-11 02:06:03 +08:00
|
|
|
if (pProxyCommitData->cacheInfo[m.param1]) {
|
2020-07-29 15:42:52 +08:00
|
|
|
self->toCommit.addTag(cacheTag);
|
|
|
|
}
|
2020-08-28 07:16:05 +08:00
|
|
|
self->toCommit.writeTypedMessage(m);
|
2021-03-11 02:06:03 +08:00
|
|
|
} else if (m.type == MutationRef::ClearRange) {
|
2020-07-29 15:42:52 +08:00
|
|
|
KeyRangeRef clearRange(KeyRangeRef(m.param1, m.param2));
|
|
|
|
auto ranges = pProxyCommitData->keyInfo.intersectingRanges(clearRange);
|
|
|
|
auto firstRange = ranges.begin();
|
|
|
|
++firstRange;
|
|
|
|
if (firstRange == ranges.end()) {
|
|
|
|
// Fast path
|
2021-08-10 16:52:36 +08:00
|
|
|
DEBUG_MUTATION("ProxyCommit", self->commitVersion, m, pProxyCommitData->dbgid)
|
2021-08-10 14:29:09 +08:00
|
|
|
.detail("To", ranges.begin().value().tags);
|
2020-07-29 15:42:52 +08:00
|
|
|
|
|
|
|
ranges.begin().value().populateTags();
|
|
|
|
self->toCommit.addTags(ranges.begin().value().tags);
|
2020-08-15 02:18:54 +08:00
|
|
|
|
2020-08-11 08:14:55 +08:00
|
|
|
// check whether clear is sampled
|
2020-08-30 03:35:31 +08:00
|
|
|
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();
|
2021-03-11 02:06:03 +08:00
|
|
|
pProxyCommitData->updateSSTagCost(
|
|
|
|
id, trs[self->transactionNum].tagSet.get(), m, trCost->get().clearIdxCosts[0].second);
|
2020-08-11 08:14:55 +08:00
|
|
|
}
|
2020-08-15 02:18:54 +08:00
|
|
|
trCost->get().clearIdxCosts.pop_front();
|
2019-11-13 05:01:29 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
} else {
|
|
|
|
TEST(true); // A clear range extends past a shard boundary
|
2020-07-29 15:42:52 +08:00
|
|
|
std::set<Tag> allSources;
|
|
|
|
for (auto r : ranges) {
|
|
|
|
r.value().populateTags();
|
|
|
|
allSources.insert(r.value().tags.begin(), r.value().tags.end());
|
2020-08-15 02:18:54 +08:00
|
|
|
|
2020-08-01 14:14:52 +08:00
|
|
|
// check whether clear is sampled
|
2020-08-30 03:35:31 +08:00
|
|
|
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();
|
2021-03-11 02:06:03 +08:00
|
|
|
pProxyCommitData->updateSSTagCost(id,
|
|
|
|
trs[self->transactionNum].tagSet.get(),
|
|
|
|
m,
|
2020-08-30 03:35:31 +08:00
|
|
|
trCost->get().clearIdxCosts[0].second);
|
2020-08-01 14:48:40 +08:00
|
|
|
}
|
2020-08-15 02:18:54 +08:00
|
|
|
trCost->get().clearIdxCosts.pop_front();
|
2020-08-01 14:48:40 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2021-08-10 16:52:36 +08:00
|
|
|
DEBUG_MUTATION("ProxyCommit", self->commitVersion, m, pProxyCommitData->dbgid)
|
2021-08-10 14:29:09 +08:00
|
|
|
.detail("To", allSources);
|
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
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (pProxyCommitData->needsCacheTag(clearRange)) {
|
2020-07-29 15:42:52 +08:00
|
|
|
self->toCommit.addTag(cacheTag);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2020-08-28 07:16:05 +08:00
|
|
|
self->toCommit.writeTypedMessage(m);
|
2020-07-29 15:42:52 +08:00
|
|
|
} else {
|
|
|
|
UNREACHABLE();
|
|
|
|
}
|
2019-03-01 09:45:00 +08:00
|
|
|
|
2020-07-29 15:42:52 +08:00
|
|
|
// Check on backing up key, if backup ranges are defined and a normal key
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!(pProxyCommitData->vecBackupKeys.size() > 1 &&
|
|
|
|
(normalKeys.contains(m.param1) || m.param1 == metadataVersionKey))) {
|
2020-07-29 15:42:52 +08:00
|
|
|
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
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
} else {
|
2020-07-29 15:42:52 +08:00
|
|
|
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
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto backupRange : pProxyCommitData->vecBackupKeys.intersectingRanges(mutationRange)) {
|
2020-07-29 15:42:52 +08:00
|
|
|
// Get the backup sub range
|
2021-03-11 02:06:03 +08:00
|
|
|
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
|
2021-03-11 02:06:03 +08:00
|
|
|
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()) {
|
2021-03-11 02:06:03 +08:00
|
|
|
self->logRangeMutations[backupName].push_back_deep(self->logRangeMutationsArena,
|
|
|
|
backupMutation);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2020-08-21 07:00:45 +08:00
|
|
|
|
2020-08-30 03:35:31 +08:00
|
|
|
if (checkSample) {
|
2020-08-21 07:00:45 +08:00
|
|
|
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();
|
|
|
|
}
|
|
|
|
|
2020-08-06 05:11:43 +08:00
|
|
|
ACTOR Future<Void> postResolution(CommitBatchContext* self) {
|
2021-08-06 00:17:37 +08:00
|
|
|
state double postResolutionStart = now();
|
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;
|
2020-08-29 03:02:51 +08:00
|
|
|
state Span span("MP:postResolution"_loc, self->span.context);
|
2020-07-29 15:42:52 +08:00
|
|
|
|
2021-04-14 01:48:52 +08:00
|
|
|
bool queuedCommits = pProxyCommitData->latestLocalCommitBatchLogging.get() < localBatchNumber - 1;
|
|
|
|
TEST(queuedCommits); // Queuing post-resolution commit processing
|
2020-07-29 15:42:52 +08:00
|
|
|
wait(pProxyCommitData->latestLocalCommitBatchLogging.whenAtLeast(localBatchNumber - 1));
|
2021-08-06 00:17:37 +08:00
|
|
|
state double postResolutionQueuing = now();
|
|
|
|
pProxyCommitData->stats.postResolutionDist->sampleSeconds(postResolutionQueuing - postResolutionStart);
|
2020-07-29 15:42:52 +08:00
|
|
|
wait(yield(TaskPriority::ProxyCommitYield1));
|
|
|
|
|
2020-08-12 05:12:36 +08:00
|
|
|
self->computeStart = g_network->timer();
|
2020-07-29 15:42:52 +08:00
|
|
|
|
|
|
|
pProxyCommitData->stats.txnCommitResolved += trs.size();
|
|
|
|
|
|
|
|
if (debugID.present()) {
|
2021-03-11 02:06:03 +08:00
|
|
|
g_traceBatch.addEvent(
|
|
|
|
"CommitDebug", debugID.get().first(), "CommitProxyServer.commitBatch.ProcessingMutations");
|
2020-07-29 15:42:52 +08:00
|
|
|
}
|
|
|
|
|
2022-02-26 03:18:42 +08:00
|
|
|
self->isMyFirstBatch = !pProxyCommitData->version.get();
|
2020-07-29 15:42:52 +08:00
|
|
|
self->oldCoordinators = pProxyCommitData->txnStateStore->readValue(coordinatorsKey).get();
|
|
|
|
|
|
|
|
assertResolutionStateMutationsSizeConsistent(self->resolution);
|
|
|
|
|
|
|
|
applyMetadataEffect(self);
|
|
|
|
|
2022-01-07 04:15:51 +08:00
|
|
|
if (debugID.present()) {
|
|
|
|
g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "CommitProxyServer.commitBatch.ApplyMetadaEffect");
|
|
|
|
}
|
|
|
|
|
2020-07-29 15:42:52 +08:00
|
|
|
determineCommittedTransactions(self);
|
|
|
|
|
2022-01-07 04:15:51 +08:00
|
|
|
if (debugID.present()) {
|
|
|
|
g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "CommitProxyServer.commitBatch.ApplyMetadaEffect");
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->forceRecovery) {
|
|
|
|
wait(Future<Void>(Never()));
|
2020-07-29 15:42:52 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// First pass
|
|
|
|
wait(applyMetadataToCommittedTransactions(self));
|
|
|
|
|
2022-01-07 04:15:51 +08:00
|
|
|
if (debugID.present()) {
|
|
|
|
g_traceBatch.addEvent(
|
|
|
|
"CommitDebug", debugID.get().first(), "CommitProxyServer.commitBatch.ApplyMetadaToCommittedTxn");
|
|
|
|
}
|
|
|
|
|
2020-07-29 15:42:52 +08:00
|
|
|
// Second pass
|
|
|
|
wait(assignMutationsToStorageServers(self));
|
|
|
|
|
2022-01-07 04:15:51 +08:00
|
|
|
if (debugID.present()) {
|
|
|
|
g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "CommitProxyServer.commitBatch.AssignMutationToSS");
|
|
|
|
}
|
|
|
|
|
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()) {
|
2021-03-11 02:06:03 +08:00
|
|
|
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
|
|
|
|
2021-03-11 02:06:03 +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
|
|
|
|
if (pProxyCommitData->committedVersion.get() <
|
|
|
|
self->commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS) {
|
2020-07-26 12:55:22 +08:00
|
|
|
self->computeDuration += g_network->timer() - self->computeStart;
|
2020-07-08 00:06:13 +08:00
|
|
|
state Span waitVersionSpan;
|
2021-03-11 02:06:03 +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);
|
2021-03-11 02:06:03 +08:00
|
|
|
waitVersionSpan = Span(
|
|
|
|
deterministicRandom()->randomUniqueID(), "MP:overMaxReadTransactionLifeVersions"_loc, { span.context });
|
|
|
|
choose {
|
|
|
|
when(wait(pProxyCommitData->committedVersion.whenAtLeast(
|
|
|
|
self->commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS))) {
|
2020-01-27 11:47:12 +08:00
|
|
|
wait(yield());
|
2020-07-26 12:44:10 +08:00
|
|
|
break;
|
2020-01-27 11:47:12 +08:00
|
|
|
}
|
2020-08-12 10:22:10 +08:00
|
|
|
when(wait(pProxyCommitData->cx->onProxiesChanged())) {}
|
|
|
|
when(GetRawCommittedVersionReply v = wait(pProxyCommitData->master.getLiveCommittedVersion.getReply(
|
2021-03-11 02:06:03 +08:00
|
|
|
GetRawCommittedVersionRequest(waitVersionSpan.context, debugID),
|
|
|
|
TaskPriority::GetLiveCommittedVersionReply))) {
|
|
|
|
if (v.version > pProxyCommitData->committedVersion.get()) {
|
2020-07-26 12:55:22 +08:00
|
|
|
pProxyCommitData->locked = v.locked;
|
|
|
|
pProxyCommitData->metadataVersion = v.metadataVersion;
|
|
|
|
pProxyCommitData->committedVersion.set(v.version);
|
2020-01-27 11:47:12 +08:00
|
|
|
}
|
2020-06-09 07:09:37 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (pProxyCommitData->committedVersion.get() <
|
|
|
|
self->commitVersion - SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS)
|
2020-01-27 11:47:12 +08:00
|
|
|
wait(delay(SERVER_KNOBS->PROXY_SPIN_DELAY));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2020-07-08 00:06:13 +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())
|
2021-03-11 02:06:03 +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;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto m : self->msg.messages) {
|
|
|
|
if (firstMessage) {
|
2020-07-26 12:55:22 +08:00
|
|
|
self->toCommit.addTxsTag();
|
2017-07-16 06:15:03 +08:00
|
|
|
}
|
2020-08-28 07:16:05 +08:00
|
|
|
self->toCommit.writeMessage(StringRef(m.begin(), m.size()), !firstMessage);
|
2017-05-26 04:48:44 +08:00
|
|
|
firstMessage = false;
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +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)
|
2021-03-11 02:06:03 +08:00
|
|
|
// .detail("TransactionsSubmitted", trs.size()).detail("TransactionsCommitted", commitCount).detail("TxsPopTo",
|
|
|
|
// msg.popTo);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->prevVersion && self->commitVersion - self->prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT / 2)
|
2020-07-26 12:55:22 +08:00
|
|
|
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;
|
2021-03-11 02:06:03 +08:00
|
|
|
self->loggingComplete = pProxyCommitData->logSystem->push(self->prevVersion,
|
|
|
|
self->commitVersion,
|
|
|
|
pProxyCommitData->committedVersion.get(),
|
|
|
|
pProxyCommitData->minKnownCommittedVersion,
|
|
|
|
self->toCommit,
|
|
|
|
span.context,
|
|
|
|
self->debugID);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-06-29 02:08:20 +08:00
|
|
|
float ratio = self->toCommit.getEmptyMessageRatio();
|
2021-06-25 12:20:21 +08:00
|
|
|
pProxyCommitData->stats.commitBatchingEmptyMessageRatio.addMeasurement(ratio);
|
|
|
|
|
2020-07-26 12:55:22 +08:00
|
|
|
if (!self->forceRecovery) {
|
2021-03-11 02:06:03 +08:00
|
|
|
ASSERT(pProxyCommitData->latestLocalCommitBatchLogging.get() == self->localBatchNumber - 1);
|
2020-07-26 12:55:22 +08:00
|
|
|
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;
|
2020-10-20 13:51:56 +08:00
|
|
|
if (self->batchOperations > 0) {
|
2021-03-11 02:06:03 +08:00
|
|
|
double computePerOperation =
|
|
|
|
std::min(SERVER_KNOBS->MAX_COMPUTE_PER_OPERATION, 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]);
|
|
|
|
}
|
|
|
|
pProxyCommitData->stats.maxComputeNS =
|
|
|
|
std::max<int64_t>(pProxyCommitData->stats.maxComputeNS,
|
|
|
|
1e9 * pProxyCommitData->commitComputePerOperation[self->latencyBucket]);
|
|
|
|
pProxyCommitData->stats.minComputeNS =
|
|
|
|
std::min<int64_t>(pProxyCommitData->stats.minComputeNS,
|
|
|
|
1e9 * pProxyCommitData->commitComputePerOperation[self->latencyBucket]);
|
|
|
|
}
|
2020-01-27 11:47:12 +08:00
|
|
|
|
2021-08-06 00:17:37 +08:00
|
|
|
pProxyCommitData->stats.processingMutationDist->sampleSeconds(now() - postResolutionQueuing);
|
2020-07-26 12:55:22 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2020-08-06 05:11:43 +08:00
|
|
|
ACTOR Future<Void> transactionLogging(CommitBatchContext* self) {
|
2021-08-06 00:17:37 +08:00
|
|
|
state double tLoggingStart = now();
|
2020-07-26 09:11:25 +08:00
|
|
|
state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData;
|
2020-08-29 03:02:51 +08:00
|
|
|
state Span span("MP:transactionLogging"_loc, self->span.context);
|
2018-06-22 06:29:46 +08:00
|
|
|
|
|
|
|
try {
|
|
|
|
choose {
|
2020-07-26 09:11:25 +08:00
|
|
|
when(Version ver = wait(self->loggingComplete)) {
|
|
|
|
pProxyCommitData->minKnownCommittedVersion = std::max(pProxyCommitData->minKnownCommittedVersion, ver);
|
2018-06-22 06:29:46 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
when(wait(pProxyCommitData->committedVersion.whenAtLeast(self->commitVersion + 1))) {}
|
2018-06-22 06:29:46 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
} catch (Error& e) {
|
|
|
|
if (e.code() == error_code_broken_promise) {
|
2022-01-07 04:15:51 +08:00
|
|
|
throw tlog_failed();
|
2018-06-22 06:29:46 +08:00
|
|
|
}
|
|
|
|
throw;
|
|
|
|
}
|
2020-07-02 10:39:32 +08:00
|
|
|
|
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
|
|
|
|
2020-01-27 11:47:12 +08:00
|
|
|
wait(yield(TaskPriority::ProxyCommitYield2));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-03-11 02:06:03 +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) {
|
2018-09-18 09:32:39 +08:00
|
|
|
TraceEvent(SevWarnAlways, "DiscardingTxsPopHistory").suppressFor(1.0);
|
2020-07-26 09:11:25 +08:00
|
|
|
pProxyCommitData->txsPopVersions.pop_front();
|
2018-09-18 09:32:39 +08:00
|
|
|
}
|
|
|
|
|
2020-07-26 09:11:25 +08:00
|
|
|
pProxyCommitData->txsPopVersions.emplace_back(self->commitVersion, self->msg.popTo);
|
2018-09-18 09:32:39 +08:00
|
|
|
}
|
2020-07-26 09:11:25 +08:00
|
|
|
pProxyCommitData->logSystem->popTxs(self->msg.popTo);
|
2021-08-06 00:17:37 +08:00
|
|
|
pProxyCommitData->stats.tlogLoggingDist->sampleSeconds(now() - tLoggingStart);
|
2020-07-26 09:11:25 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2020-08-06 05:11:43 +08:00
|
|
|
ACTOR Future<Void> reply(CommitBatchContext* self) {
|
2021-08-06 00:17:37 +08:00
|
|
|
state double replyStart = now();
|
2020-07-26 09:11:25 +08:00
|
|
|
state ProxyCommitData* const pProxyCommitData = self->pProxyCommitData;
|
2020-08-29 03:02:51 +08:00
|
|
|
state Span span("MP:reply"_loc, self->span.context);
|
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
|
|
|
|
2022-01-07 04:15:51 +08:00
|
|
|
if (self->prevVersion && self->commitVersion - self->prevVersion < SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT / 2) {
|
|
|
|
//TraceEvent("CPAdvanceMinVersion", self->pProxyCommitData->dbgid).detail("PrvVersion", self->prevVersion).detail("CommitVersion", self->commitVersion).detail("Master", self->pProxyCommitData->master.id().toString()).detail("TxSize", self->trs.size());
|
2020-07-26 09:11:25 +08:00
|
|
|
debug_advanceMinCommittedVersion(UID(), self->commitVersion);
|
2022-01-07 04:15:51 +08:00
|
|
|
}
|
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
|
|
|
|
2021-03-11 02:06:03 +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-03 07:14:50 +08:00
|
|
|
// After logging finishes, we report the commit version to master so that every other proxy can get the most
|
2021-03-11 02:06:03 +08:00
|
|
|
// 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.
|
2021-08-28 08:17:19 +08:00
|
|
|
TEST(pProxyCommitData->committedVersion.get() > self->commitVersion); // 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(
|
2021-03-11 02:06:03 +08:00
|
|
|
ReportRawCommittedVersionRequest(self->commitVersion,
|
|
|
|
self->lockedAfter,
|
|
|
|
self->metadataVersionAfter,
|
2020-08-13 01:34:07 +08:00
|
|
|
pProxyCommitData->minKnownCommittedVersion),
|
|
|
|
TaskPriority::ProxyMasterVersionReply));
|
2020-07-03 07:14:50 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->commitVersion > pProxyCommitData->committedVersion.get()) {
|
2020-07-26 09:11:25 +08:00
|
|
|
pProxyCommitData->locked = self->lockedAfter;
|
|
|
|
pProxyCommitData->metadataVersion = self->metadataVersionAfter;
|
|
|
|
pProxyCommitData->committedVersion.set(self->commitVersion);
|
2020-07-03 07:14:50 +08:00
|
|
|
}
|
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
|
2020-01-27 11:47:12 +08:00
|
|
|
double endTime = g_network->timer();
|
2020-03-25 00:48:03 +08:00
|
|
|
// 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));
|
2021-03-11 02:06:03 +08:00
|
|
|
} else if (self->committed[t] == ConflictBatch::TransactionTooOld) {
|
2020-07-26 09:11:25 +08:00
|
|
|
tr.reply.sendError(transaction_too_old());
|
2022-02-19 13:22:31 +08:00
|
|
|
} else if (self->committed[t] == ConflictBatch::TransactionTenantFailure) {
|
|
|
|
// We already sent the error
|
|
|
|
ASSERT(tr.reply.isSet());
|
2021-03-11 02:06:03 +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) {
|
2020-03-25 00:48:03 +08:00
|
|
|
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]
|
2021-03-11 02:06:03 +08:00
|
|
|
.conflictingKeyRangeMap[self->nextTr[resolverInd]]; // nextTr[resolverInd] -> index of this
|
|
|
|
// trs[t] 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]);
|
2020-03-25 00:48:03 +08:00
|
|
|
}
|
|
|
|
// At least one keyRange index should be returned
|
|
|
|
ASSERT(conflictingKRIndices.size());
|
2021-03-11 02:06:03 +08:00
|
|
|
tr.reply.send(CommitID(
|
|
|
|
invalidVersion, t, Optional<Value>(), Optional<Standalone<VectorRef<int>>>(conflictingKRIndices)));
|
2020-03-25 00:48:03 +08:00
|
|
|
} else {
|
2020-07-26 09:11:25 +08:00
|
|
|
tr.reply.sendError(not_committed());
|
2020-03-25 00:48:03 +08:00
|
|
|
}
|
2018-12-01 02:46:04 +08:00
|
|
|
}
|
|
|
|
|
2020-03-25 00:48:03 +08:00
|
|
|
// Update corresponding transaction indices on each resolver
|
2021-03-11 02:06:03 +08:00
|
|
|
for (int resolverInd : self->transactionResolverMap[t])
|
|
|
|
self->nextTr[resolverInd]++;
|
2020-03-25 00:48:03 +08:00
|
|
|
|
2018-12-01 02:46:04 +08:00
|
|
|
// 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);
|
2021-03-11 02:06:03 +08:00
|
|
|
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);
|
2019-01-19 08:18:34 +08:00
|
|
|
}
|
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
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (now() - pProxyCommitData->lastCoalesceTime > SERVER_KNOBS->RESOLVER_COALESCE_TIME) {
|
2020-07-26 09:11:25 +08:00
|
|
|
pProxyCommitData->lastCoalesceTime = now();
|
|
|
|
int lastSize = pProxyCommitData->keyResolvers.size();
|
|
|
|
auto rs = pProxyCommitData->keyResolvers.ranges();
|
|
|
|
Version oldestVersion = self->prevVersion - SERVER_KNOBS->MAX_WRITE_TRANSACTION_LIFE_VERSIONS;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto r = rs.begin(); r != rs.end(); ++r) {
|
|
|
|
while (r->value().size() > 1 && r->value()[1].first < oldestVersion)
|
2017-05-26 04:48:44 +08:00
|
|
|
r->value().pop_front();
|
2021-03-11 02:06:03 +08:00
|
|
|
if (r->value().size() && r->value().front().first < oldestVersion)
|
2017-05-26 04:48:44 +08:00
|
|
|
r->value().front().first = 0;
|
|
|
|
}
|
2020-07-26 09:11:25 +08:00
|
|
|
pProxyCommitData->keyResolvers.coalesce(allKeys);
|
2021-03-11 02:06:03 +08:00
|
|
|
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
|
2021-03-11 02:06:03 +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 +
|
|
|
|
pProxyCommitData->commitBatchInterval *
|
|
|
|
(1 - SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA)));
|
2020-07-26 09:11:25 +08:00
|
|
|
|
2021-03-18 00:28:12 +08:00
|
|
|
pProxyCommitData->stats.commitBatchingWindowSize.addMeasurement(pProxyCommitData->commitBatchInterval);
|
2020-07-26 09:11:25 +08:00
|
|
|
pProxyCommitData->commitBatchesMemBytesCount -= self->currentBatchMemBytesCount;
|
|
|
|
ASSERT_ABORT(pProxyCommitData->commitBatchesMemBytesCount >= 0);
|
|
|
|
wait(self->releaseFuture);
|
2021-08-06 00:17:37 +08:00
|
|
|
pProxyCommitData->stats.replyCommitDist->sampleSeconds(now() - replyStart);
|
2020-07-26 09:11:25 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
} // namespace CommitBatch
|
2020-07-26 09:11:25 +08:00
|
|
|
|
|
|
|
// Commit one batch of transactions trs
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> commitBatch(ProxyCommitData* self,
|
2021-09-17 08:42:34 +08:00
|
|
|
std::vector<CommitTransactionRequest>* trs,
|
2021-03-11 02:06:03 +08:00
|
|
|
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
|
2020-08-06 05:11:43 +08:00
|
|
|
state CommitBatch::CommitBatchContext context(self, trs, currentBatchMemBytesCount);
|
2021-05-28 02:25:00 +08:00
|
|
|
getCurrentLineage()->modify(&TransactionLineage::operation) = TransactionLineage::Operation::Commit;
|
2020-07-26 09:11:25 +08:00
|
|
|
|
2021-03-11 02:06:03 +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
|
2020-07-26 09:11:25 +08:00
|
|
|
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
|
|
|
|
2021-03-11 02:06:03 +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)
|
2020-07-26 09:11:25 +08:00
|
|
|
wait(CommitBatch::preresolutionProcessing(&context));
|
2021-03-20 02:30:50 +08:00
|
|
|
if (context.rejected) {
|
|
|
|
self->commitBatchesMemBytesCount -= currentBatchMemBytesCount;
|
2021-03-11 02:06:03 +08:00
|
|
|
return Void();
|
2021-03-20 02:30:50 +08:00
|
|
|
}
|
2020-07-26 09:11:25 +08:00
|
|
|
|
|
|
|
/////// Phase 2: Resolution (waiting on the network; pipelined)
|
|
|
|
wait(CommitBatch::getResolution(&context));
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
////// Phase 3: Post-resolution processing (CPU bound except for very rare situations; ordered; currently atomic but
|
|
|
|
/// doesn't need to be)
|
2020-07-26 09:11:25 +08:00
|
|
|
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));
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
/////// Phase 5: Replies (CPU bound; no particular order required, though ordered execution would be best for
|
|
|
|
/// latency)
|
2020-07-26 09:11:25 +08:00
|
|
|
wait(CommitBatch::reply(&context));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-06-03 23:31:16 +08:00
|
|
|
// Add tss mapping data to the reply, if any of the included storage servers have a TSS pair
|
2021-05-29 02:15:52 +08:00
|
|
|
void maybeAddTssMapping(GetKeyServerLocationsReply& reply,
|
|
|
|
ProxyCommitData* commitData,
|
|
|
|
std::unordered_set<UID>& included,
|
|
|
|
UID ssId) {
|
|
|
|
if (!included.count(ssId)) {
|
|
|
|
auto mappingItr = commitData->tssMapping.find(ssId);
|
|
|
|
if (mappingItr != commitData->tssMapping.end()) {
|
|
|
|
reply.resultsTssMapping.push_back(*mappingItr);
|
|
|
|
}
|
2021-06-08 03:54:24 +08:00
|
|
|
included.insert(ssId);
|
2021-05-29 02:15:52 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
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
|
2021-05-28 02:25:00 +08:00
|
|
|
getCurrentLineage()->modify(&TransactionLineage::operation) = TransactionLineage::Operation::GetKeyServersLocations;
|
|
|
|
getCurrentLineage()->modify(&TransactionLineage::txID) = req.spanContext.first();
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(commitData->validState.getFuture());
|
2020-03-14 01:17:49 +08:00
|
|
|
wait(delay(0, TaskPriority::DefaultEndpoint));
|
|
|
|
|
2022-02-26 03:18:42 +08:00
|
|
|
state ErrorOr<Optional<TenantMapEntry>> tenantEntry;
|
2022-03-09 05:33:56 +08:00
|
|
|
state Version minTenantVersion =
|
|
|
|
req.minTenantVersion == latestVersion ? commitData->stats.lastCommitVersionAssigned + 1 : req.minTenantVersion;
|
|
|
|
|
|
|
|
// If a large minTenantVersion is specified, we limit how long we wait for it to be available
|
|
|
|
state Future<Void> futureVersionDelay = minTenantVersion > commitData->stats.lastCommitVersionAssigned + 1
|
|
|
|
? delay(SERVER_KNOBS->FUTURE_VERSION_DELAY)
|
|
|
|
: Never();
|
|
|
|
|
2022-02-26 03:18:42 +08:00
|
|
|
while (tenantEntry.isError()) {
|
2022-03-09 05:33:56 +08:00
|
|
|
bool finalQuery = commitData->version.get() >= minTenantVersion;
|
2022-03-04 07:40:38 +08:00
|
|
|
ErrorOr<Optional<TenantMapEntry>> _tenantEntry =
|
2022-03-09 05:33:56 +08:00
|
|
|
getTenantEntry(commitData, req.tenant, Optional<int64_t>(), finalQuery);
|
2022-02-26 03:18:42 +08:00
|
|
|
tenantEntry = _tenantEntry;
|
|
|
|
|
|
|
|
if (tenantEntry.isError()) {
|
2022-03-09 05:33:56 +08:00
|
|
|
if (finalQuery) {
|
2022-02-26 03:18:42 +08:00
|
|
|
req.reply.sendError(tenant_not_found());
|
|
|
|
return Void();
|
|
|
|
} else {
|
2022-03-09 05:33:56 +08:00
|
|
|
choose {
|
|
|
|
// Wait until we are sure that we've received metadata updates through minTenantVersion
|
|
|
|
// If latestVersion is specified, this will wait until we have definitely received
|
|
|
|
// updates through the version at the time we received the request
|
|
|
|
when(wait(commitData->version.whenAtLeast(minTenantVersion))) {}
|
|
|
|
when(wait(futureVersionDelay)) {
|
|
|
|
req.reply.sendError(future_version());
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
}
|
2022-02-26 03:18:42 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-05-29 02:15:52 +08:00
|
|
|
std::unordered_set<UID> tssMappingsIncluded;
|
2020-03-14 01:17:49 +08:00
|
|
|
GetKeyServerLocationsReply rep;
|
2022-02-26 03:18:42 +08:00
|
|
|
|
|
|
|
if (tenantEntry.get().present()) {
|
|
|
|
rep.tenantEntry = tenantEntry.get().get();
|
|
|
|
req.begin = req.begin.withPrefix(rep.tenantEntry.prefix, req.arena);
|
|
|
|
if (req.end.present()) {
|
|
|
|
req.end = req.end.get().withPrefix(rep.tenantEntry.prefix, req.arena);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!req.end.present()) {
|
|
|
|
auto r = req.reverse ? commitData->keyInfo.rangeContainingKeyBefore(req.begin)
|
|
|
|
: commitData->keyInfo.rangeContaining(req.begin);
|
2021-09-17 08:42:34 +08:00
|
|
|
std::vector<StorageServerInterface> ssis;
|
2020-03-14 01:17:49 +08:00
|
|
|
ssis.reserve(r.value().src_info.size());
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& it : r.value().src_info) {
|
2020-03-14 01:17:49 +08:00
|
|
|
ssis.push_back(it->interf);
|
2021-05-29 02:15:52 +08:00
|
|
|
maybeAddTssMapping(rep, commitData, tssMappingsIncluded, it->interf.id());
|
2020-03-14 01:17:49 +08:00
|
|
|
}
|
2021-05-11 07:32:02 +08:00
|
|
|
rep.results.emplace_back(r.range(), ssis);
|
2021-03-11 02:06:03 +08:00
|
|
|
} else if (!req.reverse) {
|
2020-03-14 01:17:49 +08:00
|
|
|
int count = 0;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto r = commitData->keyInfo.rangeContaining(req.begin);
|
|
|
|
r != commitData->keyInfo.ranges().end() && count < req.limit && r.begin() < req.end.get();
|
|
|
|
++r) {
|
2021-09-17 08:42:34 +08:00
|
|
|
std::vector<StorageServerInterface> ssis;
|
2019-04-26 08:07:35 +08:00
|
|
|
ssis.reserve(r.value().src_info.size());
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& it : r.value().src_info) {
|
2019-04-26 08:07:35 +08:00
|
|
|
ssis.push_back(it->interf);
|
2021-05-29 02:15:52 +08:00
|
|
|
maybeAddTssMapping(rep, commitData, tssMappingsIncluded, it->interf.id());
|
2019-04-26 08:07:35 +08:00
|
|
|
}
|
2021-05-11 07:32:02 +08:00
|
|
|
rep.results.emplace_back(r.range(), ssis);
|
2020-03-14 01:17:49 +08:00
|
|
|
count++;
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
int count = 0;
|
|
|
|
auto r = commitData->keyInfo.rangeContainingKeyBefore(req.end.get());
|
2021-03-11 02:06:03 +08:00
|
|
|
while (count < req.limit && req.begin < r.end()) {
|
2021-09-17 08:42:34 +08:00
|
|
|
std::vector<StorageServerInterface> ssis;
|
2020-03-14 01:17:49 +08:00
|
|
|
ssis.reserve(r.value().src_info.size());
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& it : r.value().src_info) {
|
2020-03-14 01:17:49 +08:00
|
|
|
ssis.push_back(it->interf);
|
2021-05-29 02:15:52 +08:00
|
|
|
maybeAddTssMapping(rep, commitData, tssMappingsIncluded, it->interf.id());
|
2019-04-26 08:07:35 +08:00
|
|
|
}
|
2021-05-11 07:32:02 +08:00
|
|
|
rep.results.emplace_back(r.range(), ssis);
|
2021-03-11 02:06:03 +08:00
|
|
|
if (r == commitData->keyInfo.ranges().begin()) {
|
2020-03-14 01:17:49 +08:00
|
|
|
break;
|
2019-04-26 08:07:35 +08:00
|
|
|
}
|
2020-03-14 01:17:49 +08:00
|
|
|
count++;
|
|
|
|
--r;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
req.reply.send(rep);
|
|
|
|
++commitData->stats.keyServerLocationOut;
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR static Future<Void> readRequestServer(CommitProxyInterface proxy,
|
|
|
|
PromiseStream<Future<Void>> addActor,
|
2020-09-11 08:44:15 +08:00
|
|
|
ProxyCommitData* commitData) {
|
2020-03-14 01:17:49 +08:00
|
|
|
loop {
|
|
|
|
GetKeyServerLocationsRequest req = waitNext(proxy.getKeyServersLocations.getFuture());
|
2021-03-11 02:06:03 +08:00
|
|
|
// 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) {
|
2020-03-14 01:17:49 +08:00
|
|
|
++commitData->stats.keyServerLocationErrors;
|
|
|
|
req.reply.sendError(proxy_memory_limit_exceeded());
|
2020-03-14 09:31:22 +08:00
|
|
|
TraceEvent(SevWarnAlways, "ProxyLocationRequestThresholdExceeded").suppressFor(60);
|
2020-03-14 01:17:49 +08:00
|
|
|
} else {
|
|
|
|
++commitData->stats.keyServerLocationIn;
|
|
|
|
addActor.send(doKeyServerLocationRequest(req, commitData));
|
2019-04-26 08:07:35 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-09-11 08:44:15 +08:00
|
|
|
ACTOR static Future<Void> rejoinServer(CommitProxyInterface proxy, ProxyCommitData* commitData) {
|
2019-04-26 08:07:35 +08:00
|
|
|
// We can't respond to these requests until we have valid txnStateStore
|
|
|
|
wait(commitData->validState.getFuture());
|
|
|
|
|
2021-08-10 16:52:36 +08:00
|
|
|
TraceEvent("ProxyReadyForReads", proxy.id()).log();
|
2020-03-14 01:17:49 +08:00
|
|
|
|
2019-04-26 08:07:35 +08:00
|
|
|
loop {
|
|
|
|
GetStorageServerRejoinInfoRequest req = waitNext(proxy.getStorageServerRejoinInfo.getFuture());
|
|
|
|
if (commitData->txnStateStore->readValue(serverListKeyFor(req.id)).get().present()) {
|
|
|
|
GetStorageServerRejoinInfoReply rep;
|
2022-02-26 03:18:42 +08:00
|
|
|
rep.version = commitData->version.get();
|
2021-03-11 02:06:03 +08:00
|
|
|
rep.tag = decodeServerTagValue(commitData->txnStateStore->readValue(serverTagKeyFor(req.id)).get().get());
|
2021-05-04 04:14:16 +08:00
|
|
|
RangeResult history = commitData->txnStateStore->readRange(serverTagHistoryRangeFor(req.id)).get();
|
2021-03-11 02:06:03 +08:00
|
|
|
for (int i = history.size() - 1; i >= 0; i--) {
|
|
|
|
rep.history.push_back(
|
|
|
|
std::make_pair(decodeServerTagHistoryKey(history[i].key), decodeServerTagValue(history[i].value)));
|
2019-04-26 08:07:35 +08:00
|
|
|
}
|
|
|
|
auto localityKey = commitData->txnStateStore->readValue(tagLocalityListKeyFor(req.dcId)).get();
|
2020-05-09 07:17:10 +08:00
|
|
|
rep.newLocality = false;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (localityKey.present()) {
|
2019-04-26 08:07:35 +08:00
|
|
|
int8_t locality = decodeTagLocalityListValue(localityKey.get());
|
2021-03-11 02:06:03 +08:00
|
|
|
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) {
|
2019-04-26 08:07:35 +08:00
|
|
|
uint16_t tagId = 0;
|
|
|
|
std::vector<uint16_t> usedTags;
|
|
|
|
auto tagKeys = commitData->txnStateStore->readRange(serverTagKeys).get();
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& kv : tagKeys) {
|
|
|
|
Tag t = decodeServerTagValue(kv.value);
|
|
|
|
if (t.locality == locality) {
|
2019-04-26 08:07:35 +08:00
|
|
|
usedTags.push_back(t.id);
|
2017-12-16 12:13:44 +08:00
|
|
|
}
|
|
|
|
}
|
2019-04-26 08:07:35 +08:00
|
|
|
auto historyKeys = commitData->txnStateStore->readRange(serverTagHistoryKeys).get();
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& kv : historyKeys) {
|
|
|
|
Tag t = decodeServerTagValue(kv.value);
|
|
|
|
if (t.locality == locality) {
|
2019-04-26 08:07:35 +08:00
|
|
|
usedTags.push_back(t.id);
|
2017-12-16 12:13:44 +08:00
|
|
|
}
|
2019-04-26 08:07:35 +08:00
|
|
|
}
|
|
|
|
std::sort(usedTags.begin(), usedTags.end());
|
|
|
|
|
|
|
|
int usedIdx = 0;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (; usedTags.size() > 0 && tagId <= usedTags.end()[-1]; tagId++) {
|
|
|
|
if (tagId < usedTags[usedIdx]) {
|
2017-12-16 12:13:44 +08:00
|
|
|
break;
|
2019-04-26 08:07:35 +08:00
|
|
|
} else {
|
|
|
|
usedIdx++;
|
2017-12-16 12:13:44 +08:00
|
|
|
}
|
|
|
|
}
|
2019-04-26 08:07:35 +08:00
|
|
|
rep.newTag = Tag(locality, tagId);
|
2017-07-27 04:45:11 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
} else if (rep.tag.locality != tagLocalityUpgraded) {
|
|
|
|
TraceEvent(SevWarnAlways, "SSRejoinedWithUnknownLocality")
|
|
|
|
.detail("Tag", rep.tag.toString())
|
|
|
|
.detail("DcId", req.dcId);
|
2019-04-26 08:07:35 +08:00
|
|
|
} else {
|
|
|
|
rep.newLocality = true;
|
|
|
|
int8_t maxTagLocality = -1;
|
|
|
|
auto localityKeys = commitData->txnStateStore->readRange(tagLocalityListKeys).get();
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& kv : localityKeys) {
|
|
|
|
maxTagLocality = std::max(maxTagLocality, decodeTagLocalityListValue(kv.value));
|
2017-08-04 07:16:36 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
rep.newTag = Tag(maxTagLocality + 1, 0);
|
2017-08-04 07:16:36 +08:00
|
|
|
}
|
2019-04-26 08:07:35 +08:00
|
|
|
req.reply.send(rep);
|
|
|
|
} else {
|
|
|
|
req.reply.sendError(worker_removed());
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-07-12 13:17:09 +08:00
|
|
|
ACTOR Future<Void> ddMetricsRequestServer(CommitProxyInterface proxy, Reference<AsyncVar<ServerDBInfo> const> db) {
|
2019-05-17 07:46:33 +08:00
|
|
|
loop {
|
|
|
|
choose {
|
2021-03-11 02:06:03 +08:00
|
|
|
when(state GetDDMetricsRequest req = waitNext(proxy.getDDMetrics.getFuture())) {
|
2020-08-30 03:35:31 +08:00
|
|
|
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()) {
|
2019-05-18 07:11:50 +08:00
|
|
|
req.reply.sendError(reply.getError());
|
|
|
|
} else {
|
|
|
|
GetDDMetricsReply newReply;
|
|
|
|
newReply.storageMetricsList = reply.get().storageMetricsList;
|
|
|
|
req.reply.send(newReply);
|
|
|
|
}
|
2019-05-17 07:46:33 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-04-13 04:23:02 +08:00
|
|
|
ACTOR Future<Void> monitorRemoteCommitted(ProxyCommitData* self) {
|
2018-09-18 09:32:39 +08:00
|
|
|
loop {
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(delay(0)); // allow this actor to be cancelled if we are removed after db changes.
|
2018-09-18 09:32:39 +08:00
|
|
|
state Optional<std::vector<OptionalInterface<TLogInterface>>> remoteLogs;
|
2021-03-11 02:06:03 +08:00
|
|
|
if (self->db->get().recoveryState >= RecoveryState::ALL_LOGS_RECRUITED) {
|
|
|
|
for (auto& logSet : self->db->get().logSystemConfig.tLogs) {
|
|
|
|
if (!logSet.isLocal) {
|
2018-09-18 09:32:39 +08:00
|
|
|
remoteLogs = logSet.tLogs;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& tLog : logSet.tLogs) {
|
|
|
|
if (!tLog.present()) {
|
2018-09-18 09:32:39 +08:00
|
|
|
remoteLogs = Optional<std::vector<OptionalInterface<TLogInterface>>>();
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (!remoteLogs.present()) {
|
2019-04-13 04:23:02 +08:00
|
|
|
wait(self->db->onChange());
|
2018-09-18 09:32:39 +08:00
|
|
|
continue;
|
|
|
|
}
|
2018-11-27 07:17:17 +08:00
|
|
|
self->popRemoteTxs = true;
|
2018-09-18 09:32:39 +08:00
|
|
|
|
2019-04-13 04:23:02 +08:00
|
|
|
state Future<Void> onChange = self->db->onChange();
|
2018-09-18 09:32:39 +08:00
|
|
|
loop {
|
|
|
|
state std::vector<Future<TLogQueuingMetricsReply>> replies;
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& it : remoteLogs.get()) {
|
|
|
|
replies.push_back(
|
|
|
|
brokenPromiseToNever(it.interf().getQueuingMetrics.getReply(TLogQueuingMetricsRequest())));
|
2018-09-18 09:32:39 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(waitForAll(replies) || onChange);
|
2018-09-18 09:32:39 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
if (onChange.isReady()) {
|
2018-09-18 09:32:39 +08:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// FIXME: use the configuration to calculate a more precise minimum recovery version.
|
2018-09-18 09:32:39 +08:00
|
|
|
Version minVersion = std::numeric_limits<Version>::max();
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto& it : replies) {
|
2018-09-18 09:32:39 +08:00
|
|
|
minVersion = std::min(minVersion, it.get().v);
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
while (self->txsPopVersions.size() && self->txsPopVersions.front().first <= minVersion) {
|
2018-09-18 09:32:39 +08:00
|
|
|
self->lastTxsPop = self->txsPopVersions.front().second;
|
2019-06-20 09:15:09 +08:00
|
|
|
self->logSystem->popTxs(self->txsPopVersions.front().second, tagLocalityRemoteLog);
|
2018-09-18 09:32:39 +08:00
|
|
|
self->txsPopVersions.pop_front();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
wait(delay(SERVER_KNOBS->UPDATE_REMOTE_LOG_VERSION_INTERVAL) || onChange);
|
|
|
|
if (onChange.isReady()) {
|
2018-09-18 09:32:39 +08:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-07-28 06:08:13 +08:00
|
|
|
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
|
2019-07-13 01:56:27 +08:00
|
|
|
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
|
2021-03-11 02:06:03 +08:00
|
|
|
if (commitData->db->get().recoveryState != RecoveryState::FULLY_RECOVERED) {
|
2019-06-20 02:11:50 +08:00
|
|
|
// 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-07-21 16:00:29 +08:00
|
|
|
|
2019-06-20 02:11:50 +08:00
|
|
|
auto result =
|
2021-03-11 02:06:03 +08:00
|
|
|
commitData->txnStateStore->readValue(LiteralStringRef("log_anti_quorum").withPrefix(configKeysPrefix))
|
|
|
|
.get();
|
2019-06-20 02:11:50 +08:00
|
|
|
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) {
|
2021-11-15 08:12:28 +08:00
|
|
|
TraceEvent("SnapCommitProxy_LogAntiQuorumNotSupported")
|
2020-09-11 08:44:15 +08:00
|
|
|
.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()) {
|
2019-08-17 06:13:53 +08:00
|
|
|
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
|
|
|
}
|
2021-03-11 02:06:03 +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 {
|
2019-07-21 16:00:29 +08:00
|
|
|
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")
|
2022-02-25 04:25:52 +08:00
|
|
|
.errorUnsuppressed(e)
|
2020-09-11 08:44:15 +08:00
|
|
|
.detail("SnapPayload", snapReq.snapPayload)
|
2022-02-25 04:25:52 +08:00
|
|
|
.detail("SnapUID", snapReq.snapUID);
|
2019-07-21 16:00:29 +08:00
|
|
|
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")
|
2022-02-25 04:25:52 +08:00
|
|
|
.errorUnsuppressed(e)
|
2020-09-11 08:44:15 +08:00
|
|
|
.detail("SnapPayload", snapReq.snapPayload)
|
2022-02-25 04:25:52 +08:00
|
|
|
.detail("SnapUID", snapReq.snapUID);
|
2019-07-21 16:00:29 +08:00
|
|
|
if (e.code() != error_code_operation_cancelled) {
|
2019-06-20 02:11:50 +08:00
|
|
|
snapReq.reply.sendError(e);
|
2019-07-21 16:00:29 +08:00
|
|
|
} 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();
|
|
|
|
}
|
|
|
|
|
2021-07-12 13:17:09 +08:00
|
|
|
ACTOR Future<Void> proxyCheckSafeExclusion(Reference<AsyncVar<ServerDBInfo> const> db,
|
|
|
|
ExclusionSafetyCheckRequest req) {
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("SafetyCheckCommitProxyBegin").log();
|
2019-09-19 04:40:18 +08:00
|
|
|
state ExclusionSafetyCheckReply reply(false);
|
2019-08-17 06:13:53 +08:00
|
|
|
if (!db->get().distributor.present()) {
|
|
|
|
TraceEvent(SevWarnAlways, "DataDistributorNotPresent").detail("Operation", "ExclusionSafetyCheck");
|
2019-09-19 04:40:18 +08:00
|
|
|
req.reply.send(reply);
|
2019-08-17 06:13:53 +08:00
|
|
|
return Void();
|
|
|
|
}
|
2019-08-17 06:42:04 +08:00
|
|
|
try {
|
2022-02-26 01:30:21 +08:00
|
|
|
state Future<ErrorOr<DistributorExclusionSafetyCheckReply>> ddSafeFuture =
|
2019-09-19 04:40:18 +08:00
|
|
|
db->get().distributor.get().distributorExclCheckReq.tryGetReply(
|
|
|
|
DistributorExclusionSafetyCheckRequest(req.exclusions));
|
2022-02-26 01:30:21 +08:00
|
|
|
DistributorExclusionSafetyCheckReply _reply = wait(throwErrorOr(ddSafeFuture));
|
2019-09-19 04:40:18 +08:00
|
|
|
reply.safe = _reply.safe;
|
2022-02-26 01:30:21 +08:00
|
|
|
if (db->get().blobManager.present()) {
|
|
|
|
TraceEvent("SafetyCheckCommitProxyBM").detail("BMID", db->get().blobManager.get().id());
|
|
|
|
state Future<ErrorOr<BlobManagerExclusionSafetyCheckReply>> bmSafeFuture =
|
|
|
|
db->get().blobManager.get().blobManagerExclCheckReq.tryGetReply(
|
|
|
|
BlobManagerExclusionSafetyCheckRequest(req.exclusions));
|
|
|
|
BlobManagerExclusionSafetyCheckReply _reply = wait(throwErrorOr(bmSafeFuture));
|
|
|
|
reply.safe &= _reply.safe;
|
|
|
|
} else {
|
|
|
|
TraceEvent("SafetyCheckCommitProxyNoBM");
|
|
|
|
}
|
2019-08-17 06:42:04 +08:00
|
|
|
} catch (Error& e) {
|
2020-09-11 08:44:15 +08:00
|
|
|
TraceEvent("SafetyCheckCommitProxyResponseError").error(e);
|
2019-08-17 06:42:04 +08:00
|
|
|
if (e.code() != error_code_operation_cancelled) {
|
|
|
|
req.reply.sendError(e);
|
|
|
|
return Void();
|
|
|
|
} else {
|
|
|
|
throw e;
|
2019-08-17 06:13:53 +08:00
|
|
|
}
|
|
|
|
}
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("SafetyCheckCommitProxyFinish").log();
|
2019-09-19 04:40:18 +08:00
|
|
|
req.reply.send(reply);
|
2019-08-09 07:30:05 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> reportTxnTagCommitCost(UID myID,
|
2021-07-12 13:17:09 +08:00
|
|
|
Reference<AsyncVar<ServerDBInfo> const> db,
|
2020-08-30 03:35:31 +08:00
|
|
|
UIDTransactionTagMap<TransactionCommitCostEstimation>* ssTrTagCommitCost) {
|
|
|
|
state Future<Void> nextRequestTimer = Never();
|
|
|
|
state Future<Void> nextReply = Never();
|
2021-03-11 02:06:03 +08:00
|
|
|
if (db->get().ratekeeper.present())
|
|
|
|
nextRequestTimer = Void();
|
2020-08-30 03:35:31 +08:00
|
|
|
loop choose {
|
|
|
|
when(wait(db->onChange())) {
|
|
|
|
if (db->get().ratekeeper.present()) {
|
|
|
|
TraceEvent("ProxyRatekeeperChanged", myID).detail("RKID", db->get().ratekeeper.get().id());
|
|
|
|
nextRequestTimer = Void();
|
|
|
|
} else {
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("ProxyRatekeeperDied", myID).log();
|
2020-08-30 03:35:31 +08:00
|
|
|
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);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-08-28 14:38:34 +08:00
|
|
|
namespace {
|
|
|
|
|
|
|
|
struct TransactionStateResolveContext {
|
|
|
|
// Maximum sequence for txnStateRequest, this is defined when the request last flag is set.
|
|
|
|
Sequence maxSequence = std::numeric_limits<Sequence>::max();
|
|
|
|
|
|
|
|
// Flags marks received transaction state requests, we only process the transaction request when *all* requests are
|
|
|
|
// received.
|
|
|
|
std::unordered_set<Sequence> receivedSequences;
|
|
|
|
|
|
|
|
ProxyCommitData* pCommitData = nullptr;
|
|
|
|
|
|
|
|
// Pointer to transaction state store, shortcut for commitData.txnStateStore
|
|
|
|
IKeyValueStore* pTxnStateStore = nullptr;
|
|
|
|
|
2022-03-19 06:24:31 +08:00
|
|
|
Future<Void> txnRecovery;
|
|
|
|
|
2021-08-28 14:38:34 +08:00
|
|
|
// Actor streams
|
|
|
|
PromiseStream<Future<Void>>* pActors = nullptr;
|
|
|
|
|
|
|
|
// Flag reports if the transaction state request is complete. This request should only happen during recover, i.e.
|
|
|
|
// once per commit proxy.
|
|
|
|
bool processed = false;
|
|
|
|
|
|
|
|
TransactionStateResolveContext() = default;
|
|
|
|
|
|
|
|
TransactionStateResolveContext(ProxyCommitData* pCommitData_, PromiseStream<Future<Void>>* pActors_)
|
|
|
|
: pCommitData(pCommitData_), pTxnStateStore(pCommitData_->txnStateStore), pActors(pActors_) {
|
|
|
|
ASSERT(pTxnStateStore != nullptr);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
ACTOR Future<Void> processCompleteTransactionStateRequest(TransactionStateResolveContext* pContext) {
|
|
|
|
state KeyRange txnKeys = allKeys;
|
|
|
|
state std::map<Tag, UID> tag_uid;
|
|
|
|
|
|
|
|
RangeResult UIDtoTagMap = pContext->pTxnStateStore->readRange(serverTagKeys).get();
|
|
|
|
for (const KeyValueRef& kv : UIDtoTagMap) {
|
|
|
|
tag_uid[decodeServerTagValue(kv.value)] = decodeServerTagKey(kv.key);
|
|
|
|
}
|
|
|
|
|
|
|
|
loop {
|
|
|
|
wait(yield());
|
|
|
|
|
|
|
|
RangeResult data =
|
|
|
|
pContext->pTxnStateStore
|
|
|
|
->readRange(txnKeys, SERVER_KNOBS->BUGGIFIED_ROW_LIMIT, SERVER_KNOBS->APPLY_MUTATION_BYTES)
|
|
|
|
.get();
|
|
|
|
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;
|
|
|
|
std::vector<UID> src, dest;
|
|
|
|
ServerCacheInfo info;
|
|
|
|
// NOTE: An ACTOR will be compiled into several classes, the this pointer is from one of them.
|
|
|
|
auto updateTagInfo = [this](const std::vector<UID>& uids,
|
|
|
|
std::vector<Tag>& tags,
|
|
|
|
std::vector<Reference<StorageInfo>>& storageInfoItems) {
|
|
|
|
for (const auto& id : uids) {
|
|
|
|
auto storageInfo = getStorageInfo(id, &pContext->pCommitData->storageCache, pContext->pTxnStateStore);
|
|
|
|
ASSERT(storageInfo->tag != invalidTag);
|
|
|
|
tags.push_back(storageInfo->tag);
|
|
|
|
storageInfoItems.push_back(storageInfo);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
for (auto& kv : data) {
|
|
|
|
if (!kv.key.startsWith(keyServersPrefix)) {
|
|
|
|
mutations.emplace_back(mutations.arena(), MutationRef::SetValue, kv.key, kv.value);
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
|
|
|
|
KeyRef k = kv.key.removePrefix(keyServersPrefix);
|
|
|
|
if (k == allKeys.end) {
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
decodeKeyServersValue(tag_uid, kv.value, src, dest);
|
|
|
|
|
|
|
|
info.tags.clear();
|
|
|
|
|
|
|
|
info.src_info.clear();
|
|
|
|
updateTagInfo(src, info.tags, info.src_info);
|
|
|
|
|
|
|
|
info.dest_info.clear();
|
|
|
|
updateTagInfo(dest, info.tags, info.dest_info);
|
|
|
|
|
|
|
|
uniquify(info.tags);
|
|
|
|
keyInfoData.emplace_back(MapPair<Key, ServerCacheInfo>(k, info), 1);
|
|
|
|
}
|
|
|
|
|
|
|
|
// insert keyTag data separately from metadata mutations so that we can do one bulk insert which
|
|
|
|
// avoids a lot of map lookups.
|
|
|
|
pContext->pCommitData->keyInfo.rawInsert(keyInfoData);
|
|
|
|
|
|
|
|
Arena arena;
|
|
|
|
bool confChanges;
|
|
|
|
applyMetadataMutations(SpanID(),
|
|
|
|
*pContext->pCommitData,
|
|
|
|
arena,
|
|
|
|
Reference<ILogSystem>(),
|
|
|
|
mutations,
|
|
|
|
/* pToCommit= */ nullptr,
|
|
|
|
confChanges,
|
2022-02-19 13:22:31 +08:00
|
|
|
/* version= */ 0,
|
2021-08-28 14:38:34 +08:00
|
|
|
/* popVersion= */ 0,
|
|
|
|
/* initialCommit= */ true);
|
|
|
|
} // loop
|
|
|
|
|
|
|
|
auto lockedKey = pContext->pTxnStateStore->readValue(databaseLockedKey).get();
|
|
|
|
pContext->pCommitData->locked = lockedKey.present() && lockedKey.get().size();
|
|
|
|
pContext->pCommitData->metadataVersion = pContext->pTxnStateStore->readValue(metadataVersionKey).get();
|
|
|
|
|
|
|
|
pContext->pTxnStateStore->enableSnapshot();
|
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
ACTOR Future<Void> processTransactionStateRequestPart(TransactionStateResolveContext* pContext,
|
|
|
|
TxnStateRequest request) {
|
|
|
|
state const TxnStateRequest& req = request;
|
|
|
|
state ProxyCommitData& commitData = *pContext->pCommitData;
|
|
|
|
state PromiseStream<Future<Void>>& addActor = *pContext->pActors;
|
|
|
|
state Sequence& maxSequence = pContext->maxSequence;
|
|
|
|
state ReplyPromise<Void> reply = req.reply;
|
|
|
|
state std::unordered_set<Sequence>& txnSequences = pContext->receivedSequences;
|
|
|
|
|
|
|
|
ASSERT(pContext->pCommitData != nullptr);
|
|
|
|
ASSERT(pContext->pActors != nullptr);
|
|
|
|
|
|
|
|
if (pContext->receivedSequences.count(request.sequence)) {
|
2022-03-19 06:24:31 +08:00
|
|
|
if (pContext->receivedSequences.size() == pContext->maxSequence) {
|
|
|
|
wait(pContext->txnRecovery);
|
|
|
|
}
|
2021-08-28 14:38:34 +08:00
|
|
|
// This part is already received. Still we will re-broadcast it to other CommitProxies
|
|
|
|
pContext->pActors->send(broadcastTxnRequest(request, SERVER_KNOBS->TXN_STATE_SEND_AMOUNT, true));
|
|
|
|
wait(yield());
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (request.last) {
|
|
|
|
// This is the last piece of subsequence, yet other pieces might still on the way.
|
|
|
|
pContext->maxSequence = request.sequence + 1;
|
|
|
|
}
|
|
|
|
pContext->receivedSequences.insert(request.sequence);
|
|
|
|
|
|
|
|
// 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
|
|
|
|
ASSERT(!pContext->pCommitData->validState.isSet());
|
|
|
|
|
|
|
|
for (auto& kv : request.data) {
|
|
|
|
pContext->pTxnStateStore->set(kv, &request.arena);
|
|
|
|
}
|
|
|
|
pContext->pTxnStateStore->commit(true);
|
|
|
|
|
|
|
|
if (pContext->receivedSequences.size() == pContext->maxSequence) {
|
|
|
|
// Received all components of the txnStateRequest
|
|
|
|
ASSERT(!pContext->processed);
|
2022-03-19 06:24:31 +08:00
|
|
|
pContext->txnRecovery = processCompleteTransactionStateRequest(pContext);
|
|
|
|
wait(pContext->txnRecovery);
|
2021-08-28 14:38:34 +08:00
|
|
|
pContext->processed = true;
|
|
|
|
}
|
|
|
|
|
|
|
|
pContext->pActors->send(broadcastTxnRequest(request, SERVER_KNOBS->TXN_STATE_SEND_AMOUNT, true));
|
|
|
|
wait(yield());
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
} // anonymous namespace
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> commitProxyServerCore(CommitProxyInterface proxy,
|
|
|
|
MasterInterface master,
|
2022-01-07 04:15:51 +08:00
|
|
|
LifetimeToken masterLifetime,
|
2021-07-12 13:17:09 +08:00
|
|
|
Reference<AsyncVar<ServerDBInfo> const> db,
|
2021-03-11 02:06:03 +08:00
|
|
|
LogEpoch epoch,
|
|
|
|
Version recoveryTransactionVersion,
|
|
|
|
bool firstProxy,
|
2020-09-11 08:44:15 +08:00
|
|
|
std::string whitelistBinPaths) {
|
2021-03-11 02:06:03 +08:00
|
|
|
state ProxyCommitData commitData(
|
|
|
|
proxy.id(), master, proxy.getConsistentReadVersion, recoveryTransactionVersion, proxy.commit, db, firstProxy);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
state Future<Sequence> sequenceFuture = (Sequence)0;
|
2021-09-17 08:42:34 +08:00
|
|
|
state PromiseStream<std::pair<std::vector<CommitTransactionRequest>, int>> batchedCommits;
|
2018-11-10 06:19:18 +08:00
|
|
|
state Future<Void> commitBatcherActor;
|
2017-05-26 04:48:44 +08:00
|
|
|
state Future<Void> lastCommitComplete = Void();
|
|
|
|
|
|
|
|
state PromiseStream<Future<Void>> addActor;
|
2022-01-07 04:15:51 +08:00
|
|
|
state Future<Void> onError = transformError(actorCollection(addActor.getFuture()), broken_promise(), tlog_failed());
|
2017-05-26 04:48:44 +08:00
|
|
|
state double lastCommit = 0;
|
|
|
|
|
2019-02-24 02:13:43 +08:00
|
|
|
state GetHealthMetricsReply healthMetricsReply;
|
|
|
|
state GetHealthMetricsReply detailedHealthMetricsReply;
|
2019-02-01 05:25:57 +08:00
|
|
|
|
2021-03-11 02:06:03 +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
|
2022-01-07 04:15:51 +08:00
|
|
|
while (!(masterLifetime.isEqual(commitData.db->get().masterLifetime) &&
|
2021-03-11 02:06:03 +08:00
|
|
|
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);
|
2019-03-21 00:29:09 +08:00
|
|
|
wait(commitData.db->onChange());
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2019-03-21 00:29:09 +08:00
|
|
|
state Future<Void> dbInfoChange = commitData.db->onChange();
|
2017-05-26 04:48:44 +08:00
|
|
|
//TraceEvent("ProxyInit3", proxy.id());
|
|
|
|
|
2019-03-21 00:29:09 +08:00
|
|
|
commitData.resolvers = commitData.db->get().resolvers;
|
2017-05-26 04:48:44 +08:00
|
|
|
ASSERT(commitData.resolvers.size() != 0);
|
2021-08-06 00:17:37 +08:00
|
|
|
for (int i = 0; i < commitData.resolvers.size(); ++i) {
|
2021-08-28 08:07:47 +08:00
|
|
|
commitData.stats.resolverDist.push_back(
|
|
|
|
Histogram::getHistogram(LiteralStringRef("CommitProxy"),
|
|
|
|
"ToResolver_" + commitData.resolvers[i].id().toString(),
|
|
|
|
Histogram::Unit::microseconds));
|
2021-08-06 00:17:37 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
auto rs = commitData.keyResolvers.modify(allKeys);
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto r = rs.begin(); r != rs.end(); ++r)
|
|
|
|
r->value().emplace_back(0, 0);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2019-03-21 00:29:09 +08:00
|
|
|
commitData.logSystem = ILogSystem::fromServerDBInfo(proxy.id(), commitData.db->get(), false, addActor);
|
2021-03-11 02:06:03 +08:00
|
|
|
commitData.logAdapter =
|
|
|
|
new LogSystemDiskQueueAdapter(commitData.logSystem, Reference<AsyncVar<PeekTxsInfo>>(), 1, false);
|
2018-09-01 04:07:48 +08:00
|
|
|
commitData.txnStateStore = keyValueStoreLogSystem(commitData.logAdapter, proxy.id(), 2e9, true, true, true);
|
2019-04-13 04:23:02 +08:00
|
|
|
createWhitelistBinPathVec(whitelistBinPaths, commitData.whitelistedBinPathVec);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2019-11-21 03:44:18 +08:00
|
|
|
commitData.updateLatencyBandConfig(commitData.db->get().latencyBandConfig);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
// ((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.
|
2022-04-07 11:06:24 +08:00
|
|
|
state int64_t commitBatchesMemoryLimit = SERVER_KNOBS->COMMIT_BATCHES_MEM_BYTES_HARD_LIMIT;
|
|
|
|
if (SERVER_KNOBS->SERVER_MEM_LIMIT > 0) {
|
|
|
|
commitBatchesMemoryLimit = std::min(
|
|
|
|
commitBatchesMemoryLimit,
|
|
|
|
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));
|
|
|
|
}
|
2018-06-09 02:11:08 +08:00
|
|
|
TraceEvent(SevInfo, "CommitBatchesMemoryLimit").detail("BytesLimit", commitBatchesMemoryLimit);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2019-04-13 04:23:02 +08:00
|
|
|
addActor.send(monitorRemoteCommitted(&commitData));
|
2020-03-14 01:17:49 +08:00
|
|
|
addActor.send(readRequestServer(proxy, addActor, &commitData));
|
2019-04-26 08:07:35 +08:00
|
|
|
addActor.send(rejoinServer(proxy, &commitData));
|
2019-05-17 07:46:33 +08:00
|
|
|
addActor.send(ddMetricsRequestServer(proxy, db));
|
2020-08-30 03:35:31 +08:00
|
|
|
addActor.send(reportTxnTagCommitCost(proxy.id(), db, &commitData.ssTrTagCommitCost));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
// wait for txnStateStore recovery
|
2019-02-13 08:07:17 +08:00
|
|
|
wait(success(commitData.txnStateStore->readValue(StringRef())));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2019-08-09 06:00:33 +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(),
|
2019-08-09 06:00:33 +08:00
|
|
|
SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_BYTES_SCALE_POWER)));
|
2018-11-10 06:19:18 +08:00
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
commitBatcherActor = commitBatcher(
|
|
|
|
&commitData, batchedCommits, proxy.commit.getFuture(), commitBatchByteLimit, commitBatchesMemoryLimit);
|
2021-08-28 14:38:34 +08:00
|
|
|
|
|
|
|
// This has to be declared after the commitData.txnStateStore get initialized
|
|
|
|
state TransactionStateResolveContext transactionStateResolveContext(&commitData, &addActor);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
loop choose {
|
|
|
|
when(wait(dbInfoChange)) {
|
2019-03-21 00:29:09 +08:00
|
|
|
dbInfoChange = commitData.db->onChange();
|
2022-01-07 04:15:51 +08:00
|
|
|
if (masterLifetime.isEqual(commitData.db->get().masterLifetime) &&
|
2021-03-11 02:06:03 +08:00
|
|
|
commitData.db->get().recoveryState >= RecoveryState::RECOVERY_TRANSACTION) {
|
2019-03-21 00:29:09 +08:00
|
|
|
commitData.logSystem = ILogSystem::fromServerDBInfo(proxy.id(), commitData.db->get(), false, addActor);
|
2021-03-11 02:06:03 +08:00
|
|
|
for (auto it : commitData.tag_popped) {
|
2018-04-26 06:38:34 +08:00
|
|
|
commitData.logSystem->pop(it.second, it.first);
|
|
|
|
}
|
2019-06-20 09:15:09 +08:00
|
|
|
commitData.logSystem->popTxs(commitData.lastTxsPop, tagLocalityRemoteLog);
|
2019-01-19 08:18:34 +08:00
|
|
|
}
|
|
|
|
|
2019-11-21 03:44:18 +08:00
|
|
|
commitData.updateLatencyBandConfig(commitData.db->get().latencyBandConfig);
|
2018-04-26 01:16:31 +08:00
|
|
|
}
|
2018-08-11 04:57:10 +08:00
|
|
|
when(wait(onError)) {}
|
2021-09-17 08:42:34 +08:00
|
|
|
when(std::pair<std::vector<CommitTransactionRequest>, int> batchedRequests =
|
|
|
|
waitNext(batchedCommits.getFuture())) {
|
2021-03-11 02:06:03 +08:00
|
|
|
// WARNING: this code is run at a high priority, so it needs to do as little work as possible
|
2021-09-17 08:42:34 +08:00
|
|
|
const std::vector<CommitTransactionRequest>& trs = batchedRequests.first;
|
2018-06-02 06:21:40 +08:00
|
|
|
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);
|
2022-01-07 04:15:51 +08:00
|
|
|
//TraceEvent("CommitProxyCore", commitData.dbgid).detail("TxSize", trs.size()).detail("MasterLifetime", masterLifetime.toString()).detail("DbMasterLifetime", commitData.db->get().masterLifetime.toString()).detail("RecoveryState", commitData.db->get().recoveryState).detail("CCInf", commitData.db->get().clusterInterface.id().toString());
|
2021-03-11 02:06:03 +08:00
|
|
|
if (trs.size() || (commitData.db->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS &&
|
2022-01-07 04:15:51 +08:00
|
|
|
masterLifetime.isEqual(commitData.db->get().masterLifetime) &&
|
2021-03-11 02:06:03 +08:00
|
|
|
now() - lastCommit >= SERVER_KNOBS->MAX_COMMIT_BATCH_INTERVAL)) {
|
2017-05-26 04:48:44 +08:00
|
|
|
lastCommit = now();
|
|
|
|
|
|
|
|
if (trs.size() || lastCommitComplete.isReady()) {
|
2021-07-20 12:00:28 +08:00
|
|
|
lastCommitComplete = transformError(
|
|
|
|
timeoutError(
|
|
|
|
commitBatch(&commitData,
|
|
|
|
const_cast<std::vector<CommitTransactionRequest>*>(&batchedRequests.first),
|
|
|
|
batchBytes),
|
|
|
|
SERVER_KNOBS->COMMIT_PROXY_LIVENESS_TIMEOUT),
|
|
|
|
timed_out(),
|
|
|
|
failed_to_progress());
|
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())) {
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent(SevDebug, "SnapMasterEnqueue").log();
|
2019-06-20 02:11:50 +08:00
|
|
|
addActor.send(proxySnapCreate(snapReq, &commitData));
|
2019-02-28 07:40:33 +08:00
|
|
|
}
|
2019-08-09 07:30:05 +08:00
|
|
|
when(ExclusionSafetyCheckRequest exclCheckReq = waitNext(proxy.exclusionSafetyCheckReq.getFuture())) {
|
|
|
|
addActor.send(proxyCheckSafeExclusion(db, exclCheckReq));
|
2019-02-28 07:40:33 +08:00
|
|
|
}
|
2021-08-28 14:38:34 +08:00
|
|
|
when(TxnStateRequest request = waitNext(proxy.txnState.getFuture())) {
|
2021-08-29 12:10:10 +08:00
|
|
|
addActor.send(processTransactionStateRequestPart(&transactionStateResolveContext, request));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-07-12 13:17:09 +08:00
|
|
|
ACTOR Future<Void> checkRemoved(Reference<AsyncVar<ServerDBInfo> const> db,
|
2021-03-11 02:06:03 +08:00
|
|
|
uint64_t recoveryCount,
|
2020-09-11 08:44:15 +08:00
|
|
|
CommitProxyInterface myInterface) {
|
2021-03-11 02:06:03 +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)) {
|
2019-07-09 03:53:40 +08:00
|
|
|
throw worker_removed();
|
|
|
|
}
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(db->onChange());
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
ACTOR Future<Void> commitProxyServer(CommitProxyInterface proxy,
|
|
|
|
InitializeCommitProxyRequest req,
|
2021-07-12 13:17:09 +08:00
|
|
|
Reference<AsyncVar<ServerDBInfo> const> db,
|
2021-03-11 02:06:03 +08:00
|
|
|
std::string whitelistBinPaths) {
|
2017-05-26 04:48:44 +08:00
|
|
|
try {
|
2021-03-11 02:06:03 +08:00
|
|
|
state Future<Void> core = commitProxyServerCore(proxy,
|
|
|
|
req.master,
|
2022-01-07 04:15:51 +08:00
|
|
|
req.masterLifetime,
|
2021-03-11 02:06:03 +08:00
|
|
|
db,
|
|
|
|
req.recoveryCount,
|
|
|
|
req.recoveryTransactionVersion,
|
|
|
|
req.firstProxy,
|
|
|
|
whitelistBinPaths);
|
2019-07-09 03:53:40 +08:00
|
|
|
wait(core || checkRemoved(db, req.recoveryCount, proxy));
|
2021-03-11 02:06:03 +08:00
|
|
|
} catch (Error& e) {
|
2022-02-25 04:25:52 +08:00
|
|
|
TraceEvent("CommitProxyTerminated", proxy.id()).errorUnsuppressed(e);
|
2019-07-09 03:53:40 +08:00
|
|
|
|
|
|
|
if (e.code() != error_code_worker_removed && e.code() != error_code_tlog_stopped &&
|
2022-01-07 04:15:51 +08:00
|
|
|
e.code() != error_code_tlog_failed && e.code() != error_code_coordinators_changed &&
|
2021-07-20 12:00:28 +08:00
|
|
|
e.code() != error_code_coordinated_state_conflict && e.code() != error_code_new_coordinators_timed_out &&
|
|
|
|
e.code() != error_code_failed_to_progress) {
|
2019-07-09 03:53:40 +08:00
|
|
|
throw;
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2021-07-20 12:00:28 +08:00
|
|
|
TEST(e.code() == error_code_failed_to_progress); // Commit proxy failed to progress
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2019-07-28 07:46:22 +08:00
|
|
|
return Void();
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|