foundationdb/fdbserver/QuietDatabase.actor.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

929 lines
38 KiB
C++
Raw Normal View History

2017-05-26 04:48:44 +08:00
/*
* QuietDatabase.actor.cpp
*
* 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
*
2017-05-26 04:48:44 +08:00
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
2017-05-26 04:48:44 +08:00
* http://www.apache.org/licenses/LICENSE-2.0
*
2017-05-26 04:48:44 +08:00
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include <cinttypes>
2021-10-20 23:54:19 +08:00
#include <vector>
#include <type_traits>
2021-10-20 23:54:19 +08:00
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/SystemData.h"
2017-05-26 04:48:44 +08:00
#include "flow/ActorCollection.h"
#include "fdbrpc/simulator.h"
#include "flow/Trace.h"
#include "fdbclient/DatabaseContext.h"
#include "fdbclient/NativeAPI.actor.h"
#include "fdbclient/ReadYourWrites.h"
#include "fdbclient/RunTransaction.actor.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/TesterInterface.actor.h"
#include "fdbserver/WorkerInterface.actor.h"
#include "fdbserver/ServerDBInfo.h"
#include "fdbserver/Status.h"
#include "fdbclient/ManagementAPI.actor.h"
2019-05-22 01:53:36 +08:00
#include <boost/lexical_cast.hpp>
#include "flow/actorcompiler.h" // This must be the last #include.
#include "flow/flow.h"
2017-05-26 04:48:44 +08:00
ACTOR Future<std::vector<WorkerDetails>> getWorkers(Reference<AsyncVar<ServerDBInfo> const> dbInfo, int flags = 0) {
2017-05-26 04:48:44 +08:00
loop {
choose {
when(std::vector<WorkerDetails> w = wait(brokenPromiseToNever(
dbInfo->get().clusterInterface.getWorkers.getReply(GetWorkersRequest(flags))))) {
2017-05-26 04:48:44 +08:00
return w;
}
when(wait(dbInfo->onChange())) {}
2017-05-26 04:48:44 +08:00
}
}
}
// Gets the WorkerInterface representing the Master server.
ACTOR Future<WorkerInterface> getMasterWorker(Database cx, Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
TraceEvent("GetMasterWorker").detail("Stage", "GettingWorkers");
2017-05-26 04:48:44 +08:00
loop {
state std::vector<WorkerDetails> workers = wait(getWorkers(dbInfo));
2017-05-26 04:48:44 +08:00
for (int i = 0; i < workers.size(); i++) {
if (workers[i].interf.address() == dbInfo->get().master.address()) {
TraceEvent("GetMasterWorker")
.detail("Stage", "GotWorkers")
.detail("MasterId", dbInfo->get().master.id())
.detail("WorkerId", workers[i].interf.id());
return workers[i].interf;
2017-05-26 04:48:44 +08:00
}
}
TraceEvent(SevWarn, "GetMasterWorkerError")
.detail("Error", "MasterWorkerNotFound")
2017-05-26 04:48:44 +08:00
.detail("Master", dbInfo->get().master.id())
.detail("MasterAddress", dbInfo->get().master.address())
.detail("WorkerCount", workers.size());
wait(delay(1.0));
2017-05-26 04:48:44 +08:00
}
}
// Gets the WorkerInterface representing the data distributor.
ACTOR Future<WorkerInterface> getDataDistributorWorker(Database cx, Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
TraceEvent("GetDataDistributorWorker").detail("Stage", "GettingWorkers");
loop {
state std::vector<WorkerDetails> workers = wait(getWorkers(dbInfo));
if (!dbInfo->get().distributor.present())
continue;
for (int i = 0; i < workers.size(); i++) {
if (workers[i].interf.address() == dbInfo->get().distributor.get().address()) {
TraceEvent("GetDataDistributorWorker")
.detail("Stage", "GotWorkers")
.detail("DataDistributorId", dbInfo->get().distributor.get().id())
.detail("WorkerId", workers[i].interf.id());
return workers[i].interf;
}
}
TraceEvent(SevWarn, "GetDataDistributorWorker")
.detail("Error", "DataDistributorWorkerNotFound")
.detail("DataDistributorId", dbInfo->get().distributor.get().id())
.detail("DataDistributorAddress", dbInfo->get().distributor.get().address())
.detail("WorkerCount", workers.size());
}
}
// Gets the number of bytes in flight from the data distributor.
ACTOR Future<int64_t> getDataInFlight(Database cx, WorkerInterface distributorWorker) {
2017-05-26 04:48:44 +08:00
try {
TraceEvent("DataInFlight").detail("Stage", "ContactingDataDistributor");
TraceEventFields md = wait(timeoutError(
distributorWorker.eventLogRequest.getReply(EventLogRequest(LiteralStringRef("TotalDataInFlight"))), 1.0));
2019-05-22 01:53:36 +08:00
int64_t dataInFlight = boost::lexical_cast<int64_t>(md.getValue("TotalBytes"));
2017-05-26 04:48:44 +08:00
return dataInFlight;
} catch (Error& e) {
TraceEvent("QuietDatabaseFailure", distributorWorker.id())
.error(e)
.detail("Reason", "Failed to extract DataInFlight");
2017-05-26 04:48:44 +08:00
throw;
}
}
// Gets the number of bytes in flight from the data distributor.
ACTOR Future<int64_t> getDataInFlight(Database cx, Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
WorkerInterface distributorInterf = wait(getDataDistributorWorker(cx, dbInfo));
int64_t dataInFlight = wait(getDataInFlight(cx, distributorInterf));
2017-05-26 04:48:44 +08:00
return dataInFlight;
}
// Computes the queue size for storage servers and tlogs using the bytesInput and bytesDurable attributes
int64_t getQueueSize(const TraceEventFields& md) {
double inputRate, durableRate;
double inputRoughness, durableRoughness;
int64_t inputBytes, durableBytes;
2017-05-26 04:48:44 +08:00
sscanf(md.getValue("BytesInput").c_str(), "%lf %lf %" SCNd64, &inputRate, &inputRoughness, &inputBytes);
sscanf(md.getValue("BytesDurable").c_str(), "%lf %lf %" SCNd64, &durableRate, &durableRoughness, &durableBytes);
2017-05-26 04:48:44 +08:00
return inputBytes - durableBytes;
2017-05-26 04:48:44 +08:00
}
int64_t getDurableVersion(const TraceEventFields& md) {
return boost::lexical_cast<int64_t>(md.getValue("DurableVersion"));
}
// Computes the popped version lag for tlogs
int64_t getPoppedVersionLag(const TraceEventFields& md) {
2019-05-22 01:53:36 +08:00
int64_t persistentDataDurableVersion = boost::lexical_cast<int64_t>(md.getValue("PersistentDataDurableVersion"));
int64_t queuePoppedVersion = boost::lexical_cast<int64_t>(md.getValue("QueuePoppedVersion"));
return persistentDataDurableVersion - queuePoppedVersion;
}
ACTOR Future<std::vector<WorkerInterface>> getCoordWorkers(Database cx,
Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
state std::vector<WorkerDetails> workers = wait(getWorkers(dbInfo));
Optional<Value> coordinators =
wait(runRYWTransaction(cx, [=](Reference<ReadYourWritesTransaction> tr) -> Future<Optional<Value>> {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
return tr->get(coordinatorsKey);
}));
if (!coordinators.present()) {
throw operation_failed();
}
2022-04-28 12:54:13 +08:00
ClusterConnectionString ccs(coordinators.get().toString());
std::vector<NetworkAddress> coordinatorsAddr = wait(ccs.tryResolveHostnames());
std::set<NetworkAddress> coordinatorsAddrSet;
for (const auto& addr : coordinatorsAddr) {
TraceEvent(SevDebug, "CoordinatorAddress").detail("Addr", addr);
coordinatorsAddrSet.insert(addr);
}
std::vector<WorkerInterface> result;
for (const auto& worker : workers) {
NetworkAddress primary = worker.interf.address();
Optional<NetworkAddress> secondary = worker.interf.tLog.getEndpoint().addresses.secondaryAddress;
if (coordinatorsAddrSet.find(primary) != coordinatorsAddrSet.end() ||
(secondary.present() && (coordinatorsAddrSet.find(secondary.get()) != coordinatorsAddrSet.end()))) {
result.push_back(worker.interf);
}
}
return result;
}
2017-05-26 04:48:44 +08:00
// This is not robust in the face of a TLog failure
ACTOR Future<std::pair<int64_t, int64_t>> getTLogQueueInfo(Database cx,
Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
TraceEvent("MaxTLogQueueSize").detail("Stage", "ContactingLogs");
state std::vector<WorkerDetails> workers = wait(getWorkers(dbInfo));
std::map<NetworkAddress, WorkerInterface> workersMap;
for (auto worker : workers) {
workersMap[worker.interf.address()] = worker.interf;
}
2017-05-26 04:48:44 +08:00
state std::vector<Future<TraceEventFields>> messages;
2017-05-26 04:48:44 +08:00
state std::vector<TLogInterface> tlogs = dbInfo->get().logSystemConfig.allPresentLogs();
for (int i = 0; i < tlogs.size(); i++) {
auto itr = workersMap.find(tlogs[i].address());
if (itr == workersMap.end()) {
TraceEvent("QuietDatabaseFailure")
.detail("Reason", "Could not find worker for log server")
.detail("Tlog", tlogs[i].id());
throw attribute_not_found();
}
messages.push_back(timeoutError(
itr->second.eventLogRequest.getReply(EventLogRequest(StringRef(tlogs[i].id().toString() + "/TLogMetrics"))),
1.0));
2017-05-26 04:48:44 +08:00
}
wait(waitForAll(messages));
2017-05-26 04:48:44 +08:00
TraceEvent("MaxTLogQueueSize").detail("Stage", "ComputingMax").detail("MessageCount", messages.size());
2017-05-26 04:48:44 +08:00
state int64_t maxQueueSize = 0;
state int64_t maxPoppedVersionLag = 0;
2017-05-26 04:48:44 +08:00
state int i = 0;
for (; i < messages.size(); i++) {
try {
maxQueueSize = std::max(maxQueueSize, getQueueSize(messages[i].get()));
maxPoppedVersionLag = std::max(maxPoppedVersionLag, getPoppedVersionLag(messages[i].get()));
2017-05-26 04:48:44 +08:00
} catch (Error& e) {
TraceEvent("QuietDatabaseFailure")
.detail("Reason", "Failed to extract MaxTLogQueue")
.detail("Tlog", tlogs[i].id());
2017-05-26 04:48:44 +08:00
throw;
}
}
return std::make_pair(maxQueueSize, maxPoppedVersionLag);
2017-05-26 04:48:44 +08:00
}
2021-09-29 07:15:32 +08:00
// Returns a vector of blob worker interfaces which have been persisted under the system key space
ACTOR Future<std::vector<BlobWorkerInterface>> getBlobWorkers(Database cx,
bool use_system_priority = false,
Version* grv = nullptr) {
state Transaction tr(cx);
loop {
if (use_system_priority) {
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
}
tr.setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
try {
RangeResult blobWorkersList = wait(tr.getRange(blobWorkerListKeys, CLIENT_KNOBS->TOO_MANY));
ASSERT(!blobWorkersList.more && blobWorkersList.size() < CLIENT_KNOBS->TOO_MANY);
std::vector<BlobWorkerInterface> blobWorkers;
blobWorkers.reserve(blobWorkersList.size());
for (int i = 0; i < blobWorkersList.size(); i++) {
blobWorkers.push_back(decodeBlobWorkerListValue(blobWorkersList[i].value));
}
if (grv) {
*grv = tr.getReadVersion().get();
}
return blobWorkers;
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
ACTOR Future<std::vector<StorageServerInterface>> getStorageServers(Database cx, bool use_system_priority = false) {
2017-05-26 04:48:44 +08:00
state Transaction tr(cx);
loop {
if (use_system_priority) {
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
}
tr.setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
2017-05-26 04:48:44 +08:00
try {
2021-05-04 04:14:16 +08:00
RangeResult serverList = wait(tr.getRange(serverListKeys, CLIENT_KNOBS->TOO_MANY));
2017-05-26 04:48:44 +08:00
ASSERT(!serverList.more && serverList.size() < CLIENT_KNOBS->TOO_MANY);
std::vector<StorageServerInterface> servers;
servers.reserve(serverList.size());
for (int i = 0; i < serverList.size(); i++)
servers.push_back(decodeServerListValue(serverList[i].value));
2017-05-26 04:48:44 +08:00
return servers;
} catch (Error& e) {
wait(tr.onError(e));
2017-05-26 04:48:44 +08:00
}
}
}
ACTOR Future<std::pair<std::vector<WorkerInterface>, int>>
getStorageWorkers(Database cx, Reference<AsyncVar<ServerDBInfo> const> dbInfo, bool localOnly) {
state std::vector<StorageServerInterface> servers = wait(getStorageServers(cx));
state std::map<NetworkAddress, WorkerInterface> workersMap;
std::vector<WorkerDetails> workers = wait(getWorkers(dbInfo));
for (const auto& worker : workers) {
workersMap[worker.interf.address()] = worker.interf;
}
Optional<Value> regionsValue =
wait(runRYWTransaction(cx, [=](Reference<ReadYourWritesTransaction> tr) -> Future<Optional<Value>> {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
return tr->get(LiteralStringRef("usable_regions").withPrefix(configKeysPrefix));
}));
int usableRegions = 1;
if (regionsValue.present()) {
usableRegions = atoi(regionsValue.get().toString().c_str());
}
auto masterDcId = dbInfo->get().master.locality.dcId();
std::pair<std::vector<WorkerInterface>, int> result;
auto& [workerInterfaces, failures] = result;
failures = 0;
for (const auto& server : servers) {
TraceEvent(SevDebug, "DcIdInfo")
.detail("ServerLocalityID", server.locality.dcId())
.detail("MasterDcID", masterDcId);
if (!localOnly || (usableRegions == 1 || server.locality.dcId() == masterDcId)) {
auto itr = workersMap.find(server.address());
if (itr == workersMap.end()) {
TraceEvent(SevWarn, "GetStorageWorkers")
.detail("Reason", "Could not find worker for storage server")
.detail("SS", server.id());
++failures;
2022-04-09 02:21:29 +08:00
} else {
workerInterfaces.push_back(itr->second);
}
}
}
return result;
}
2021-06-16 06:49:27 +08:00
// Helper function to extract he maximum SQ size of all provided messages. All futures in the
// messages vector have to be ready.
2021-06-16 06:00:28 +08:00
int64_t extractMaxQueueSize(const std::vector<Future<TraceEventFields>>& messages,
const std::vector<StorageServerInterface>& servers) {
int64_t maxQueueSize = 0;
UID maxQueueServer;
for (int i = 0; i < messages.size(); i++) {
try {
auto queueSize = getQueueSize(messages[i].get());
if (queueSize > maxQueueSize) {
maxQueueSize = queueSize;
maxQueueServer = servers[i].id();
}
} catch (Error& e) {
TraceEvent("QuietDatabaseFailure")
.detail("Reason", "Failed to extract MaxStorageServerQueue")
.detail("SS", servers[i].id());
for (auto& m : messages) {
TraceEvent("Messages").detail("Info", m.get().toString());
}
throw;
}
}
TraceEvent("QuietDatabaseGotMaxStorageServerQueueSize")
.detail("Stage", "MaxComputed")
.detail("Max", maxQueueSize)
.detail("MaxQueueServer", format("%016" PRIx64, maxQueueServer.first()));
return maxQueueSize;
}
2021-06-16 06:49:27 +08:00
// Timeout wrapper when getting the storage metrics. This will do some additional tracing
ACTOR Future<TraceEventFields> getStorageMetricsTimeout(UID storage, WorkerInterface wi, Version version) {
state int retries = 0;
loop {
++retries;
state Future<TraceEventFields> result =
wi.eventLogRequest.getReply(EventLogRequest(StringRef(storage.toString() + "/StorageMetrics")));
state Future<Void> timeout = delay(30.0);
choose {
when(TraceEventFields res = wait(result)) {
if (version == invalidVersion || getDurableVersion(res) >= static_cast<int64_t>(version)) {
return res;
}
}
when(wait(timeout)) {
TraceEvent("QuietDatabaseFailure")
.detail("Reason", "Could not fetch StorageMetrics")
.detail("Storage", format("%016" PRIx64, storage.first()));
throw timed_out();
}
}
if (retries > 30) {
2021-06-16 06:00:28 +08:00
TraceEvent("QuietDatabaseFailure")
.detail("Reason", "Could not fetch StorageMetrics x30")
.detail("Storage", format("%016" PRIx64, storage.first()))
.detail("Version", version);
2021-06-16 06:00:28 +08:00
throw timed_out();
}
wait(delay(1.0));
2021-06-16 06:00:28 +08:00
}
}
2021-06-16 06:00:28 +08:00
2017-05-26 04:48:44 +08:00
// Gets the maximum size of all the storage server queues
ACTOR Future<int64_t> getMaxStorageServerQueueSize(Database cx,
Reference<AsyncVar<ServerDBInfo> const> dbInfo,
Version version) {
TraceEvent("MaxStorageServerQueueSize").detail("Stage", "ContactingStorageServers");
Future<std::vector<StorageServerInterface>> serversFuture = getStorageServers(cx);
state Future<std::vector<WorkerDetails>> workersFuture = getWorkers(dbInfo);
state std::vector<StorageServerInterface> servers = wait(serversFuture);
state std::vector<WorkerDetails> workers = wait(workersFuture);
std::map<NetworkAddress, WorkerInterface> workersMap;
for (auto worker : workers) {
workersMap[worker.interf.address()] = worker.interf;
}
2017-05-26 04:48:44 +08:00
state std::vector<Future<TraceEventFields>> messages;
2017-05-26 04:48:44 +08:00
for (int i = 0; i < servers.size(); i++) {
auto itr = workersMap.find(servers[i].address());
if (itr == workersMap.end()) {
TraceEvent("QuietDatabaseFailure")
.detail("Reason", "Could not find worker for storage server")
.detail("SS", servers[i].id());
throw attribute_not_found();
}
messages.push_back(getStorageMetricsTimeout(servers[i].id(), itr->second, version));
2017-05-26 04:48:44 +08:00
}
wait(waitForAll(messages));
2017-05-26 04:48:44 +08:00
TraceEvent("MaxStorageServerQueueSize").detail("Stage", "ComputingMax").detail("MessageCount", messages.size());
2017-05-26 04:48:44 +08:00
2021-06-16 06:00:28 +08:00
return extractMaxQueueSize(messages, servers);
2017-05-26 04:48:44 +08:00
}
// Gets the size of the data distribution queue. If reportInFlight is true, then data in flight is considered part of
// the queue
ACTOR Future<int64_t> getDataDistributionQueueSize(Database cx,
WorkerInterface distributorWorker,
bool reportInFlight) {
2017-05-26 04:48:44 +08:00
try {
TraceEvent("DataDistributionQueueSize").detail("Stage", "ContactingDataDistributor");
2017-05-26 04:48:44 +08:00
TraceEventFields movingDataMessage = wait(timeoutError(
distributorWorker.eventLogRequest.getReply(EventLogRequest(LiteralStringRef("MovingData"))), 1.0));
2017-05-26 04:48:44 +08:00
TraceEvent("DataDistributionQueueSize").detail("Stage", "GotString");
2017-05-26 04:48:44 +08:00
2019-05-22 01:53:36 +08:00
int64_t inQueue = boost::lexical_cast<int64_t>(movingDataMessage.getValue("InQueue"));
2017-05-26 04:48:44 +08:00
if (reportInFlight) {
2019-05-22 01:53:36 +08:00
int64_t inFlight = boost::lexical_cast<int64_t>(movingDataMessage.getValue("InFlight"));
2017-05-26 04:48:44 +08:00
inQueue += inFlight;
}
return inQueue;
} catch (Error& e) {
TraceEvent("QuietDatabaseFailure", distributorWorker.id())
.detail("Reason", "Failed to extract DataDistributionQueueSize");
2017-05-26 04:48:44 +08:00
throw;
}
}
// Gets the size of the data distribution queue. If reportInFlight is true, then data in flight is considered part of
// the queue Convenience method that first finds the master worker from a zookeeper interface
ACTOR Future<int64_t> getDataDistributionQueueSize(Database cx,
Reference<AsyncVar<ServerDBInfo> const> dbInfo,
2017-05-26 04:48:44 +08:00
bool reportInFlight) {
WorkerInterface distributorInterf = wait(getDataDistributorWorker(cx, dbInfo));
int64_t inQueue = wait(getDataDistributionQueueSize(cx, distributorInterf, reportInFlight));
2017-05-26 04:48:44 +08:00
return inQueue;
}
2019-02-13 05:41:18 +08:00
// Gets if the number of process and machine teams does not exceed the maximum allowed number of teams
2019-02-16 09:03:40 +08:00
ACTOR Future<bool> getTeamCollectionValid(Database cx, WorkerInterface dataDistributorWorker) {
state int attempts = 0;
state bool ret = false;
loop {
try {
if (!g_network->isSimulated()) {
return true;
}
TraceEvent("GetTeamCollectionValid").detail("Stage", "ContactingMaster");
2019-02-13 05:41:18 +08:00
TraceEventFields teamCollectionInfoMessage = wait(timeoutError(
2019-02-16 09:03:40 +08:00
dataDistributorWorker.eventLogRequest.getReply(EventLogRequest(LiteralStringRef("TeamCollectionInfo"))),
1.0));
TraceEvent("GetTeamCollectionValid").detail("Stage", "GotString");
state int64_t currentTeams =
boost::lexical_cast<int64_t>(teamCollectionInfoMessage.getValue("CurrentServerTeams"));
state int64_t desiredTeams =
boost::lexical_cast<int64_t>(teamCollectionInfoMessage.getValue("DesiredTeams"));
state int64_t maxTeams = boost::lexical_cast<int64_t>(teamCollectionInfoMessage.getValue("MaxTeams"));
state int64_t currentMachineTeams =
boost::lexical_cast<int64_t>(teamCollectionInfoMessage.getValue("CurrentMachineTeams"));
state int64_t healthyMachineTeams =
boost::lexical_cast<int64_t>(teamCollectionInfoMessage.getValue("CurrentHealthyMachineTeams"));
state int64_t desiredMachineTeams =
boost::lexical_cast<int64_t>(teamCollectionInfoMessage.getValue("DesiredMachineTeams"));
state int64_t maxMachineTeams =
boost::lexical_cast<int64_t>(teamCollectionInfoMessage.getValue("MaxMachineTeams"));
state int64_t minServerTeamsOnServer =
boost::lexical_cast<int64_t>(teamCollectionInfoMessage.getValue("MinTeamsOnServer"));
state int64_t maxServerTeamsOnServer =
boost::lexical_cast<int64_t>(teamCollectionInfoMessage.getValue("MaxTeamsOnServer"));
state int64_t minMachineTeamsOnMachine =
boost::lexical_cast<int64_t>(teamCollectionInfoMessage.getValue("MinMachineTeamsOnMachine"));
state int64_t maxMachineTeamsOnMachine =
boost::lexical_cast<int64_t>(teamCollectionInfoMessage.getValue("MaxMachineTeamsOnMachine"));
2019-07-12 13:05:20 +08:00
// The if condition should be consistent with the condition in serverTeamRemover() and
// machineTeamRemover() that decides if redundant teams exist.
2019-07-12 13:05:20 +08:00
// Team number is always valid when we disable teamRemover, which avoids false positive in simulation test.
// The minimun team number per server (and per machine) should be no less than 0 so that newly added machine
// can host data on it.
//
// If the machineTeamRemover does not remove the machine team with the most machine teams,
// we may oscillate between building more server teams by teamBuilder() and removing those teams by
// teamRemover To avoid false positive in simulation, we skip the consistency check in this case.
// This is a corner case. This is a work-around if case the team number requirements cannot be satisfied.
//
// The checking for too many teams is disabled because teamRemover may not remove a team if it leads to 0
// team on a server
//(!SERVER_KNOBS->TR_FLAG_DISABLE_MACHINE_TEAM_REMOVER &&
// healthyMachineTeams > desiredMachineTeams) ||
// (!SERVER_KNOBS->TR_FLAG_DISABLE_SERVER_TEAM_REMOVER && currentTeams > desiredTeams) ||
if ((minMachineTeamsOnMachine <= 0 || minServerTeamsOnServer <= 0) &&
SERVER_KNOBS->TR_FLAG_REMOVE_MT_WITH_MOST_TEAMS) {
ret = false;
if (attempts++ < 10) {
wait(delay(60));
continue; // We may not receive the most recent TeamCollectionInfo
}
2019-06-29 07:01:05 +08:00
// When DESIRED_TEAMS_PER_SERVER == 1, we see minMachineTeamOnMachine can be 0 in one out of 30k test
// cases. Only check DESIRED_TEAMS_PER_SERVER == 3 for now since it is mostly used configuration.
2019-07-12 13:05:20 +08:00
// TODO: Remove the constraint SERVER_KNOBS->DESIRED_TEAMS_PER_SERVER == 3 to ensure that
// the minimun team number per server (and per machine) is always > 0 for any number of replicas
TraceEvent("GetTeamCollectionValid")
.detail("CurrentServerTeams", currentTeams)
.detail("DesiredTeams", desiredTeams)
.detail("MaxTeams", maxTeams)
.detail("CurrentHealthyMachineTeams", healthyMachineTeams)
.detail("DesiredMachineTeams", desiredMachineTeams)
.detail("CurrentMachineTeams", currentMachineTeams)
.detail("MaxMachineTeams", maxMachineTeams)
.detail("MinTeamsOnServer", minServerTeamsOnServer)
.detail("MaxTeamsOnServer", maxServerTeamsOnServer)
.detail("MinMachineTeamsOnMachine", minMachineTeamsOnMachine)
.detail("MaxMachineTeamsOnMachine", maxMachineTeamsOnMachine)
2019-06-27 10:38:12 +08:00
.detail("DesiredTeamsPerServer", SERVER_KNOBS->DESIRED_TEAMS_PER_SERVER)
.detail("MaxTeamsPerServer", SERVER_KNOBS->MAX_TEAMS_PER_SERVER)
2019-07-16 09:40:14 +08:00
.detail("RemoveMTWithMostTeams", SERVER_KNOBS->TR_FLAG_REMOVE_MT_WITH_MOST_TEAMS);
return ret;
} else {
return true;
}
2019-02-13 05:41:18 +08:00
} catch (Error& e) {
2019-07-26 07:15:28 +08:00
if (e.code() == error_code_actor_cancelled) {
throw;
}
2019-02-16 09:03:40 +08:00
TraceEvent("QuietDatabaseFailure", dataDistributorWorker.id())
2019-02-13 05:41:18 +08:00
.detail("Reason", "Failed to extract GetTeamCollectionValid information");
attempts++;
2019-02-13 05:41:18 +08:00
if (attempts > 10) {
2019-02-16 09:03:40 +08:00
TraceEvent("QuietDatabaseNoTeamCollectionInfo", dataDistributorWorker.id())
2019-02-13 05:41:18 +08:00
.detail("Reason", "Had never called build team to build any team");
return true;
}
2019-02-13 05:41:18 +08:00
// throw;
wait(delay(10.0));
}
};
}
2019-02-13 05:41:18 +08:00
// Gets if the number of process and machine teams does not exceed the maximum allowed number of teams
// Convenience method that first finds the master worker from a zookeeper interface
ACTOR Future<bool> getTeamCollectionValid(Database cx, Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
2019-02-16 09:03:40 +08:00
WorkerInterface dataDistributorWorker = wait(getDataDistributorWorker(cx, dbInfo));
bool valid = wait(getTeamCollectionValid(cx, dataDistributorWorker));
return valid;
}
// Checks that data distribution is active
ACTOR Future<bool> getDataDistributionActive(Database cx, WorkerInterface distributorWorker) {
2017-05-26 04:48:44 +08:00
try {
TraceEvent("DataDistributionActive").detail("Stage", "ContactingDataDistributor");
2017-05-26 04:48:44 +08:00
TraceEventFields activeMessage = wait(timeoutError(
distributorWorker.eventLogRequest.getReply(EventLogRequest(LiteralStringRef("DDTrackerStarting"))), 1.0));
2017-05-26 04:48:44 +08:00
return activeMessage.getValue("State") == "Active";
2017-05-26 04:48:44 +08:00
} catch (Error& e) {
TraceEvent("QuietDatabaseFailure", distributorWorker.id())
.detail("Reason", "Failed to extract DataDistributionActive");
2017-05-26 04:48:44 +08:00
throw;
}
}
// Checks to see if any storage servers are being recruited
ACTOR Future<bool> getStorageServersRecruiting(Database cx, WorkerInterface distributorWorker, UID distributorUID) {
2017-05-26 04:48:44 +08:00
try {
TraceEvent("StorageServersRecruiting").detail("Stage", "ContactingDataDistributor");
TraceEventFields recruitingMessage =
wait(timeoutError(distributorWorker.eventLogRequest.getReply(
EventLogRequest(StringRef("StorageServerRecruitment_" + distributorUID.toString()))),
1.0));
2017-05-26 04:48:44 +08:00
TraceEvent("StorageServersRecruiting").detail("Message", recruitingMessage.toString());
2021-03-06 03:28:15 +08:00
if (recruitingMessage.getValue("State") == "Recruiting") {
std::string tssValue;
// if we're tss recruiting, that's fine because that can block indefinitely if only 1 free storage process
if (!recruitingMessage.tryGetValue("IsTSS", tssValue) || tssValue == "False") {
return true;
}
}
return false;
2017-05-26 04:48:44 +08:00
} catch (Error& e) {
TraceEvent("QuietDatabaseFailure", distributorWorker.id())
.detail("Reason", "Failed to extract StorageServersRecruiting")
.detail("DataDistributorID", distributorUID);
2017-05-26 04:48:44 +08:00
throw;
}
}
Add fdbcli command to read/write version epoch (#6480) * Initialize cluster version at wall-clock time Previously, new clusters would begin at version 0. After this change, clusters will initialize at a version matching wall-clock time. Instead of using the Unix epoch (or Windows epoch), FDB clusters will use a new epoch, defaulting to January 1, 2010, 01:00:00+00:00. In the future, this base epoch will be modifiable through fdbcli, allowing administrators to advance the cluster version. Basing the version off of time allows different FDB clusters to share data without running into version issues. * Send version epoch to master * Cleanup * Update fdbserver/storageserver.actor.cpp Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com> * Jump directly to expected version if possible * Fix initial version issue on storage servers * Add random recovery offset to start version in simulation * Type fixes * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Use correct recoveryTransactionVersion when recovering * Allow version epoch to be adjusted forwards (to decrease the version) * Set version epoch in simulation * Add quiet database check to ensure small version offset * Fix initial version issue on storage servers * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Add fdbcli command to read/write version epoch * Cause recovery when version epoch is set * Handle optional version epoch key * Add ability to clear the version epoch This causes version advancement to revert to the old methodology whereas versions attempt to advance by about a million versions per second, instead of trying to match the clock. * Update transaction access * Modify version epoch to use microseconds instead of seconds * Modify fdbcli version target API Move commands from `versionepoch` to `targetversion` top level command. * Add fdbcli tests for * Temporarily disable targetversion cli tests * Fix version epoch fetch issue * Fix Arena issue * Reduce max version jump in simulation to 1,000,000 * Rework fdbcli API It now requires two commands to fully switch a cluster to using the version epoch. First, enable the version epoch with `versionepoch enable` or `versionepoch set <versionepoch>`. At this point, versions will be given out at a faster or slower rate in an attempt to reach the expected version. Then, run `versionepoch commit` to perform a one time jump to the expected version. This is essentially irreversible. * Temporarily disable old targetversion tests * Cleanup * Move version epoch buggify to sequencer This will cause some issues with the QuietDatabase check for the version offset - namely, it won't do anything, since the version epoch is not being written to the txnStateStore in simulation. This will get fixed in the future. Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com>
2022-04-09 03:33:19 +08:00
// Gets the difference between the expected version (based on the version
// epoch) and the actual version.
ACTOR Future<int64_t> getVersionOffset(Database cx,
WorkerInterface distributorWorker,
Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
loop {
state Transaction tr(cx);
try {
TraceEvent("GetVersionOffset").detail("Stage", "ReadingVersionEpoch");
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
state Version rv = wait(tr.getReadVersion());
Optional<Standalone<StringRef>> versionEpochValue = wait(tr.get(versionEpochKey));
if (!versionEpochValue.present()) {
return 0;
}
int64_t versionEpoch = BinaryReader::fromStringRef<int64_t>(versionEpochValue.get(), Unversioned());
int64_t versionOffset = abs(rv - (g_network->timer() * SERVER_KNOBS->VERSIONS_PER_SECOND - versionEpoch));
return versionOffset;
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
ACTOR Future<Void> repairDeadDatacenter(Database cx,
Reference<AsyncVar<ServerDBInfo> const> dbInfo,
std::string context) {
if (g_network->isSimulated() && g_simulator.usableRegions > 1) {
bool primaryDead = g_simulator.datacenterDead(g_simulator.primaryDcId);
bool remoteDead = g_simulator.datacenterDead(g_simulator.remoteDcId);
// FIXME: the primary and remote can both be considered dead because excludes are not handled properly by the
// datacenterDead function
if (primaryDead && remoteDead) {
TraceEvent(SevWarnAlways, "CannotDisableFearlessConfiguration").log();
return Void();
}
if (primaryDead || remoteDead) {
TraceEvent(SevWarnAlways, "DisablingFearlessConfiguration")
.detail("Location", context)
.detail("Stage", "Repopulate")
.detail("RemoteDead", remoteDead)
.detail("PrimaryDead", primaryDead);
g_simulator.usableRegions = 1;
wait(success(ManagementAPI::changeConfig(
cx.getReference(),
(primaryDead ? g_simulator.disablePrimary : g_simulator.disableRemote) + " repopulate_anti_quorum=1",
true)));
while (dbInfo->get().recoveryState < RecoveryState::STORAGE_RECOVERED) {
wait(dbInfo->onChange());
}
TraceEvent(SevWarnAlways, "DisablingFearlessConfiguration")
.detail("Location", context)
.detail("Stage", "Usable_Regions");
wait(success(ManagementAPI::changeConfig(cx.getReference(), "usable_regions=1", true)));
}
}
return Void();
}
ACTOR Future<Void> reconfigureAfter(Database cx,
double time,
Reference<AsyncVar<ServerDBInfo> const> dbInfo,
std::string context) {
wait(delay(time));
wait(repairDeadDatacenter(cx, dbInfo, context));
return Void();
}
struct QuietDatabaseChecker {
double start = now();
double maxDDRunTime;
QuietDatabaseChecker(double maxDDRunTime) : maxDDRunTime(maxDDRunTime) {}
struct Impl {
double start;
std::string const& phase;
double maxDDRunTime;
std::vector<std::string> failReasons;
Impl(double start, const std::string& phase, const double maxDDRunTime)
: start(start), phase(phase), maxDDRunTime(maxDDRunTime) {}
2022-04-28 23:32:30 +08:00
template <class T, class Comparison = std::less_equal<>>
2022-04-29 00:09:06 +08:00
Impl& add(BaseTraceEvent& evt,
const char* name,
T value,
T expected,
Comparison const& cmp = std::less_equal<>()) {
std::string k = fmt::format("{}Gate", name);
evt.detail(name, value).detail(k.c_str(), expected);
if (!cmp(value, expected)) {
failReasons.push_back(name);
}
return *this;
}
bool success() {
bool timedOut = now() - start > maxDDRunTime;
if (!failReasons.empty()) {
std::string traceMessage = fmt::format("QuietDatabase{}Fail", phase);
std::string reasons = fmt::format("{}", fmt::join(failReasons, ", "));
TraceEvent(timedOut ? SevError : SevWarnAlways, traceMessage.c_str())
2022-04-29 00:09:06 +08:00
.detail("Reasons", reasons)
.detail("FailedAfter", now() - start)
.detail("Timeout", maxDDRunTime);
2022-04-29 00:09:06 +08:00
if (timedOut) {
// this bool is just created to make the assertion more readable
bool ddGotStuck = true;
// This assertion is here to make the test fail more quickly. If quietDatabase takes this
// long without completing, we can assume that the test will eventually time out. However,
// time outs are more annoying to debug. This will hopefully be easier to track down.
2022-04-29 01:32:35 +08:00
ASSERT(!ddGotStuck || !g_network->isSimulated());
2022-04-29 00:09:06 +08:00
}
return false;
}
return true;
}
};
Impl startIteration(std::string const& phase) const {
Impl res(start, phase, maxDDRunTime);
return res;
}
};
2021-06-16 06:49:27 +08:00
// Waits until a database quiets down (no data in flight, small tlog queue, low SQ, no active data distribution). This
// requires the database to be available and healthy in order to succeed.
2017-05-26 04:48:44 +08:00
ACTOR Future<Void> waitForQuietDatabase(Database cx,
Reference<AsyncVar<ServerDBInfo> const> dbInfo,
2017-05-26 04:48:44 +08:00
std::string phase,
int64_t dataInFlightGate = 2e6,
int64_t maxTLogQueueGate = 5e6,
int64_t maxStorageServerQueueGate = 5e6,
int64_t maxDataDistributionQueueSize = 0,
Add fdbcli command to read/write version epoch (#6480) * Initialize cluster version at wall-clock time Previously, new clusters would begin at version 0. After this change, clusters will initialize at a version matching wall-clock time. Instead of using the Unix epoch (or Windows epoch), FDB clusters will use a new epoch, defaulting to January 1, 2010, 01:00:00+00:00. In the future, this base epoch will be modifiable through fdbcli, allowing administrators to advance the cluster version. Basing the version off of time allows different FDB clusters to share data without running into version issues. * Send version epoch to master * Cleanup * Update fdbserver/storageserver.actor.cpp Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com> * Jump directly to expected version if possible * Fix initial version issue on storage servers * Add random recovery offset to start version in simulation * Type fixes * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Use correct recoveryTransactionVersion when recovering * Allow version epoch to be adjusted forwards (to decrease the version) * Set version epoch in simulation * Add quiet database check to ensure small version offset * Fix initial version issue on storage servers * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Add fdbcli command to read/write version epoch * Cause recovery when version epoch is set * Handle optional version epoch key * Add ability to clear the version epoch This causes version advancement to revert to the old methodology whereas versions attempt to advance by about a million versions per second, instead of trying to match the clock. * Update transaction access * Modify version epoch to use microseconds instead of seconds * Modify fdbcli version target API Move commands from `versionepoch` to `targetversion` top level command. * Add fdbcli tests for * Temporarily disable targetversion cli tests * Fix version epoch fetch issue * Fix Arena issue * Reduce max version jump in simulation to 1,000,000 * Rework fdbcli API It now requires two commands to fully switch a cluster to using the version epoch. First, enable the version epoch with `versionepoch enable` or `versionepoch set <versionepoch>`. At this point, versions will be given out at a faster or slower rate in an attempt to reach the expected version. Then, run `versionepoch commit` to perform a one time jump to the expected version. This is essentially irreversible. * Temporarily disable old targetversion tests * Cleanup * Move version epoch buggify to sequencer This will cause some issues with the QuietDatabase check for the version offset - namely, it won't do anything, since the version epoch is not being written to the txnStateStore in simulation. This will get fixed in the future. Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com>
2022-04-09 03:33:19 +08:00
int64_t maxPoppedVersionLag = 30e6,
int64_t maxVersionOffset = 1e6) {
state QuietDatabaseChecker checker(isBuggifyEnabled(BuggifyType::General) ? 3600.0 : 1000.0);
state Future<Void> reconfig =
reconfigureAfter(cx, 100 + (deterministicRandom()->random01() * 100), dbInfo, "QuietDatabase");
2021-06-16 06:00:28 +08:00
state Future<int64_t> dataInFlight;
state Future<std::pair<int64_t, int64_t>> tLogQueueInfo;
state Future<int64_t> dataDistributionQueueSize;
state Future<bool> teamCollectionValid;
state Future<int64_t> storageQueueSize;
state Future<bool> dataDistributionActive;
state Future<bool> storageServersRecruiting;
Add fdbcli command to read/write version epoch (#6480) * Initialize cluster version at wall-clock time Previously, new clusters would begin at version 0. After this change, clusters will initialize at a version matching wall-clock time. Instead of using the Unix epoch (or Windows epoch), FDB clusters will use a new epoch, defaulting to January 1, 2010, 01:00:00+00:00. In the future, this base epoch will be modifiable through fdbcli, allowing administrators to advance the cluster version. Basing the version off of time allows different FDB clusters to share data without running into version issues. * Send version epoch to master * Cleanup * Update fdbserver/storageserver.actor.cpp Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com> * Jump directly to expected version if possible * Fix initial version issue on storage servers * Add random recovery offset to start version in simulation * Type fixes * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Use correct recoveryTransactionVersion when recovering * Allow version epoch to be adjusted forwards (to decrease the version) * Set version epoch in simulation * Add quiet database check to ensure small version offset * Fix initial version issue on storage servers * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Add fdbcli command to read/write version epoch * Cause recovery when version epoch is set * Handle optional version epoch key * Add ability to clear the version epoch This causes version advancement to revert to the old methodology whereas versions attempt to advance by about a million versions per second, instead of trying to match the clock. * Update transaction access * Modify version epoch to use microseconds instead of seconds * Modify fdbcli version target API Move commands from `versionepoch` to `targetversion` top level command. * Add fdbcli tests for * Temporarily disable targetversion cli tests * Fix version epoch fetch issue * Fix Arena issue * Reduce max version jump in simulation to 1,000,000 * Rework fdbcli API It now requires two commands to fully switch a cluster to using the version epoch. First, enable the version epoch with `versionepoch enable` or `versionepoch set <versionepoch>`. At this point, versions will be given out at a faster or slower rate in an attempt to reach the expected version. Then, run `versionepoch commit` to perform a one time jump to the expected version. This is essentially irreversible. * Temporarily disable old targetversion tests * Cleanup * Move version epoch buggify to sequencer This will cause some issues with the QuietDatabase check for the version offset - namely, it won't do anything, since the version epoch is not being written to the txnStateStore in simulation. This will get fixed in the future. Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com>
2022-04-09 03:33:19 +08:00
state Future<int64_t> versionOffset;
2020-04-23 05:14:56 +08:00
auto traceMessage = "QuietDatabase" + phase + "Begin";
TraceEvent(traceMessage.c_str()).log();
2017-05-26 04:48:44 +08:00
// In a simulated environment, wait 5 seconds so that workers can move to their optimal locations
if (g_network->isSimulated())
wait(delay(5.0));
TraceEvent("QuietDatabaseWaitingOnFullRecovery").log();
while (dbInfo->get().recoveryState != RecoveryState::FULLY_RECOVERED) {
wait(dbInfo->onChange());
}
2021-06-04 03:58:28 +08:00
// The quiet database check (which runs at the end of every test) will always time out due to active data movement.
// To get around this, quiet Database will disable the perpetual wiggle in the setup phase.
printf("Set perpetual_storage_wiggle=0 ...\n");
state Version version = wait(setPerpetualStorageWiggle(cx, false, LockAware::True));
printf("Set perpetual_storage_wiggle=0 Done.\n");
2021-06-04 03:58:28 +08:00
// Require 3 consecutive successful quiet database checks spaced 2 second apart
2017-05-26 04:48:44 +08:00
state int numSuccesses = 0;
loop {
try {
TraceEvent("QuietDatabaseWaitingOnDataDistributor").log();
WorkerInterface distributorWorker = wait(getDataDistributorWorker(cx, dbInfo));
2020-01-17 13:21:25 +08:00
UID distributorUID = dbInfo->get().distributor.get().id();
TraceEvent("QuietDatabaseGotDataDistributor", distributorUID)
.detail("Locality", distributorWorker.locality.toString());
2021-06-16 06:00:28 +08:00
dataInFlight = getDataInFlight(cx, distributorWorker);
tLogQueueInfo = getTLogQueueInfo(cx, dbInfo);
dataDistributionQueueSize = getDataDistributionQueueSize(cx, distributorWorker, dataInFlightGate == 0);
teamCollectionValid = getTeamCollectionValid(cx, distributorWorker);
storageQueueSize = getMaxStorageServerQueueSize(cx, dbInfo, version);
2021-06-16 06:00:28 +08:00
dataDistributionActive = getDataDistributionActive(cx, distributorWorker);
storageServersRecruiting = getStorageServersRecruiting(cx, distributorWorker, distributorUID);
Add fdbcli command to read/write version epoch (#6480) * Initialize cluster version at wall-clock time Previously, new clusters would begin at version 0. After this change, clusters will initialize at a version matching wall-clock time. Instead of using the Unix epoch (or Windows epoch), FDB clusters will use a new epoch, defaulting to January 1, 2010, 01:00:00+00:00. In the future, this base epoch will be modifiable through fdbcli, allowing administrators to advance the cluster version. Basing the version off of time allows different FDB clusters to share data without running into version issues. * Send version epoch to master * Cleanup * Update fdbserver/storageserver.actor.cpp Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com> * Jump directly to expected version if possible * Fix initial version issue on storage servers * Add random recovery offset to start version in simulation * Type fixes * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Use correct recoveryTransactionVersion when recovering * Allow version epoch to be adjusted forwards (to decrease the version) * Set version epoch in simulation * Add quiet database check to ensure small version offset * Fix initial version issue on storage servers * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Add fdbcli command to read/write version epoch * Cause recovery when version epoch is set * Handle optional version epoch key * Add ability to clear the version epoch This causes version advancement to revert to the old methodology whereas versions attempt to advance by about a million versions per second, instead of trying to match the clock. * Update transaction access * Modify version epoch to use microseconds instead of seconds * Modify fdbcli version target API Move commands from `versionepoch` to `targetversion` top level command. * Add fdbcli tests for * Temporarily disable targetversion cli tests * Fix version epoch fetch issue * Fix Arena issue * Reduce max version jump in simulation to 1,000,000 * Rework fdbcli API It now requires two commands to fully switch a cluster to using the version epoch. First, enable the version epoch with `versionepoch enable` or `versionepoch set <versionepoch>`. At this point, versions will be given out at a faster or slower rate in an attempt to reach the expected version. Then, run `versionepoch commit` to perform a one time jump to the expected version. This is essentially irreversible. * Temporarily disable old targetversion tests * Cleanup * Move version epoch buggify to sequencer This will cause some issues with the QuietDatabase check for the version offset - namely, it won't do anything, since the version epoch is not being written to the txnStateStore in simulation. This will get fixed in the future. Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com>
2022-04-09 03:33:19 +08:00
versionOffset = getVersionOffset(cx, distributorWorker, dbInfo);
2017-05-26 04:48:44 +08:00
2020-01-17 13:21:25 +08:00
wait(success(dataInFlight) && success(tLogQueueInfo) && success(dataDistributionQueueSize) &&
success(teamCollectionValid) && success(storageQueueSize) && success(dataDistributionActive) &&
Add fdbcli command to read/write version epoch (#6480) * Initialize cluster version at wall-clock time Previously, new clusters would begin at version 0. After this change, clusters will initialize at a version matching wall-clock time. Instead of using the Unix epoch (or Windows epoch), FDB clusters will use a new epoch, defaulting to January 1, 2010, 01:00:00+00:00. In the future, this base epoch will be modifiable through fdbcli, allowing administrators to advance the cluster version. Basing the version off of time allows different FDB clusters to share data without running into version issues. * Send version epoch to master * Cleanup * Update fdbserver/storageserver.actor.cpp Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com> * Jump directly to expected version if possible * Fix initial version issue on storage servers * Add random recovery offset to start version in simulation * Type fixes * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Use correct recoveryTransactionVersion when recovering * Allow version epoch to be adjusted forwards (to decrease the version) * Set version epoch in simulation * Add quiet database check to ensure small version offset * Fix initial version issue on storage servers * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Add fdbcli command to read/write version epoch * Cause recovery when version epoch is set * Handle optional version epoch key * Add ability to clear the version epoch This causes version advancement to revert to the old methodology whereas versions attempt to advance by about a million versions per second, instead of trying to match the clock. * Update transaction access * Modify version epoch to use microseconds instead of seconds * Modify fdbcli version target API Move commands from `versionepoch` to `targetversion` top level command. * Add fdbcli tests for * Temporarily disable targetversion cli tests * Fix version epoch fetch issue * Fix Arena issue * Reduce max version jump in simulation to 1,000,000 * Rework fdbcli API It now requires two commands to fully switch a cluster to using the version epoch. First, enable the version epoch with `versionepoch enable` or `versionepoch set <versionepoch>`. At this point, versions will be given out at a faster or slower rate in an attempt to reach the expected version. Then, run `versionepoch commit` to perform a one time jump to the expected version. This is essentially irreversible. * Temporarily disable old targetversion tests * Cleanup * Move version epoch buggify to sequencer This will cause some issues with the QuietDatabase check for the version offset - namely, it won't do anything, since the version epoch is not being written to the txnStateStore in simulation. This will get fixed in the future. Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com>
2022-04-09 03:33:19 +08:00
success(storageServersRecruiting) && success(versionOffset));
2019-05-22 02:31:34 +08:00
Add fdbcli command to read/write version epoch (#6480) * Initialize cluster version at wall-clock time Previously, new clusters would begin at version 0. After this change, clusters will initialize at a version matching wall-clock time. Instead of using the Unix epoch (or Windows epoch), FDB clusters will use a new epoch, defaulting to January 1, 2010, 01:00:00+00:00. In the future, this base epoch will be modifiable through fdbcli, allowing administrators to advance the cluster version. Basing the version off of time allows different FDB clusters to share data without running into version issues. * Send version epoch to master * Cleanup * Update fdbserver/storageserver.actor.cpp Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com> * Jump directly to expected version if possible * Fix initial version issue on storage servers * Add random recovery offset to start version in simulation * Type fixes * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Use correct recoveryTransactionVersion when recovering * Allow version epoch to be adjusted forwards (to decrease the version) * Set version epoch in simulation * Add quiet database check to ensure small version offset * Fix initial version issue on storage servers * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Add fdbcli command to read/write version epoch * Cause recovery when version epoch is set * Handle optional version epoch key * Add ability to clear the version epoch This causes version advancement to revert to the old methodology whereas versions attempt to advance by about a million versions per second, instead of trying to match the clock. * Update transaction access * Modify version epoch to use microseconds instead of seconds * Modify fdbcli version target API Move commands from `versionepoch` to `targetversion` top level command. * Add fdbcli tests for * Temporarily disable targetversion cli tests * Fix version epoch fetch issue * Fix Arena issue * Reduce max version jump in simulation to 1,000,000 * Rework fdbcli API It now requires two commands to fully switch a cluster to using the version epoch. First, enable the version epoch with `versionepoch enable` or `versionepoch set <versionepoch>`. At this point, versions will be given out at a faster or slower rate in an attempt to reach the expected version. Then, run `versionepoch commit` to perform a one time jump to the expected version. This is essentially irreversible. * Temporarily disable old targetversion tests * Cleanup * Move version epoch buggify to sequencer This will cause some issues with the QuietDatabase check for the version offset - namely, it won't do anything, since the version epoch is not being written to the txnStateStore in simulation. This will get fixed in the future. Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com>
2022-04-09 03:33:19 +08:00
maxVersionOffset += dbInfo->get().recoveryCount * SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT;
2017-05-26 04:48:44 +08:00
auto check = checker.startIteration(phase);
std::string evtType = "QuietDatabase" + phase;
TraceEvent evt(evtType.c_str());
check.add(evt, "DataInFlight", dataInFlight.get(), dataInFlightGate)
.add(evt, "MaxTLogQueueSize", tLogQueueInfo.get().first, maxTLogQueueGate)
.add(evt, "MaxTLogPoppedVersionLag", tLogQueueInfo.get().second, maxPoppedVersionLag)
.add(evt, "DataDistributionQueueSize", dataDistributionQueueSize.get(), maxDataDistributionQueueSize)
.add(evt, "TeamCollectionValid", teamCollectionValid.get(), true, std::equal_to<>())
.add(evt, "MaxStorageQueueSize", storageQueueSize.get(), maxStorageServerQueueGate)
.add(evt, "DataDistributionActive", dataDistributionActive.get(), true, std::equal_to<>())
.add(evt, "StorageServersRecruiting", storageServersRecruiting.get(), false, std::equal_to<>())
.add(evt, "VersionOffset", versionOffset.get(), maxVersionOffset);
evt.detail("RecoveryCount", dbInfo->get().recoveryCount).detail("NumSuccesses", numSuccesses);
evt.log();
if (check.success()) {
if (++numSuccesses == 3) {
2020-04-23 05:14:56 +08:00
auto msg = "QuietDatabase" + phase + "Done";
TraceEvent(msg.c_str()).log();
2017-05-26 04:48:44 +08:00
break;
2019-08-15 05:19:50 +08:00
} else {
wait(delay(g_network->isSimulated() ? 2.0 : 30.0));
2017-05-26 04:48:44 +08:00
}
} else {
wait(delay(1.0));
numSuccesses = 0;
2017-05-26 04:48:44 +08:00
}
} catch (Error& e) {
TraceEvent(("QuietDatabase" + phase + "Error").c_str()).errorUnsuppressed(e);
2019-08-15 05:19:50 +08:00
if (e.code() != error_code_actor_cancelled && e.code() != error_code_attribute_not_found &&
e.code() != error_code_timed_out)
2017-05-26 04:48:44 +08:00
TraceEvent(("QuietDatabase" + phase + "Error").c_str()).error(e);
// Client invalid operation occurs if we don't get back a message from one of the servers, often corrected
// by retrying
if (e.code() != error_code_attribute_not_found && e.code() != error_code_timed_out)
throw;
2021-06-16 06:00:28 +08:00
auto evtType = "QuietDatabase" + phase + "Retry";
TraceEvent evt(evtType.c_str());
evt.error(e);
int notReadyCount = 0;
if (dataInFlight.isReady() && dataInFlight.isError()) {
auto key = "NotReady" + std::to_string(notReadyCount++);
evt.detail(key.c_str(), "dataInFlight");
}
if (tLogQueueInfo.isReady() && tLogQueueInfo.isError()) {
auto key = "NotReady" + std::to_string(notReadyCount++);
evt.detail(key.c_str(), "tLogQueueInfo");
}
if (dataDistributionQueueSize.isReady() && dataDistributionQueueSize.isError()) {
auto key = "NotReady" + std::to_string(notReadyCount++);
evt.detail(key.c_str(), "dataDistributionQueueSize");
}
if (teamCollectionValid.isReady() && teamCollectionValid.isError()) {
auto key = "NotReady" + std::to_string(notReadyCount++);
evt.detail(key.c_str(), "teamCollectionValid");
}
if (storageQueueSize.isReady() && storageQueueSize.isError()) {
auto key = "NotReady" + std::to_string(notReadyCount++);
evt.detail(key.c_str(), "storageQueueSize");
}
if (dataDistributionActive.isReady() && dataDistributionActive.isError()) {
auto key = "NotReady" + std::to_string(notReadyCount++);
evt.detail(key.c_str(), "dataDistributionActive");
}
if (storageServersRecruiting.isReady() && storageServersRecruiting.isError()) {
auto key = "NotReady" + std::to_string(notReadyCount++);
evt.detail(key.c_str(), "storageServersRecruiting");
}
Add fdbcli command to read/write version epoch (#6480) * Initialize cluster version at wall-clock time Previously, new clusters would begin at version 0. After this change, clusters will initialize at a version matching wall-clock time. Instead of using the Unix epoch (or Windows epoch), FDB clusters will use a new epoch, defaulting to January 1, 2010, 01:00:00+00:00. In the future, this base epoch will be modifiable through fdbcli, allowing administrators to advance the cluster version. Basing the version off of time allows different FDB clusters to share data without running into version issues. * Send version epoch to master * Cleanup * Update fdbserver/storageserver.actor.cpp Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com> * Jump directly to expected version if possible * Fix initial version issue on storage servers * Add random recovery offset to start version in simulation * Type fixes * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Use correct recoveryTransactionVersion when recovering * Allow version epoch to be adjusted forwards (to decrease the version) * Set version epoch in simulation * Add quiet database check to ensure small version offset * Fix initial version issue on storage servers * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Add fdbcli command to read/write version epoch * Cause recovery when version epoch is set * Handle optional version epoch key * Add ability to clear the version epoch This causes version advancement to revert to the old methodology whereas versions attempt to advance by about a million versions per second, instead of trying to match the clock. * Update transaction access * Modify version epoch to use microseconds instead of seconds * Modify fdbcli version target API Move commands from `versionepoch` to `targetversion` top level command. * Add fdbcli tests for * Temporarily disable targetversion cli tests * Fix version epoch fetch issue * Fix Arena issue * Reduce max version jump in simulation to 1,000,000 * Rework fdbcli API It now requires two commands to fully switch a cluster to using the version epoch. First, enable the version epoch with `versionepoch enable` or `versionepoch set <versionepoch>`. At this point, versions will be given out at a faster or slower rate in an attempt to reach the expected version. Then, run `versionepoch commit` to perform a one time jump to the expected version. This is essentially irreversible. * Temporarily disable old targetversion tests * Cleanup * Move version epoch buggify to sequencer This will cause some issues with the QuietDatabase check for the version offset - namely, it won't do anything, since the version epoch is not being written to the txnStateStore in simulation. This will get fixed in the future. Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com>
2022-04-09 03:33:19 +08:00
if (versionOffset.isReady() && versionOffset.isError()) {
auto key = "NotReady" + std::to_string(notReadyCount++);
evt.detail(key.c_str(), "versionOffset");
}
wait(delay(1.0));
2017-05-26 04:48:44 +08:00
numSuccesses = 0;
}
}
return Void();
}
Future<Void> quietDatabase(Database const& cx,
Reference<AsyncVar<ServerDBInfo> const> const& dbInfo,
2017-05-26 04:48:44 +08:00
std::string phase,
int64_t dataInFlightGate,
int64_t maxTLogQueueGate,
int64_t maxStorageServerQueueGate,
int64_t maxDataDistributionQueueSize,
Add fdbcli command to read/write version epoch (#6480) * Initialize cluster version at wall-clock time Previously, new clusters would begin at version 0. After this change, clusters will initialize at a version matching wall-clock time. Instead of using the Unix epoch (or Windows epoch), FDB clusters will use a new epoch, defaulting to January 1, 2010, 01:00:00+00:00. In the future, this base epoch will be modifiable through fdbcli, allowing administrators to advance the cluster version. Basing the version off of time allows different FDB clusters to share data without running into version issues. * Send version epoch to master * Cleanup * Update fdbserver/storageserver.actor.cpp Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com> * Jump directly to expected version if possible * Fix initial version issue on storage servers * Add random recovery offset to start version in simulation * Type fixes * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Use correct recoveryTransactionVersion when recovering * Allow version epoch to be adjusted forwards (to decrease the version) * Set version epoch in simulation * Add quiet database check to ensure small version offset * Fix initial version issue on storage servers * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Add fdbcli command to read/write version epoch * Cause recovery when version epoch is set * Handle optional version epoch key * Add ability to clear the version epoch This causes version advancement to revert to the old methodology whereas versions attempt to advance by about a million versions per second, instead of trying to match the clock. * Update transaction access * Modify version epoch to use microseconds instead of seconds * Modify fdbcli version target API Move commands from `versionepoch` to `targetversion` top level command. * Add fdbcli tests for * Temporarily disable targetversion cli tests * Fix version epoch fetch issue * Fix Arena issue * Reduce max version jump in simulation to 1,000,000 * Rework fdbcli API It now requires two commands to fully switch a cluster to using the version epoch. First, enable the version epoch with `versionepoch enable` or `versionepoch set <versionepoch>`. At this point, versions will be given out at a faster or slower rate in an attempt to reach the expected version. Then, run `versionepoch commit` to perform a one time jump to the expected version. This is essentially irreversible. * Temporarily disable old targetversion tests * Cleanup * Move version epoch buggify to sequencer This will cause some issues with the QuietDatabase check for the version offset - namely, it won't do anything, since the version epoch is not being written to the txnStateStore in simulation. This will get fixed in the future. Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com>
2022-04-09 03:33:19 +08:00
int64_t maxPoppedVersionLag,
int64_t maxVersionOffset) {
return waitForQuietDatabase(cx,
dbInfo,
phase,
dataInFlightGate,
maxTLogQueueGate,
maxStorageServerQueueGate,
maxDataDistributionQueueSize,
Add fdbcli command to read/write version epoch (#6480) * Initialize cluster version at wall-clock time Previously, new clusters would begin at version 0. After this change, clusters will initialize at a version matching wall-clock time. Instead of using the Unix epoch (or Windows epoch), FDB clusters will use a new epoch, defaulting to January 1, 2010, 01:00:00+00:00. In the future, this base epoch will be modifiable through fdbcli, allowing administrators to advance the cluster version. Basing the version off of time allows different FDB clusters to share data without running into version issues. * Send version epoch to master * Cleanup * Update fdbserver/storageserver.actor.cpp Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com> * Jump directly to expected version if possible * Fix initial version issue on storage servers * Add random recovery offset to start version in simulation * Type fixes * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Use correct recoveryTransactionVersion when recovering * Allow version epoch to be adjusted forwards (to decrease the version) * Set version epoch in simulation * Add quiet database check to ensure small version offset * Fix initial version issue on storage servers * Disable reference time by default Enable on a cluster using the fdbcli command `versionepoch add 0`. * Add fdbcli command to read/write version epoch * Cause recovery when version epoch is set * Handle optional version epoch key * Add ability to clear the version epoch This causes version advancement to revert to the old methodology whereas versions attempt to advance by about a million versions per second, instead of trying to match the clock. * Update transaction access * Modify version epoch to use microseconds instead of seconds * Modify fdbcli version target API Move commands from `versionepoch` to `targetversion` top level command. * Add fdbcli tests for * Temporarily disable targetversion cli tests * Fix version epoch fetch issue * Fix Arena issue * Reduce max version jump in simulation to 1,000,000 * Rework fdbcli API It now requires two commands to fully switch a cluster to using the version epoch. First, enable the version epoch with `versionepoch enable` or `versionepoch set <versionepoch>`. At this point, versions will be given out at a faster or slower rate in an attempt to reach the expected version. Then, run `versionepoch commit` to perform a one time jump to the expected version. This is essentially irreversible. * Temporarily disable old targetversion tests * Cleanup * Move version epoch buggify to sequencer This will cause some issues with the QuietDatabase check for the version offset - namely, it won't do anything, since the version epoch is not being written to the txnStateStore in simulation. This will get fixed in the future. Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com>
2022-04-09 03:33:19 +08:00
maxPoppedVersionLag,
maxVersionOffset);
2017-05-26 04:48:44 +08:00
}