foundationdb/fdbserver/DDTeamCollection.h
2022-02-04 10:59:01 -08:00

2110 lines
83 KiB
C++

/*
* DDTeamCollection.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2018 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
#include <set>
#include <sstream>
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/Knobs.h"
#include "fdbclient/StorageServerInterface.h"
#include "fdbclient/SystemData.h"
#include "fdbclient/DatabaseContext.h"
#include "fdbclient/ManagementAPI.actor.h"
#include "fdbclient/RunTransaction.actor.h"
#include "fdbrpc/Replication.h"
#include "fdbserver/DataDistribution.actor.h"
#include "fdbserver/FDBExecHelper.actor.h"
#include "fdbserver/IKeyValueStore.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/MoveKeys.actor.h"
#include "fdbserver/QuietDatabase.h"
#include "fdbserver/ServerDBInfo.h"
#include "fdbserver/TCInfo.h"
#include "fdbserver/TLogInterface.h"
#include "fdbserver/WaitFailure.h"
#include "flow/ActorCollection.h"
#include "flow/Arena.h"
#include "flow/BooleanParam.h"
#include "flow/Trace.h"
#include "flow/UnitTest.h"
#include "flow/actorcompiler.h" // This must be the last #include.
class TCTeamInfo;
class TCMachineInfo;
class TCMachineTeamInfo;
FDB_DECLARE_BOOLEAN_PARAM(IsPrimary);
// All state that represents an ongoing tss pair recruitment
struct TSSPairState : ReferenceCounted<TSSPairState>, NonCopyable {
Promise<Optional<std::pair<UID, Version>>>
ssPairInfo; // if set, for ss to pass its id to tss pair once it is successfully recruited
Promise<bool> tssPairDone; // if set, for tss to pass ss that it was successfully recruited
Promise<Void> complete;
Optional<Key> dcId; // dc
Optional<Key> dataHallId; // data hall
bool active;
TSSPairState() : active(false) {}
TSSPairState(const LocalityData& locality)
: dcId(locality.dcId()), dataHallId(locality.dataHallId()), active(true) {}
bool inDataZone(const LocalityData& locality) {
return locality.dcId() == dcId && locality.dataHallId() == dataHallId;
}
void cancel() {
// only cancel if both haven't been set, otherwise one half of pair could think it was successful but the other
// half would think it failed
if (active && ssPairInfo.canBeSet() && tssPairDone.canBeSet()) {
ssPairInfo.send(Optional<std::pair<UID, Version>>());
// callback of ssPairInfo could have cancelled tssPairDone already, so double check before cancelling
if (tssPairDone.canBeSet()) {
tssPairDone.send(false);
}
if (complete.canBeSet()) {
complete.send(Void());
}
}
}
bool tssRecruitSuccess() {
if (active && tssPairDone.canBeSet()) {
tssPairDone.send(true);
return true;
}
return false;
}
bool tssRecruitFailed() {
if (active && tssPairDone.canBeSet()) {
tssPairDone.send(false);
return true;
}
return false;
}
bool ssRecruitSuccess(std::pair<UID, Version> ssInfo) {
if (active && ssPairInfo.canBeSet()) {
ssPairInfo.send(Optional<std::pair<UID, Version>>(ssInfo));
return true;
}
return false;
}
bool ssRecruitFailed() {
if (active && ssPairInfo.canBeSet()) {
ssPairInfo.send(Optional<std::pair<UID, Version>>());
return true;
}
return false;
}
bool markComplete() {
if (active && complete.canBeSet()) {
complete.send(Void());
return true;
}
return false;
}
Future<Optional<std::pair<UID, Version>>> waitOnSS() { return ssPairInfo.getFuture(); }
Future<bool> waitOnTSS() { return tssPairDone.getFuture(); }
Future<Void> waitComplete() { return complete.getFuture(); }
};
class ServerStatus {
public:
bool isWiggling;
bool isFailed;
bool isUndesired;
bool isWrongConfiguration;
bool initialized; // AsyncMap erases default constructed objects
LocalityData locality;
ServerStatus()
: isWiggling(false), isFailed(true), isUndesired(false), isWrongConfiguration(false), initialized(false) {}
ServerStatus(bool isFailed, bool isUndesired, bool isWiggling, LocalityData const& locality)
: isWiggling(isWiggling), isFailed(isFailed), isUndesired(isUndesired), isWrongConfiguration(false),
initialized(true), locality(locality) {}
bool isUnhealthy() const { return isFailed || isUndesired; }
const char* toString() const {
return isFailed ? "Failed" : isUndesired ? "Undesired" : isWiggling ? "Wiggling" : "Healthy";
}
bool operator==(ServerStatus const& r) const {
return isFailed == r.isFailed && isUndesired == r.isUndesired && isWiggling == r.isWiggling &&
isWrongConfiguration == r.isWrongConfiguration && locality == r.locality && initialized == r.initialized;
}
bool operator!=(ServerStatus const& r) const { return !(*this == r); }
// If a process has reappeared without the storage server that was on it (isFailed == true), we don't need to
// exclude it We also don't need to exclude processes who are in the wrong configuration (since those servers will
// be removed)
bool excludeOnRecruit() const { return !isFailed && !isWrongConfiguration; }
};
typedef AsyncMap<UID, ServerStatus> ServerStatusMap;
class DDTeamCollection : public ReferenceCounted<DDTeamCollection> {
friend class DDTeamCollectionImpl;
public:
// clang-format off
enum class Status { NONE = 0, WIGGLING = 1, EXCLUDED = 2, FAILED = 3};
// addActor: add to actorCollection so that when an actor has error, the ActorCollection can catch the error.
// addActor is used to create the actorCollection when the dataDistributionTeamCollection is created
PromiseStream<Future<Void>> addActor;
Database cx;
UID distributorId;
DatabaseConfiguration configuration;
bool doBuildTeams;
bool lastBuildTeamsFailed;
Future<Void> teamBuilder;
AsyncTrigger restartTeamBuilder;
AsyncVar<bool> waitUntilRecruited; // make teambuilder wait until one new SS is recruited
MoveKeysLock lock;
PromiseStream<RelocateShard> output;
std::vector<UID> allServers;
ServerStatusMap server_status;
int64_t unhealthyServers;
std::map<int,int> priority_teams;
std::map<UID, Reference<TCServerInfo>> server_info;
std::map<Key, std::vector<Reference<TCServerInfo>>> pid2server_info; // some process may serve as multiple storage servers
std::vector<AddressExclusion> wiggle_addresses; // collection of wiggling servers' address
std::map<UID, Reference<TCServerInfo>> tss_info_by_pair;
std::map<UID, Reference<TCServerInfo>> server_and_tss_info; // TODO could replace this with an efficient way to do a read-only concatenation of 2 data structures?
std::map<Key, int> lagging_zones; // zone to number of storage servers lagging
AsyncVar<bool> disableFailingLaggingServers;
Optional<Key> wigglingPid; // Process id of current wiggling storage server;
Reference<AsyncVar<bool>> pauseWiggle;
Reference<AsyncVar<bool>> processingWiggle; // track whether wiggling relocation is being processed
// machine_info has all machines info; key must be unique across processes on the same machine
std::map<Standalone<StringRef>, Reference<TCMachineInfo>> machine_info;
std::vector<Reference<TCMachineTeamInfo>> machineTeams; // all machine teams
LocalityMap<UID> machineLocalityMap; // locality info of machines
std::vector<Reference<TCTeamInfo>> teams;
std::vector<Reference<TCTeamInfo>> badTeams;
Reference<ShardsAffectedByTeamFailure> shardsAffectedByTeamFailure;
PromiseStream<UID> removedServers;
PromiseStream<UID> removedTSS;
std::set<UID> recruitingIds; // The IDs of the SS/TSS which are being recruited
std::set<NetworkAddress> recruitingLocalities;
Future<Void> initialFailureReactionDelay;
Future<Void> initializationDoneActor;
Promise<Void> serverTrackerErrorOut;
AsyncVar<int> recruitingStream;
Debouncer restartRecruiting;
int healthyTeamCount;
Reference<AsyncVar<bool>> zeroHealthyTeams;
int optimalTeamCount;
AsyncVar<bool> zeroOptimalTeams;
int bestTeamKeepStuckCount = 0;
bool isTssRecruiting; // If tss recruiting is waiting on a pair, don't consider DD recruiting for the purposes of QuietDB
// WIGGLING if an address is under storage wiggling.
// EXCLUDED if an address is in the excluded list in the database.
// FAILED if an address is permanently failed.
// NONE by default. Updated asynchronously (eventually)
AsyncMap< AddressExclusion, Status > excludedServers;
std::set<AddressExclusion> invalidLocalityAddr; // These address have invalidLocality for the configured storagePolicy
std::vector<Optional<Key>> includedDCs;
Optional<std::vector<Optional<Key>>> otherTrackedDCs;
bool primary;
Reference<AsyncVar<bool>> processingUnhealthy;
Future<Void> readyToStart;
Future<Void> checkTeamDelay;
Promise<Void> addSubsetComplete;
Future<Void> badTeamRemover;
Future<Void> checkInvalidLocalities;
Future<Void> wrongStoreTypeRemover;
Reference<LocalitySet> storageServerSet;
std::vector<DDTeamCollection*> teamCollections;
AsyncVar<Optional<Key>> healthyZone;
Future<bool> clearHealthyZoneFuture;
double medianAvailableSpace;
double lastMedianAvailableSpaceUpdate;
// clang-format on
int lowestUtilizationTeam;
int highestUtilizationTeam;
AsyncTrigger printDetailedTeamsInfo;
PromiseStream<GetMetricsRequest> getShardMetrics;
PromiseStream<Promise<int>> getUnhealthyRelocationCount;
Promise<UID> removeFailedServer;
Reference<EventCacheHolder> ddTrackerStartingEventHolder;
Reference<EventCacheHolder> teamCollectionInfoEventHolder;
Reference<EventCacheHolder> storageServerRecruitmentEventHolder;
void resetLocalitySet() {
storageServerSet = Reference<LocalitySet>(new LocalityMap<UID>());
LocalityMap<UID>* storageServerMap = (LocalityMap<UID>*)storageServerSet.getPtr();
for (auto& it : server_info) {
it.second->localityEntry = storageServerMap->add(it.second->lastKnownInterface.locality, &it.second->id);
}
}
bool satisfiesPolicy(const std::vector<Reference<TCServerInfo>>& team, int amount = -1) const {
std::vector<LocalityEntry> forcedEntries, resultEntries;
if (amount == -1) {
amount = team.size();
}
forcedEntries.reserve(amount);
for (int i = 0; i < amount; i++) {
forcedEntries.push_back(team[i]->localityEntry);
}
bool result = storageServerSet->selectReplicas(configuration.storagePolicy, forcedEntries, resultEntries);
return result && resultEntries.size() == 0;
}
DDTeamCollection(Database const& cx,
UID distributorId,
MoveKeysLock const& lock,
PromiseStream<RelocateShard> const& output,
Reference<ShardsAffectedByTeamFailure> const& shardsAffectedByTeamFailure,
DatabaseConfiguration configuration,
std::vector<Optional<Key>> includedDCs,
Optional<std::vector<Optional<Key>>> otherTrackedDCs,
Future<Void> readyToStart,
Reference<AsyncVar<bool>> zeroHealthyTeams,
IsPrimary primary,
Reference<AsyncVar<bool>> processingUnhealthy,
Reference<AsyncVar<bool>> processingWiggle,
PromiseStream<GetMetricsRequest> getShardMetrics,
Promise<UID> removeFailedServer,
PromiseStream<Promise<int>> getUnhealthyRelocationCount)
: cx(cx), distributorId(distributorId), configuration(configuration), doBuildTeams(true),
lastBuildTeamsFailed(false), teamBuilder(Void()), lock(lock), output(output), unhealthyServers(0),
processingWiggle(processingWiggle), shardsAffectedByTeamFailure(shardsAffectedByTeamFailure),
initialFailureReactionDelay(
delayed(readyToStart, SERVER_KNOBS->INITIAL_FAILURE_REACTION_DELAY, TaskPriority::DataDistribution)),
initializationDoneActor(logOnCompletion(readyToStart && initialFailureReactionDelay)), recruitingStream(0),
restartRecruiting(SERVER_KNOBS->DEBOUNCE_RECRUITING_DELAY), healthyTeamCount(0),
zeroHealthyTeams(zeroHealthyTeams), optimalTeamCount(0), zeroOptimalTeams(true), isTssRecruiting(false),
includedDCs(includedDCs), otherTrackedDCs(otherTrackedDCs), primary(primary),
processingUnhealthy(processingUnhealthy), readyToStart(readyToStart),
checkTeamDelay(delay(SERVER_KNOBS->CHECK_TEAM_DELAY, TaskPriority::DataDistribution)), badTeamRemover(Void()),
checkInvalidLocalities(Void()), wrongStoreTypeRemover(Void()), storageServerSet(new LocalityMap<UID>()),
clearHealthyZoneFuture(true), medianAvailableSpace(SERVER_KNOBS->MIN_AVAILABLE_SPACE_RATIO),
lastMedianAvailableSpaceUpdate(0), lowestUtilizationTeam(0), highestUtilizationTeam(0),
getShardMetrics(getShardMetrics), getUnhealthyRelocationCount(getUnhealthyRelocationCount),
removeFailedServer(removeFailedServer),
ddTrackerStartingEventHolder(makeReference<EventCacheHolder>("DDTrackerStarting")),
teamCollectionInfoEventHolder(makeReference<EventCacheHolder>("TeamCollectionInfo")),
storageServerRecruitmentEventHolder(
makeReference<EventCacheHolder>("StorageServerRecruitment_" + distributorId.toString())) {
if (!primary || configuration.usableRegions == 1) {
TraceEvent("DDTrackerStarting", distributorId)
.detail("State", "Inactive")
.trackLatest(ddTrackerStartingEventHolder->trackingKey);
}
}
~DDTeamCollection() {
TraceEvent("DDTeamCollectionDestructed", distributorId).detail("Primary", primary);
// Cancel the teamBuilder to avoid creating new teams after teams are cancelled.
teamBuilder.cancel();
// TraceEvent("DDTeamCollectionDestructed", distributorId)
// .detail("Primary", primary)
// .detail("TeamBuilderDestroyed", server_info.size());
// Other teamCollections also hold pointer to this teamCollection;
// TeamTracker may access the destructed DDTeamCollection if we do not reset the pointer
for (int i = 0; i < teamCollections.size(); i++) {
if (teamCollections[i] != nullptr && teamCollections[i] != this) {
for (int j = 0; j < teamCollections[i]->teamCollections.size(); ++j) {
if (teamCollections[i]->teamCollections[j] == this) {
teamCollections[i]->teamCollections[j] = nullptr;
}
}
}
}
// Team tracker has pointers to DDTeamCollections both in primary and remote.
// The following kills a reference cycle between the teamTracker actor and the TCTeamInfo that both holds and is
// held by the actor It also ensures that the trackers are done fiddling with healthyTeamCount before we free
// this
for (auto& team : teams) {
team->tracker.cancel();
}
// The commented TraceEvent log is useful in detecting what is running during the destruction
// TraceEvent("DDTeamCollectionDestructed", distributorId)
// .detail("Primary", primary)
// .detail("TeamTrackerDestroyed", teams.size());
for (auto& badTeam : badTeams) {
badTeam->tracker.cancel();
}
// TraceEvent("DDTeamCollectionDestructed", distributorId)
// .detail("Primary", primary)
// .detail("BadTeamTrackerDestroyed", badTeams.size());
// The following makes sure that, even if a reference to a team is held in the DD Queue, the tracker will be
// stopped
// before the server_status map to which it has a pointer, is destroyed.
for (auto& [_, info] : server_and_tss_info) {
info->tracker.cancel();
info->collection = nullptr;
}
// TraceEvent("DDTeamCollectionDestructed", distributorId)
// .detail("Primary", primary)
// .detail("ServerTrackerDestroyed", server_info.size());
}
void addLaggingStorageServer(Key zoneId) {
lagging_zones[zoneId]++;
if (lagging_zones.size() > std::max(1, configuration.storageTeamSize - 1) &&
!disableFailingLaggingServers.get())
disableFailingLaggingServers.set(true);
}
void removeLaggingStorageServer(Key zoneId) {
auto iter = lagging_zones.find(zoneId);
ASSERT(iter != lagging_zones.end());
iter->second--;
ASSERT(iter->second >= 0);
if (iter->second == 0)
lagging_zones.erase(iter);
if (lagging_zones.size() <= std::max(1, configuration.storageTeamSize - 1) &&
disableFailingLaggingServers.get())
disableFailingLaggingServers.set(false);
}
Future<Void> logOnCompletion(Future<Void> signal);
Future<Void> interruptableBuildTeams();
Future<Void> checkBuildTeams();
// Returns a random healthy team, which does not contain excludeServer.
std::vector<UID> getRandomHealthyTeam(const UID& excludeServer) {
std::vector<int> candidates, backup;
for (int i = 0; i < teams.size(); ++i) {
if (teams[i]->isHealthy() && !teams[i]->hasServer(excludeServer)) {
candidates.push_back(i);
} else if (teams[i]->size() - (teams[i]->hasServer(excludeServer) ? 1 : 0) > 0) {
// If a team has at least one other server besides excludeServer, select it
// as a backup candidate.
backup.push_back(i);
}
}
// Prefer a healthy team not containing excludeServer.
if (candidates.size() > 0) {
return teams[candidates[deterministicRandom()->randomInt(0, candidates.size())]]->getServerIDs();
} else if (backup.size() > 0) {
// The backup choice is a team with at least one server besides excludeServer, in this
// case, the team will be possibily relocated to a healthy destination later by DD.
std::vector<UID> servers =
teams[backup[deterministicRandom()->randomInt(0, backup.size())]]->getServerIDs();
std::vector<UID> res;
for (const UID& id : servers) {
if (id != excludeServer) {
res.push_back(id);
}
}
TraceEvent("FoundNonoptimalTeamForDroppedShard", excludeServer).detail("Team", describe(res));
return res;
}
return std::vector<UID>();
}
Future<Void> getTeam(GetTeamRequest);
int64_t getDebugTotalDataInFlight() const {
int64_t total = 0;
for (auto itr = server_info.begin(); itr != server_info.end(); ++itr)
total += itr->second->dataInFlightToServer;
return total;
}
Future<Void> addSubsetOfEmergencyTeams();
Future<Void> init(Reference<InitialDataDistribution> initTeams, DDEnabledState const* ddEnabledState);
// Check if server or machine has a valid locality based on configured replication policy
bool isValidLocality(Reference<IReplicationPolicy> storagePolicy, const LocalityData& locality) const {
// Future: Once we add simulation test that misconfigure a cluster, such as not setting some locality entries,
// DD_VALIDATE_LOCALITY should always be true. Otherwise, simulation test may fail.
if (!SERVER_KNOBS->DD_VALIDATE_LOCALITY) {
// Disable the checking if locality is valid
return true;
}
std::set<std::string> replicationPolicyKeys = storagePolicy->attributeKeys();
for (auto& policy : replicationPolicyKeys) {
if (!locality.isPresent(policy)) {
return false;
}
}
return true;
}
void evaluateTeamQuality() const {
int teamCount = teams.size(), serverCount = allServers.size();
double teamsPerServer = (double)teamCount * configuration.storageTeamSize / serverCount;
ASSERT(serverCount == server_info.size());
int minTeams = std::numeric_limits<int>::max();
int maxTeams = std::numeric_limits<int>::min();
double varTeams = 0;
std::map<Optional<Standalone<StringRef>>, int> machineTeams;
for (const auto& [id, info] : server_info) {
if (!server_status.get(id).isUnhealthy()) {
int stc = info->teams.size();
minTeams = std::min(minTeams, stc);
maxTeams = std::max(maxTeams, stc);
varTeams += (stc - teamsPerServer) * (stc - teamsPerServer);
// Use zoneId as server's machine id
machineTeams[info->lastKnownInterface.locality.zoneId()] += stc;
}
}
varTeams /= teamsPerServer * teamsPerServer;
int minMachineTeams = std::numeric_limits<int>::max();
int maxMachineTeams = std::numeric_limits<int>::min();
for (auto m = machineTeams.begin(); m != machineTeams.end(); ++m) {
minMachineTeams = std::min(minMachineTeams, m->second);
maxMachineTeams = std::max(maxMachineTeams, m->second);
}
TraceEvent(minTeams > 0 ? SevInfo : SevWarn, "DataDistributionTeamQuality", distributorId)
.detail("Servers", serverCount)
.detail("Teams", teamCount)
.detail("TeamsPerServer", teamsPerServer)
.detail("Variance", varTeams / serverCount)
.detail("ServerMinTeams", minTeams)
.detail("ServerMaxTeams", maxTeams)
.detail("MachineMinTeams", minMachineTeams)
.detail("MachineMaxTeams", maxMachineTeams);
}
int overlappingMembers(const std::vector<UID>& team) const {
if (team.empty()) {
return 0;
}
int maxMatchingServers = 0;
const UID& serverID = team[0];
const auto it = server_info.find(serverID);
ASSERT(it != server_info.end());
const auto& usedTeams = it->second->teams;
for (const auto& usedTeam : usedTeams) {
auto used = usedTeam->getServerIDs();
int teamIdx = 0;
int usedIdx = 0;
int matchingServers = 0;
while (teamIdx < team.size() && usedIdx < used.size()) {
if (team[teamIdx] == used[usedIdx]) {
matchingServers++;
teamIdx++;
usedIdx++;
} else if (team[teamIdx] < used[usedIdx]) {
teamIdx++;
} else {
usedIdx++;
}
}
ASSERT(matchingServers > 0);
maxMatchingServers = std::max(maxMatchingServers, matchingServers);
if (maxMatchingServers == team.size()) {
return maxMatchingServers;
}
}
return maxMatchingServers;
}
int overlappingMachineMembers(std::vector<Standalone<StringRef>> const& team) const {
if (team.empty()) {
return 0;
}
int maxMatchingServers = 0;
auto it = machine_info.find(team[0]);
ASSERT(it != machine_info.end());
auto const& machineTeams = it->second->machineTeams;
for (auto const& usedTeam : machineTeams) {
auto used = usedTeam->machineIDs;
int teamIdx = 0;
int usedIdx = 0;
int matchingServers = 0;
while (teamIdx < team.size() && usedIdx < used.size()) {
if (team[teamIdx] == used[usedIdx]) {
matchingServers++;
teamIdx++;
usedIdx++;
} else if (team[teamIdx] < used[usedIdx]) {
teamIdx++;
} else {
usedIdx++;
}
}
ASSERT(matchingServers > 0);
maxMatchingServers = std::max(maxMatchingServers, matchingServers);
if (maxMatchingServers == team.size()) {
return maxMatchingServers;
}
}
return maxMatchingServers;
}
Reference<TCMachineTeamInfo> findMachineTeam(std::vector<Standalone<StringRef>> const& machineIDs) const;
// Assume begin to end is sorted by std::sort
// Assume InputIt is iterator to UID
// Note: We must allow creating empty teams because empty team is created when a remote DB is initialized.
// The empty team is used as the starting point to move data to the remote DB
// begin : the start of the team member ID
// end : end of the team member ID
// isIntialTeam : False when the team is added by addTeamsBestOf(); True otherwise, e.g.,
// when the team added at init() when we recreate teams by looking up DB
template <class InputIt>
void addTeam(InputIt begin, InputIt end, bool isInitialTeam) {
std::vector<Reference<TCServerInfo>> newTeamServers;
for (auto i = begin; i != end; ++i) {
if (server_info.find(*i) != server_info.end()) {
newTeamServers.push_back(server_info[*i]);
}
}
addTeam(newTeamServers, isInitialTeam);
}
void addTeam(const std::vector<Reference<TCServerInfo>>& newTeamServers,
bool isInitialTeam,
bool redundantTeam = false) {
auto teamInfo = makeReference<TCTeamInfo>(newTeamServers);
// Move satisfiesPolicy to the end for performance benefit
bool badTeam = redundantTeam || teamInfo->size() != configuration.storageTeamSize ||
!satisfiesPolicy(teamInfo->getServers());
teamInfo->tracker = teamTracker(teamInfo, badTeam, redundantTeam);
// ASSERT( teamInfo->serverIDs.size() > 0 ); //team can be empty at DB initialization
if (badTeam) {
badTeams.push_back(teamInfo);
return;
}
// For a good team, we add it to teams and create machine team for it when necessary
teams.push_back(teamInfo);
for (int i = 0; i < newTeamServers.size(); ++i) {
newTeamServers[i]->teams.push_back(teamInfo);
}
// Find or create machine team for the server team
// Add the reference of machineTeam (with machineIDs) into process team
std::vector<Standalone<StringRef>> machineIDs;
for (auto server = newTeamServers.begin(); server != newTeamServers.end(); ++server) {
ASSERT_WE_THINK((*server)->machine.isValid());
machineIDs.push_back((*server)->machine->machineID);
}
sort(machineIDs.begin(), machineIDs.end());
Reference<TCMachineTeamInfo> machineTeamInfo = findMachineTeam(machineIDs);
// A team is not initial team if it is added by addTeamsBestOf() which always create a team with correct size
// A non-initial team must have its machine team created and its size must be correct
ASSERT(isInitialTeam || machineTeamInfo.isValid());
// Create a machine team if it does not exist
// Note an initial team may be added at init() even though the team size is not storageTeamSize
if (!machineTeamInfo.isValid() && !machineIDs.empty()) {
machineTeamInfo = addMachineTeam(machineIDs.begin(), machineIDs.end());
}
if (!machineTeamInfo.isValid()) {
TraceEvent(SevWarn, "AddTeamWarning")
.detail("NotFoundMachineTeam", "OKIfTeamIsEmpty")
.detail("TeamInfo", teamInfo->getDesc());
}
teamInfo->machineTeam = machineTeamInfo;
machineTeamInfo->serverTeams.push_back(teamInfo);
if (g_network->isSimulated()) {
// Update server team information for consistency check in simulation
traceTeamCollectionInfo();
}
}
void addTeam(std::set<UID> const& team, bool isInitialTeam) { addTeam(team.begin(), team.end(), isInitialTeam); }
// Add a machine team specified by input machines
Reference<TCMachineTeamInfo> addMachineTeam(std::vector<Reference<TCMachineInfo>> machines) {
auto machineTeamInfo = makeReference<TCMachineTeamInfo>(machines);
machineTeams.push_back(machineTeamInfo);
// Assign machine teams to machine
for (auto machine : machines) {
// A machine's machineTeams vector should not hold duplicate machineTeam members
ASSERT_WE_THINK(std::count(machine->machineTeams.begin(), machine->machineTeams.end(), machineTeamInfo) ==
0);
machine->machineTeams.push_back(machineTeamInfo);
}
return machineTeamInfo;
}
// Add a machine team by using the machineIDs from begin to end
Reference<TCMachineTeamInfo> addMachineTeam(std::vector<Standalone<StringRef>>::iterator begin,
std::vector<Standalone<StringRef>>::iterator end) {
std::vector<Reference<TCMachineInfo>> machines;
for (auto i = begin; i != end; ++i) {
if (machine_info.find(*i) != machine_info.end()) {
machines.push_back(machine_info[*i]);
} else {
TraceEvent(SevWarn, "AddMachineTeamError").detail("MachineIDNotExist", i->contents().toString());
}
}
return addMachineTeam(machines);
}
// Group storage servers (process) based on their machineId in LocalityData
// All created machines are healthy
// Return The number of healthy servers we grouped into machines
int constructMachinesFromServers() {
int totalServerIndex = 0;
for (auto i = server_info.begin(); i != server_info.end(); ++i) {
if (!server_status.get(i->first).isUnhealthy()) {
checkAndCreateMachine(i->second);
totalServerIndex++;
}
}
return totalServerIndex;
}
void traceConfigInfo() const {
TraceEvent("DDConfig", distributorId)
.detail("StorageTeamSize", configuration.storageTeamSize)
.detail("DesiredTeamsPerServer", SERVER_KNOBS->DESIRED_TEAMS_PER_SERVER)
.detail("MaxTeamsPerServer", SERVER_KNOBS->MAX_TEAMS_PER_SERVER)
.detail("StoreType", configuration.storageServerStoreType);
}
void traceServerInfo() const;
void traceServerTeamInfo() const {
int i = 0;
TraceEvent("ServerTeamInfo", distributorId).detail("Size", teams.size());
for (auto& team : teams) {
TraceEvent("ServerTeamInfo", distributorId)
.detail("TeamIndex", i++)
.detail("Healthy", team->isHealthy())
.detail("TeamSize", team->size())
.detail("MemberIDs", team->getServerIDsStr())
.detail("TeamID", team->getTeamID());
}
}
void traceMachineInfo() const {
int i = 0;
TraceEvent("MachineInfo").detail("Size", machine_info.size());
for (auto& machine : machine_info) {
TraceEvent("MachineInfo", distributorId)
.detail("MachineInfoIndex", i++)
.detail("Healthy", isMachineHealthy(machine.second))
.detail("MachineID", machine.first.contents().toString())
.detail("MachineTeamOwned", machine.second->machineTeams.size())
.detail("ServerNumOnMachine", machine.second->serversOnMachine.size())
.detail("ServersID", machine.second->getServersIDStr());
}
}
void traceMachineTeamInfo() const {
int i = 0;
TraceEvent("MachineTeamInfo", distributorId).detail("Size", machineTeams.size());
for (auto& team : machineTeams) {
TraceEvent("MachineTeamInfo", distributorId)
.detail("TeamIndex", i++)
.detail("MachineIDs", team->getMachineIDsStr())
.detail("ServerTeams", team->serverTeams.size());
}
}
// Locality string is hashed into integer, used as KeyIndex
// For better understand which KeyIndex is used for locality, we print this info in trace.
void traceLocalityArrayIndexName() const {
TraceEvent("LocalityRecordKeyName").detail("Size", machineLocalityMap._keymap->_lookuparray.size());
for (int i = 0; i < machineLocalityMap._keymap->_lookuparray.size(); ++i) {
TraceEvent("LocalityRecordKeyIndexName")
.detail("KeyIndex", i)
.detail("KeyName", machineLocalityMap._keymap->_lookuparray[i]);
}
}
void traceMachineLocalityMap() const {
int i = 0;
TraceEvent("MachineLocalityMap", distributorId).detail("Size", machineLocalityMap.size());
for (auto& uid : machineLocalityMap.getObjects()) {
Reference<LocalityRecord> record = machineLocalityMap.getRecord(i);
if (record.isValid()) {
TraceEvent("MachineLocalityMap", distributorId)
.detail("LocalityIndex", i++)
.detail("UID", uid->toString())
.detail("LocalityRecord", record->toString());
} else {
TraceEvent("MachineLocalityMap")
.detail("LocalityIndex", i++)
.detail("UID", uid->toString())
.detail("LocalityRecord", "[NotFound]");
}
}
}
// To enable verbose debug info, set shouldPrint to true
void traceAllInfo(bool shouldPrint = false) const {
if (!shouldPrint)
return;
// Record all team collections IDs
for (int i = 0; i < teamCollections.size(); ++i) {
if (teamCollections[i] != nullptr) {
TraceEvent("TraceAllInfo", distributorId)
.detail("TeamCollectionIndex", i)
.detail("Primary", teamCollections[i]->primary);
}
}
TraceEvent("TraceAllInfo", distributorId).detail("Primary", primary);
traceConfigInfo();
traceServerInfo();
traceServerTeamInfo();
traceMachineInfo();
traceMachineTeamInfo();
traceLocalityArrayIndexName();
traceMachineLocalityMap();
}
// We must rebuild machine locality map whenever the entry in the map is inserted or removed
void rebuildMachineLocalityMap() {
machineLocalityMap.clear();
int numHealthyMachine = 0;
for (auto machine = machine_info.begin(); machine != machine_info.end(); ++machine) {
if (machine->second->serversOnMachine.empty()) {
TraceEvent(SevWarn, "RebuildMachineLocalityMapError")
.detail("Machine", machine->second->machineID.toString())
.detail("NumServersOnMachine", 0);
continue;
}
if (!isMachineHealthy(machine->second)) {
continue;
}
Reference<TCServerInfo> representativeServer = machine->second->serversOnMachine[0];
auto& locality = representativeServer->lastKnownInterface.locality;
if (!isValidLocality(configuration.storagePolicy, locality)) {
TraceEvent(SevWarn, "RebuildMachineLocalityMapError")
.detail("Machine", machine->second->machineID.toString())
.detail("InvalidLocality", locality.toString());
continue;
}
const LocalityEntry& localityEntry = machineLocalityMap.add(locality, &representativeServer->id);
machine->second->localityEntry = localityEntry;
++numHealthyMachine;
}
}
// Create machineTeamsToBuild number of machine teams
// No operation if machineTeamsToBuild is 0
// Note: The creation of machine teams should not depend on server teams:
// No matter how server teams will be created, we will create the same set of machine teams;
// We should never use server team number in building machine teams.
//
// Five steps to create each machine team, which are document in the function
// Reuse ReplicationPolicy selectReplicas func to select machine team
// return number of added machine teams
int addBestMachineTeams(int machineTeamsToBuild) {
int addedMachineTeams = 0;
ASSERT(machineTeamsToBuild >= 0);
// The number of machines is always no smaller than the storageTeamSize in a correct configuration
ASSERT(machine_info.size() >= configuration.storageTeamSize);
// Future: Consider if we should overbuild more machine teams to
// allow machineTeamRemover() to get a more balanced machine teams per machine
// Step 1: Create machineLocalityMap which will be used in building machine team
rebuildMachineLocalityMap();
// Add a team in each iteration
while (addedMachineTeams < machineTeamsToBuild || notEnoughMachineTeamsForAMachine()) {
// Step 2: Get least used machines from which we choose machines as a machine team
std::vector<Reference<TCMachineInfo>> leastUsedMachines; // A less used machine has less number of teams
int minTeamCount = std::numeric_limits<int>::max();
for (auto& machine : machine_info) {
// Skip invalid machine whose representative server is not in server_info
ASSERT_WE_THINK(server_info.find(machine.second->serversOnMachine[0]->id) != server_info.end());
// Skip unhealthy machines
if (!isMachineHealthy(machine.second))
continue;
// Skip machine with incomplete locality
if (!isValidLocality(configuration.storagePolicy,
machine.second->serversOnMachine[0]->lastKnownInterface.locality)) {
continue;
}
// Invariant: We only create correct size machine teams.
// When configuration (e.g., team size) is changed, the DDTeamCollection will be destroyed and rebuilt
// so that the invariant will not be violated.
int teamCount = machine.second->machineTeams.size();
if (teamCount < minTeamCount) {
leastUsedMachines.clear();
minTeamCount = teamCount;
}
if (teamCount == minTeamCount) {
leastUsedMachines.push_back(machine.second);
}
}
std::vector<UID*> team;
std::vector<LocalityEntry> forcedAttributes;
// Step 4: Reuse Policy's selectReplicas() to create team for the representative process.
std::vector<UID*> bestTeam;
int bestScore = std::numeric_limits<int>::max();
int maxAttempts = SERVER_KNOBS->BEST_OF_AMT; // BEST_OF_AMT = 4
for (int i = 0; i < maxAttempts && i < 100; ++i) {
// Step 3: Create a representative process for each machine.
// Construct forcedAttribute from leastUsedMachines.
// We will use forcedAttribute to call existing function to form a team
if (leastUsedMachines.size()) {
forcedAttributes.clear();
// Randomly choose 1 least used machine
Reference<TCMachineInfo> tcMachineInfo = deterministicRandom()->randomChoice(leastUsedMachines);
ASSERT(!tcMachineInfo->serversOnMachine.empty());
LocalityEntry process = tcMachineInfo->localityEntry;
forcedAttributes.push_back(process);
TraceEvent("ChosenMachine")
.detail("MachineInfo", tcMachineInfo->machineID)
.detail("LeaseUsedMachinesSize", leastUsedMachines.size())
.detail("ForcedAttributesSize", forcedAttributes.size());
} else {
// when leastUsedMachine is empty, we will never find a team later, so we can simply return.
return addedMachineTeams;
}
// Choose a team that balances the # of teams per server among the teams
// that have the least-utilized server
team.clear();
ASSERT_WE_THINK(forcedAttributes.size() == 1);
auto success = machineLocalityMap.selectReplicas(configuration.storagePolicy, forcedAttributes, team);
// NOTE: selectReplicas() should always return success when storageTeamSize = 1
ASSERT_WE_THINK(configuration.storageTeamSize > 1 || (configuration.storageTeamSize == 1 && success));
if (!success) {
continue; // Try up to maxAttempts, since next time we may choose a different forcedAttributes
}
ASSERT(forcedAttributes.size() > 0);
team.push_back((UID*)machineLocalityMap.getObject(forcedAttributes[0]));
// selectReplicas() may NEVER return server not in server_info.
for (auto& pUID : team) {
ASSERT_WE_THINK(server_info.find(*pUID) != server_info.end());
}
// selectReplicas() should always return a team with correct size. otherwise, it has a bug
ASSERT(team.size() == configuration.storageTeamSize);
int score = 0;
std::vector<Standalone<StringRef>> machineIDs;
for (auto process = team.begin(); process != team.end(); process++) {
Reference<TCServerInfo> server = server_info[**process];
score += server->machine->machineTeams.size();
Standalone<StringRef> machine_id = server->lastKnownInterface.locality.zoneId().get();
machineIDs.push_back(machine_id);
}
// Only choose healthy machines into machine team
ASSERT_WE_THINK(isMachineTeamHealthy(machineIDs));
std::sort(machineIDs.begin(), machineIDs.end());
int overlap = overlappingMachineMembers(machineIDs);
if (overlap == machineIDs.size()) {
maxAttempts += 1;
continue;
}
score += SERVER_KNOBS->DD_OVERLAP_PENALTY * overlap;
// SOMEDAY: randomly pick one from teams with the lowest score
if (score < bestScore) {
// bestTeam is the team which has the smallest number of teams its team members belong to.
bestTeam = team;
bestScore = score;
}
}
// bestTeam should be a new valid team to be added into machine team now
// Step 5: Restore machine from its representative process team and get the machine team
if (bestTeam.size() == configuration.storageTeamSize) {
// machineIDs is used to quickly check if the machineIDs belong to an existed team
// machines keep machines reference for performance benefit by avoiding looking up machine by machineID
std::vector<Reference<TCMachineInfo>> machines;
for (auto process = bestTeam.begin(); process < bestTeam.end(); process++) {
Reference<TCMachineInfo> machine = server_info[**process]->machine;
machines.push_back(machine);
}
addMachineTeam(machines);
addedMachineTeams++;
} else {
traceAllInfo(true);
TraceEvent(SevWarn, "DataDistributionBuildTeams", distributorId)
.detail("Primary", primary)
.detail("Reason", "Unable to make desired machine Teams");
lastBuildTeamsFailed = true;
break;
}
}
return addedMachineTeams;
}
bool isMachineTeamHealthy(std::vector<Standalone<StringRef>> const& machineIDs) const;
bool isMachineTeamHealthy(TCMachineTeamInfo const& machineTeam) const {
int healthyNum = 0;
// A healthy machine team should have the desired number of machines
if (machineTeam.size() != configuration.storageTeamSize)
return false;
for (auto& machine : machineTeam.machines) {
if (isMachineHealthy(machine)) {
healthyNum++;
}
}
return (healthyNum == machineTeam.machines.size());
}
bool isMachineHealthy(Reference<TCMachineInfo> const& machine) const {
if (!machine.isValid() || machine_info.find(machine->machineID) == machine_info.end() ||
machine->serversOnMachine.empty()) {
return false;
}
// Healthy machine has at least one healthy server
for (auto& server : machine->serversOnMachine) {
if (!server_status.get(server->id).isUnhealthy()) {
return true;
}
}
return false;
}
// Return the healthy server with the least number of correct-size server teams
Reference<TCServerInfo> findOneLeastUsedServer() const {
std::vector<Reference<TCServerInfo>> leastUsedServers;
int minTeams = std::numeric_limits<int>::max();
for (auto& server : server_info) {
// Only pick healthy server, which is not failed or excluded.
if (server_status.get(server.first).isUnhealthy())
continue;
if (!isValidLocality(configuration.storagePolicy, server.second->lastKnownInterface.locality))
continue;
int numTeams = server.second->teams.size();
if (numTeams < minTeams) {
minTeams = numTeams;
leastUsedServers.clear();
}
if (minTeams == numTeams) {
leastUsedServers.push_back(server.second);
}
}
if (leastUsedServers.empty()) {
// If we cannot find a healthy server with valid locality
TraceEvent("NoHealthyAndValidLocalityServers")
.detail("Servers", server_info.size())
.detail("UnhealthyServers", unhealthyServers);
return Reference<TCServerInfo>();
} else {
return deterministicRandom()->randomChoice(leastUsedServers);
}
}
// Randomly choose one machine team that has chosenServer and has the correct size
// When configuration is changed, we may have machine teams with old storageTeamSize
Reference<TCMachineTeamInfo> findOneRandomMachineTeam(TCServerInfo const& chosenServer) const {
if (!chosenServer.machine->machineTeams.empty()) {
std::vector<Reference<TCMachineTeamInfo>> healthyMachineTeamsForChosenServer;
for (auto& mt : chosenServer.machine->machineTeams) {
if (isMachineTeamHealthy(*mt)) {
healthyMachineTeamsForChosenServer.push_back(mt);
}
}
if (!healthyMachineTeamsForChosenServer.empty()) {
return deterministicRandom()->randomChoice(healthyMachineTeamsForChosenServer);
}
}
// If we cannot find a healthy machine team
TraceEvent("NoHealthyMachineTeamForServer")
.detail("ServerID", chosenServer.id)
.detail("MachineTeams", chosenServer.machine->machineTeams.size());
return Reference<TCMachineTeamInfo>();
}
// A server team should always come from servers on a machine team
// Check if it is true
bool isOnSameMachineTeam(TCTeamInfo const& team) const {
std::vector<Standalone<StringRef>> machineIDs;
for (const auto& server : team.getServers()) {
if (!server->machine.isValid())
return false;
machineIDs.push_back(server->machine->machineID);
}
std::sort(machineIDs.begin(), machineIDs.end());
int numExistance = 0;
for (const auto& server : team.getServers()) {
for (const auto& candidateMachineTeam : server->machine->machineTeams) {
std::sort(candidateMachineTeam->machineIDs.begin(), candidateMachineTeam->machineIDs.end());
if (machineIDs == candidateMachineTeam->machineIDs) {
numExistance++;
break;
}
}
}
return (numExistance == team.size());
}
// Sanity check the property of teams in unit test
// Return true if all server teams belong to machine teams
bool sanityCheckTeams() const {
for (auto& team : teams) {
if (isOnSameMachineTeam(*team) == false) {
return false;
}
}
return true;
}
int calculateHealthyServerCount() const {
int serverCount = 0;
for (auto i = server_info.begin(); i != server_info.end(); ++i) {
if (!server_status.get(i->first).isUnhealthy()) {
++serverCount;
}
}
return serverCount;
}
int calculateHealthyMachineCount() const {
int totalHealthyMachineCount = 0;
for (auto& m : machine_info) {
if (isMachineHealthy(m.second)) {
++totalHealthyMachineCount;
}
}
return totalHealthyMachineCount;
}
std::pair<int64_t, int64_t> calculateMinMaxServerTeamsOnServer() const {
int64_t minTeams = std::numeric_limits<int64_t>::max();
int64_t maxTeams = 0;
for (auto& server : server_info) {
if (server_status.get(server.first).isUnhealthy()) {
continue;
}
minTeams = std::min((int64_t)server.second->teams.size(), minTeams);
maxTeams = std::max((int64_t)server.second->teams.size(), maxTeams);
}
return std::make_pair(minTeams, maxTeams);
}
std::pair<int64_t, int64_t> calculateMinMaxMachineTeamsOnMachine() const {
int64_t minTeams = std::numeric_limits<int64_t>::max();
int64_t maxTeams = 0;
for (auto& machine : machine_info) {
if (!isMachineHealthy(machine.second)) {
continue;
}
minTeams = std::min<int64_t>((int64_t)machine.second->machineTeams.size(), minTeams);
maxTeams = std::max<int64_t>((int64_t)machine.second->machineTeams.size(), maxTeams);
}
return std::make_pair(minTeams, maxTeams);
}
// Sanity check
bool isServerTeamCountCorrect(Reference<TCMachineTeamInfo> const& mt) const {
int num = 0;
bool ret = true;
for (auto& team : teams) {
if (team->machineTeam->machineIDs == mt->machineIDs) {
++num;
}
}
if (num != mt->serverTeams.size()) {
ret = false;
TraceEvent(SevError, "ServerTeamCountOnMachineIncorrect")
.detail("MachineTeam", mt->getMachineIDsStr())
.detail("ServerTeamsSize", mt->serverTeams.size())
.detail("CountedServerTeams", num);
}
return ret;
}
// Find the machine team with the least number of server teams
std::pair<Reference<TCMachineTeamInfo>, int> getMachineTeamWithLeastProcessTeams() const {
Reference<TCMachineTeamInfo> retMT;
int minNumProcessTeams = std::numeric_limits<int>::max();
for (auto& mt : machineTeams) {
if (EXPENSIVE_VALIDATION) {
ASSERT(isServerTeamCountCorrect(mt));
}
if (mt->serverTeams.size() < minNumProcessTeams) {
minNumProcessTeams = mt->serverTeams.size();
retMT = mt;
}
}
return std::pair<Reference<TCMachineTeamInfo>, int>(retMT, minNumProcessTeams);
}
// Find the machine team whose members are on the most number of machine teams, same logic as serverTeamRemover
std::pair<Reference<TCMachineTeamInfo>, int> getMachineTeamWithMostMachineTeams() const {
Reference<TCMachineTeamInfo> retMT;
int maxNumMachineTeams = 0;
int targetMachineTeamNumPerMachine =
(SERVER_KNOBS->DESIRED_TEAMS_PER_SERVER * (configuration.storageTeamSize + 1)) / 2;
for (auto& mt : machineTeams) {
// The representative team number for the machine team mt is
// the minimum number of machine teams of a machine in the team mt
int representNumMachineTeams = std::numeric_limits<int>::max();
for (auto& m : mt->machines) {
representNumMachineTeams = std::min<int>(representNumMachineTeams, m->machineTeams.size());
}
if (representNumMachineTeams > targetMachineTeamNumPerMachine &&
representNumMachineTeams > maxNumMachineTeams) {
maxNumMachineTeams = representNumMachineTeams;
retMT = mt;
}
}
return std::pair<Reference<TCMachineTeamInfo>, int>(retMT, maxNumMachineTeams);
}
// Find the server team whose members are on the most number of server teams
std::pair<Reference<TCTeamInfo>, int> getServerTeamWithMostProcessTeams() const {
Reference<TCTeamInfo> retST;
int maxNumProcessTeams = 0;
int targetTeamNumPerServer = (SERVER_KNOBS->DESIRED_TEAMS_PER_SERVER * (configuration.storageTeamSize + 1)) / 2;
for (auto& t : teams) {
// The minimum number of teams of a server in a team is the representative team number for the team t
int representNumProcessTeams = std::numeric_limits<int>::max();
for (auto& server : t->getServers()) {
representNumProcessTeams = std::min<int>(representNumProcessTeams, server->teams.size());
}
// We only remove the team whose representNumProcessTeams is larger than the targetTeamNumPerServer number
// otherwise, teamBuilder will build the to-be-removed team again
if (representNumProcessTeams > targetTeamNumPerServer && representNumProcessTeams > maxNumProcessTeams) {
maxNumProcessTeams = representNumProcessTeams;
retST = t;
}
}
return std::pair<Reference<TCTeamInfo>, int>(retST, maxNumProcessTeams);
}
int getHealthyMachineTeamCount() const {
int healthyTeamCount = 0;
for (const auto& mt : machineTeams) {
ASSERT(mt->machines.size() == configuration.storageTeamSize);
if (isMachineTeamHealthy(*mt)) {
++healthyTeamCount;
}
}
return healthyTeamCount;
}
// Each machine is expected to have targetMachineTeamNumPerMachine
// Return true if there exists a machine that does not have enough teams.
bool notEnoughMachineTeamsForAMachine() const {
// If we want to remove the machine team with most machine teams, we use the same logic as
// notEnoughTeamsForAServer
int targetMachineTeamNumPerMachine =
SERVER_KNOBS->TR_FLAG_REMOVE_MT_WITH_MOST_TEAMS
? (SERVER_KNOBS->DESIRED_TEAMS_PER_SERVER * (configuration.storageTeamSize + 1)) / 2
: SERVER_KNOBS->DESIRED_TEAMS_PER_SERVER;
for (auto& m : machine_info) {
// If SERVER_KNOBS->TR_FLAG_REMOVE_MT_WITH_MOST_TEAMS is false,
// The desired machine team number is not the same with the desired server team number
// in notEnoughTeamsForAServer() below, because the machineTeamRemover() does not
// remove a machine team with the most number of machine teams.
if (m.second->machineTeams.size() < targetMachineTeamNumPerMachine && isMachineHealthy(m.second)) {
return true;
}
}
return false;
}
// Each server is expected to have targetTeamNumPerServer teams.
// Return true if there exists a server that does not have enough teams.
bool notEnoughTeamsForAServer() const {
// We build more teams than we finally want so that we can use serverTeamRemover() actor to remove the teams
// whose member belong to too many teams. This allows us to get a more balanced number of teams per server.
// We want to ensure every server has targetTeamNumPerServer teams.
// The numTeamsPerServerFactor is calculated as
// (SERVER_KNOBS->DESIRED_TEAMS_PER_SERVER + ideal_num_of_teams_per_server) / 2
// ideal_num_of_teams_per_server is (#teams * storageTeamSize) / #servers, which is
// (#servers * DESIRED_TEAMS_PER_SERVER * storageTeamSize) / #servers.
int targetTeamNumPerServer = (SERVER_KNOBS->DESIRED_TEAMS_PER_SERVER * (configuration.storageTeamSize + 1)) / 2;
ASSERT(targetTeamNumPerServer > 0);
for (auto& s : server_info) {
if (s.second->teams.size() < targetTeamNumPerServer && !server_status.get(s.first).isUnhealthy()) {
return true;
}
}
return false;
}
// Create server teams based on machine teams
// Before the number of machine teams reaches the threshold, build a machine team for each server team
// When it reaches the threshold, first try to build a server team with existing machine teams; if failed,
// build an extra machine team and record the event in trace
int addTeamsBestOf(int teamsToBuild, int desiredTeams, int maxTeams) {
ASSERT(teamsToBuild >= 0);
ASSERT_WE_THINK(machine_info.size() > 0 || server_info.size() == 0);
ASSERT_WE_THINK(SERVER_KNOBS->DESIRED_TEAMS_PER_SERVER >= 1 && configuration.storageTeamSize >= 1);
int addedTeams = 0;
// Exclude machine teams who have members in the wrong configuration.
// When we change configuration, we may have machine teams with storageTeamSize in the old configuration.
int healthyMachineTeamCount = getHealthyMachineTeamCount();
int totalMachineTeamCount = machineTeams.size();
int totalHealthyMachineCount = calculateHealthyMachineCount();
int desiredMachineTeams = SERVER_KNOBS->DESIRED_TEAMS_PER_SERVER * totalHealthyMachineCount;
int maxMachineTeams = SERVER_KNOBS->MAX_TEAMS_PER_SERVER * totalHealthyMachineCount;
// machineTeamsToBuild mimics how the teamsToBuild is calculated in buildTeams()
int machineTeamsToBuild = std::max(
0, std::min(desiredMachineTeams - healthyMachineTeamCount, maxMachineTeams - totalMachineTeamCount));
{
TraceEvent te("BuildMachineTeams");
te.detail("TotalHealthyMachine", totalHealthyMachineCount)
.detail("HealthyMachineTeamCount", healthyMachineTeamCount)
.detail("DesiredMachineTeams", desiredMachineTeams)
.detail("MaxMachineTeams", maxMachineTeams)
.detail("MachineTeamsToBuild", machineTeamsToBuild);
// Pre-build all machine teams until we have the desired number of machine teams
if (machineTeamsToBuild > 0 || notEnoughMachineTeamsForAMachine()) {
auto addedMachineTeams = addBestMachineTeams(machineTeamsToBuild);
te.detail("MachineTeamsAdded", addedMachineTeams);
}
}
while (addedTeams < teamsToBuild || notEnoughTeamsForAServer()) {
// Step 1: Create 1 best machine team
std::vector<UID> bestServerTeam;
int bestScore = std::numeric_limits<int>::max();
int maxAttempts = SERVER_KNOBS->BEST_OF_AMT; // BEST_OF_AMT = 4
bool earlyQuitBuild = false;
for (int i = 0; i < maxAttempts && i < 100; ++i) {
// Step 2: Choose 1 least used server and then choose 1 least used machine team from the server
Reference<TCServerInfo> chosenServer = findOneLeastUsedServer();
if (!chosenServer.isValid()) {
TraceEvent(SevWarn, "NoValidServer").detail("Primary", primary);
earlyQuitBuild = true;
break;
}
// Note: To avoid creating correlation of picked machine teams, we simply choose a random machine team
// instead of choosing the least used machine team.
// The correlation happens, for example, when we add two new machines, we may always choose the machine
// team with these two new machines because they are typically less used.
Reference<TCMachineTeamInfo> chosenMachineTeam = findOneRandomMachineTeam(*chosenServer);
if (!chosenMachineTeam.isValid()) {
// We may face the situation that temporarily we have no healthy machine.
TraceEvent(SevWarn, "MachineTeamNotFound")
.detail("Primary", primary)
.detail("MachineTeams", machineTeams.size());
continue; // try randomly to find another least used server
}
// From here, chosenMachineTeam must have a healthy server team
// Step 3: Randomly pick 1 server from each machine in the chosen machine team to form a server team
std::vector<UID> serverTeam;
int chosenServerCount = 0;
for (auto& machine : chosenMachineTeam->machines) {
UID serverID;
if (machine == chosenServer->machine) {
serverID = chosenServer->id;
++chosenServerCount;
} else {
std::vector<Reference<TCServerInfo>> healthyProcesses;
for (auto it : machine->serversOnMachine) {
if (!server_status.get(it->id).isUnhealthy()) {
healthyProcesses.push_back(it);
}
}
serverID = deterministicRandom()->randomChoice(healthyProcesses)->id;
}
serverTeam.push_back(serverID);
}
ASSERT(chosenServerCount == 1); // chosenServer should be used exactly once
ASSERT(serverTeam.size() == configuration.storageTeamSize);
std::sort(serverTeam.begin(), serverTeam.end());
int overlap = overlappingMembers(serverTeam);
if (overlap == serverTeam.size()) {
maxAttempts += 1;
continue;
}
// Pick the server team with smallest score in all attempts
// If we use different metric here, DD may oscillate infinitely in creating and removing teams.
// SOMEDAY: Improve the code efficiency by using reservoir algorithm
int score = SERVER_KNOBS->DD_OVERLAP_PENALTY * overlap;
for (auto& server : serverTeam) {
score += server_info[server]->teams.size();
}
TraceEvent(SevDebug, "BuildServerTeams")
.detail("Score", score)
.detail("BestScore", bestScore)
.detail("TeamSize", serverTeam.size())
.detail("StorageTeamSize", configuration.storageTeamSize);
if (score < bestScore) {
bestScore = score;
bestServerTeam = serverTeam;
}
}
if (earlyQuitBuild) {
break;
}
if (bestServerTeam.size() != configuration.storageTeamSize) {
// Not find any team and will unlikely find a team
lastBuildTeamsFailed = true;
break;
}
// Step 4: Add the server team
addTeam(bestServerTeam.begin(), bestServerTeam.end(), false);
addedTeams++;
}
healthyMachineTeamCount = getHealthyMachineTeamCount();
std::pair<uint64_t, uint64_t> minMaxTeamsOnServer = calculateMinMaxServerTeamsOnServer();
std::pair<uint64_t, uint64_t> minMaxMachineTeamsOnMachine = calculateMinMaxMachineTeamsOnMachine();
TraceEvent("TeamCollectionInfo", distributorId)
.detail("Primary", primary)
.detail("AddedTeams", addedTeams)
.detail("TeamsToBuild", teamsToBuild)
.detail("CurrentServerTeams", teams.size())
.detail("DesiredTeams", desiredTeams)
.detail("MaxTeams", maxTeams)
.detail("StorageTeamSize", configuration.storageTeamSize)
.detail("CurrentMachineTeams", machineTeams.size())
.detail("CurrentHealthyMachineTeams", healthyMachineTeamCount)
.detail("DesiredMachineTeams", desiredMachineTeams)
.detail("MaxMachineTeams", maxMachineTeams)
.detail("TotalHealthyMachines", totalHealthyMachineCount)
.detail("MinTeamsOnServer", minMaxTeamsOnServer.first)
.detail("MaxTeamsOnServer", minMaxTeamsOnServer.second)
.detail("MinMachineTeamsOnMachine", minMaxMachineTeamsOnMachine.first)
.detail("MaxMachineTeamsOnMachine", minMaxMachineTeamsOnMachine.second)
.detail("DoBuildTeams", doBuildTeams)
.trackLatest(teamCollectionInfoEventHolder->trackingKey);
return addedTeams;
}
// Check if the number of server (and machine teams) is larger than the maximum allowed number
void traceTeamCollectionInfo() const {
int totalHealthyServerCount = calculateHealthyServerCount();
int desiredServerTeams = SERVER_KNOBS->DESIRED_TEAMS_PER_SERVER * totalHealthyServerCount;
int maxServerTeams = SERVER_KNOBS->MAX_TEAMS_PER_SERVER * totalHealthyServerCount;
int totalHealthyMachineCount = calculateHealthyMachineCount();
int desiredMachineTeams = SERVER_KNOBS->DESIRED_TEAMS_PER_SERVER * totalHealthyMachineCount;
int maxMachineTeams = SERVER_KNOBS->MAX_TEAMS_PER_SERVER * totalHealthyMachineCount;
int healthyMachineTeamCount = getHealthyMachineTeamCount();
std::pair<uint64_t, uint64_t> minMaxTeamsOnServer = calculateMinMaxServerTeamsOnServer();
std::pair<uint64_t, uint64_t> minMaxMachineTeamsOnMachine = calculateMinMaxMachineTeamsOnMachine();
TraceEvent("TeamCollectionInfo", distributorId)
.detail("Primary", primary)
.detail("AddedTeams", 0)
.detail("TeamsToBuild", 0)
.detail("CurrentServerTeams", teams.size())
.detail("DesiredTeams", desiredServerTeams)
.detail("MaxTeams", maxServerTeams)
.detail("StorageTeamSize", configuration.storageTeamSize)
.detail("CurrentMachineTeams", machineTeams.size())
.detail("CurrentHealthyMachineTeams", healthyMachineTeamCount)
.detail("DesiredMachineTeams", desiredMachineTeams)
.detail("MaxMachineTeams", maxMachineTeams)
.detail("TotalHealthyMachines", totalHealthyMachineCount)
.detail("MinTeamsOnServer", minMaxTeamsOnServer.first)
.detail("MaxTeamsOnServer", minMaxTeamsOnServer.second)
.detail("MinMachineTeamsOnMachine", minMaxMachineTeamsOnMachine.first)
.detail("MaxMachineTeamsOnMachine", minMaxMachineTeamsOnMachine.second)
.detail("DoBuildTeams", doBuildTeams)
.trackLatest(teamCollectionInfoEventHolder->trackingKey);
// Advance time so that we will not have multiple TeamCollectionInfo at the same time, otherwise
// simulation test will randomly pick one TeamCollectionInfo trace, which could be the one before build teams
// wait(delay(0.01));
// Debug purpose
// if (healthyMachineTeamCount > desiredMachineTeams || machineTeams.size() > maxMachineTeams) {
// // When the number of machine teams is over the limit, print out the current team info.
// traceAllInfo(true);
// }
}
// Use the current set of known processes (from server_info) to compute an optimized set of storage server teams.
// The following are guarantees of the process:
// - Each newly-built team will meet the replication policy
// - All newly-built teams will have exactly teamSize machines
//
// buildTeams() only ever adds teams to the list of teams. Teams are only removed from the list when all data has
// been removed.
//
// buildTeams will not count teams larger than teamSize against the desired teams.
Future<Void> buildTeams();
void noHealthyTeams() const {
std::set<UID> desiredServerSet;
std::string desc;
for (auto i = server_info.begin(); i != server_info.end(); ++i) {
ASSERT(i->first == i->second->id);
if (!server_status.get(i->first).isFailed) {
desiredServerSet.insert(i->first);
desc += i->first.shortString() + " (" + i->second->lastKnownInterface.toString() + "), ";
}
}
TraceEvent(SevWarn, "NoHealthyTeams", distributorId)
.detail("CurrentServerTeamCount", teams.size())
.detail("ServerCount", server_info.size())
.detail("NonFailedServerCount", desiredServerSet.size());
}
bool shouldHandleServer(const StorageServerInterface& newServer) const {
return (includedDCs.empty() ||
std::find(includedDCs.begin(), includedDCs.end(), newServer.locality.dcId()) != includedDCs.end() ||
(otherTrackedDCs.present() &&
std::find(otherTrackedDCs.get().begin(), otherTrackedDCs.get().end(), newServer.locality.dcId()) ==
otherTrackedDCs.get().end()));
}
void addServer(StorageServerInterface newServer,
ProcessClass processClass,
Promise<Void> errorOut,
Version addedVersion,
const DDEnabledState* ddEnabledState) {
if (!shouldHandleServer(newServer)) {
return;
}
if (!newServer.isTss()) {
allServers.push_back(newServer.id());
}
TraceEvent(newServer.isTss() ? "AddedTSS" : "AddedStorageServer", distributorId)
.detail("ServerID", newServer.id())
.detail("ProcessID", newServer.locality.processId())
.detail("ProcessClass", processClass.toString())
.detail("WaitFailureToken", newServer.waitFailure.getEndpoint().token)
.detail("Address", newServer.waitFailure.getEndpoint().getPrimaryAddress());
auto& r = server_and_tss_info[newServer.id()] = makeReference<TCServerInfo>(
newServer,
this,
processClass,
includedDCs.empty() ||
std::find(includedDCs.begin(), includedDCs.end(), newServer.locality.dcId()) != includedDCs.end(),
storageServerSet,
addedVersion);
if (newServer.isTss()) {
tss_info_by_pair[newServer.tssPairID.get()] = r;
if (server_info.count(newServer.tssPairID.get())) {
r->onTSSPairRemoved = server_info[newServer.tssPairID.get()]->onRemoved;
}
} else {
server_info[newServer.id()] = r;
// Establish the relation between server and machine
checkAndCreateMachine(r);
// Add storage server to pid map
ASSERT(r->lastKnownInterface.locality.processId().present());
StringRef pid = r->lastKnownInterface.locality.processId().get();
pid2server_info[pid].push_back(r);
}
r->tracker = storageServerTracker(cx, r.getPtr(), errorOut, addedVersion, ddEnabledState, newServer.isTss());
if (!newServer.isTss()) {
// link and wake up tss' tracker so it knows when this server gets removed
if (tss_info_by_pair.count(newServer.id())) {
tss_info_by_pair[newServer.id()]->onTSSPairRemoved = r->onRemoved;
if (tss_info_by_pair[newServer.id()]->wakeUpTracker.canBeSet()) {
auto p = tss_info_by_pair[newServer.id()]->wakeUpTracker;
// This callback could delete tss_info_by_pair[newServer.id()], so use a copy
p.send(Void());
}
}
doBuildTeams = true; // Adding a new server triggers to build new teams
restartTeamBuilder.trigger();
}
}
bool removeTeam(Reference<TCTeamInfo> team) {
TraceEvent("RemovedServerTeam", distributorId).detail("Team", team->getDesc());
bool found = false;
for (int t = 0; t < teams.size(); t++) {
if (teams[t] == team) {
teams[t--] = teams.back();
teams.pop_back();
found = true;
break;
}
}
for (const auto& server : team->getServers()) {
for (int t = 0; t < server->teams.size(); t++) {
if (server->teams[t] == team) {
ASSERT(found);
server->teams[t--] = server->teams.back();
server->teams.pop_back();
break; // The teams on a server should never duplicate
}
}
}
// Remove the team from its machine team
bool foundInMachineTeam = false;
for (int t = 0; t < team->machineTeam->serverTeams.size(); ++t) {
if (team->machineTeam->serverTeams[t] == team) {
team->machineTeam->serverTeams[t--] = team->machineTeam->serverTeams.back();
team->machineTeam->serverTeams.pop_back();
foundInMachineTeam = true;
break; // The same team is added to the serverTeams only once
}
}
ASSERT_WE_THINK(foundInMachineTeam);
team->tracker.cancel();
if (g_network->isSimulated()) {
// Update server team information for consistency check in simulation
traceTeamCollectionInfo();
}
return found;
}
// Check if the server belongs to a machine; if not, create the machine.
// Establish the two-direction link between server and machine
Reference<TCMachineInfo> checkAndCreateMachine(Reference<TCServerInfo> server) {
ASSERT(server.isValid() && server_info.find(server->id) != server_info.end());
auto& locality = server->lastKnownInterface.locality;
Standalone<StringRef> machine_id = locality.zoneId().get(); // locality to machine_id with std::string type
Reference<TCMachineInfo> machineInfo;
if (machine_info.find(machine_id) == machine_info.end()) {
// uid is the first storage server process on the machine
TEST(true); // First storage server in process on the machine
// For each machine, store the first server's localityEntry into machineInfo for later use.
LocalityEntry localityEntry = machineLocalityMap.add(locality, &server->id);
machineInfo = makeReference<TCMachineInfo>(server, localityEntry);
machine_info.insert(std::make_pair(machine_id, machineInfo));
} else {
machineInfo = machine_info.find(machine_id)->second;
machineInfo->serversOnMachine.push_back(server);
}
server->machine = machineInfo;
return machineInfo;
}
// Check if the serverTeam belongs to a machine team; If not, create the machine team
// Note: This function may make the machine team number larger than the desired machine team number
Reference<TCMachineTeamInfo> checkAndCreateMachineTeam(Reference<TCTeamInfo> serverTeam) {
std::vector<Standalone<StringRef>> machineIDs;
for (auto& server : serverTeam->getServers()) {
Reference<TCMachineInfo> machine = server->machine;
machineIDs.push_back(machine->machineID);
}
std::sort(machineIDs.begin(), machineIDs.end());
Reference<TCMachineTeamInfo> machineTeam = findMachineTeam(machineIDs);
if (!machineTeam.isValid()) { // Create the machine team if it does not exist
machineTeam = addMachineTeam(machineIDs.begin(), machineIDs.end());
}
machineTeam->serverTeams.push_back(serverTeam);
return machineTeam;
}
// Remove the removedMachineInfo machine and any related machine team
void removeMachine(Reference<TCMachineInfo> removedMachineInfo) {
// Find machines that share teams with the removed machine
std::set<Standalone<StringRef>> machinesWithAjoiningTeams;
for (auto& machineTeam : removedMachineInfo->machineTeams) {
machinesWithAjoiningTeams.insert(machineTeam->machineIDs.begin(), machineTeam->machineIDs.end());
}
machinesWithAjoiningTeams.erase(removedMachineInfo->machineID);
// For each machine in a machine team with the removed machine,
// erase shared machine teams from the list of teams.
for (auto it = machinesWithAjoiningTeams.begin(); it != machinesWithAjoiningTeams.end(); ++it) {
auto& machineTeams = machine_info[*it]->machineTeams;
for (int t = 0; t < machineTeams.size(); t++) {
auto& machineTeam = machineTeams[t];
if (std::count(machineTeam->machineIDs.begin(),
machineTeam->machineIDs.end(),
removedMachineInfo->machineID)) {
machineTeams[t--] = machineTeams.back();
machineTeams.pop_back();
}
}
}
removedMachineInfo->machineTeams.clear();
// Remove global machine team that includes removedMachineInfo
for (int t = 0; t < machineTeams.size(); t++) {
auto& machineTeam = machineTeams[t];
if (std::count(
machineTeam->machineIDs.begin(), machineTeam->machineIDs.end(), removedMachineInfo->machineID)) {
removeMachineTeam(machineTeam);
// removeMachineTeam will swap the last team in machineTeams vector into [t];
// t-- to avoid skipping the element
t--;
}
}
// Remove removedMachineInfo from machine's global info
machine_info.erase(removedMachineInfo->machineID);
TraceEvent("MachineLocalityMapUpdate").detail("MachineUIDRemoved", removedMachineInfo->machineID.toString());
// We do not update macineLocalityMap when a machine is removed because we will do so when we use it in
// addBestMachineTeams()
// rebuildMachineLocalityMap();
}
// Invariant: Remove a machine team only when the server teams on it has been removed
// We never actively remove a machine team.
// A machine team is removed when a machine is removed,
// which is caused by the event when all servers on the machine is removed.
// NOTE: When this function is called in the loop of iterating machineTeams, make sure NOT increase the index
// in the next iteration of the loop. Otherwise, you may miss checking some elements in machineTeams
bool removeMachineTeam(Reference<TCMachineTeamInfo> targetMT) {
bool foundMachineTeam = false;
for (int i = 0; i < machineTeams.size(); i++) {
Reference<TCMachineTeamInfo> mt = machineTeams[i];
if (mt->machineIDs == targetMT->machineIDs) {
machineTeams[i--] = machineTeams.back();
machineTeams.pop_back();
foundMachineTeam = true;
break;
}
}
// Remove machine team on each machine
for (auto& machine : targetMT->machines) {
for (int i = 0; i < machine->machineTeams.size(); ++i) {
if (machine->machineTeams[i]->machineIDs == targetMT->machineIDs) {
machine->machineTeams[i--] = machine->machineTeams.back();
machine->machineTeams.pop_back();
break; // The machineTeams on a machine should never duplicate
}
}
}
return foundMachineTeam;
}
void removeTSS(UID removedServer) {
// much simpler than remove server. tss isn't in any teams, so just remove it from data structures
TraceEvent("RemovedTSS", distributorId).detail("ServerID", removedServer);
Reference<TCServerInfo> removedServerInfo = server_and_tss_info[removedServer];
tss_info_by_pair.erase(removedServerInfo->lastKnownInterface.tssPairID.get());
server_and_tss_info.erase(removedServer);
server_status.clear(removedServer);
}
void removeServer(UID removedServer) {
TraceEvent("RemovedStorageServer", distributorId).detail("ServerID", removedServer);
// ASSERT( !shardsAffectedByTeamFailure->getServersForTeam( t ) for all t in teams that contain removedServer )
Reference<TCServerInfo> removedServerInfo = server_info[removedServer];
// Step: Remove TCServerInfo from pid2server_info
ASSERT(removedServerInfo->lastKnownInterface.locality.processId().present());
StringRef pid = removedServerInfo->lastKnownInterface.locality.processId().get();
auto& info_vec = pid2server_info[pid];
for (size_t i = 0; i < info_vec.size(); ++i) {
if (info_vec[i] == removedServerInfo) {
info_vec[i--] = info_vec.back();
info_vec.pop_back();
}
}
if (info_vec.size() == 0) {
pid2server_info.erase(pid);
}
// Step: Remove server team that relate to removedServer
// Find all servers with which the removedServer shares teams
std::set<UID> serversWithAjoiningTeams;
auto& sharedTeams = removedServerInfo->teams;
for (int i = 0; i < sharedTeams.size(); ++i) {
auto& teamIds = sharedTeams[i]->getServerIDs();
serversWithAjoiningTeams.insert(teamIds.begin(), teamIds.end());
}
serversWithAjoiningTeams.erase(removedServer);
// For each server in a team with the removedServer, erase shared teams from the list of teams in that other
// server
for (auto it = serversWithAjoiningTeams.begin(); it != serversWithAjoiningTeams.end(); ++it) {
auto& serverTeams = server_info[*it]->teams;
for (int t = 0; t < serverTeams.size(); t++) {
auto& serverIds = serverTeams[t]->getServerIDs();
if (std::count(serverIds.begin(), serverIds.end(), removedServer)) {
serverTeams[t--] = serverTeams.back();
serverTeams.pop_back();
}
}
}
// Step: Remove all teams that contain removedServer
// SOMEDAY: can we avoid walking through all teams, since we have an index of teams in which removedServer
// participated
int removedCount = 0;
for (int t = 0; t < teams.size(); t++) {
if (std::count(teams[t]->getServerIDs().begin(), teams[t]->getServerIDs().end(), removedServer)) {
TraceEvent("ServerTeamRemoved")
.detail("Primary", primary)
.detail("TeamServerIDs", teams[t]->getServerIDsStr())
.detail("TeamID", teams[t]->getTeamID());
// removeTeam also needs to remove the team from the machine team info.
removeTeam(teams[t]);
t--;
removedCount++;
}
}
if (removedCount == 0) {
TraceEvent(SevInfo, "NoTeamsRemovedWhenServerRemoved")
.detail("Primary", primary)
.detail("Debug", "ThisShouldRarelyHappen_CheckInfoBelow");
}
for (int t = 0; t < badTeams.size(); t++) {
if (std::count(badTeams[t]->getServerIDs().begin(), badTeams[t]->getServerIDs().end(), removedServer)) {
badTeams[t]->tracker.cancel();
badTeams[t--] = badTeams.back();
badTeams.pop_back();
}
}
// Step: Remove machine info related to removedServer
// Remove the server from its machine
Reference<TCMachineInfo> removedMachineInfo = removedServerInfo->machine;
for (int i = 0; i < removedMachineInfo->serversOnMachine.size(); ++i) {
if (removedMachineInfo->serversOnMachine[i] == removedServerInfo) {
// Safe even when removedServerInfo is the last one
removedMachineInfo->serversOnMachine[i--] = removedMachineInfo->serversOnMachine.back();
removedMachineInfo->serversOnMachine.pop_back();
break;
}
}
// Remove machine if no server on it
// Note: Remove machine (and machine team) after server teams have been removed, because
// we remove a machine team only when the server teams on it have been removed
if (removedMachineInfo->serversOnMachine.size() == 0) {
removeMachine(removedMachineInfo);
}
// If the machine uses removedServer's locality and the machine still has servers, the the machine's
// representative server will be updated when it is used in addBestMachineTeams()
// Note that since we do not rebuildMachineLocalityMap() here, the machineLocalityMap can be stale.
// This is ok as long as we do not arbitrarily validate if machine team satisfies replication policy.
if (server_info[removedServer]->wrongStoreTypeToRemove.get()) {
if (wrongStoreTypeRemover.isReady()) {
wrongStoreTypeRemover = removeWrongStoreType();
addActor.send(wrongStoreTypeRemover);
}
}
// Step: Remove removedServer from server's global data
for (int s = 0; s < allServers.size(); s++) {
if (allServers[s] == removedServer) {
allServers[s--] = allServers.back();
allServers.pop_back();
}
}
server_info.erase(removedServer);
server_and_tss_info.erase(removedServer);
if (server_status.get(removedServer).initialized && server_status.get(removedServer).isUnhealthy()) {
unhealthyServers--;
}
server_status.clear(removedServer);
// FIXME: add remove support to localitySet so we do not have to recreate it
resetLocalitySet();
doBuildTeams = true;
restartTeamBuilder.trigger();
TraceEvent("DataDistributionTeamCollectionUpdate", distributorId)
.detail("ServerTeams", teams.size())
.detail("BadServerTeams", badTeams.size())
.detail("Servers", allServers.size())
.detail("Machines", machine_info.size())
.detail("MachineTeams", machineTeams.size())
.detail("DesiredTeamsPerServer", SERVER_KNOBS->DESIRED_TEAMS_PER_SERVER);
}
// Adds storage servers held on process of which the Process Id is “pid” into excludeServers which prevent
// recruiting the wiggling storage servers and let teamTracker start to move data off the affected teams;
// Return a vector of futures wait for all data is moved to other teams.
std::vector<Future<Void>> excludeStorageServersForWiggle(const Value& pid) {
std::vector<Future<Void>> moveFutures;
if (this->pid2server_info.count(pid) != 0) {
for (auto& info : this->pid2server_info[pid]) {
AddressExclusion addr(info->lastKnownInterface.address().ip, info->lastKnownInterface.address().port);
if (this->excludedServers.count(addr) &&
this->excludedServers.get(addr) != DDTeamCollection::Status::NONE) {
continue; // don't overwrite the value set by actor trackExcludedServer
}
this->wiggle_addresses.push_back(addr);
this->excludedServers.set(addr, DDTeamCollection::Status::WIGGLING);
moveFutures.push_back(info->onRemoved);
}
if (!moveFutures.empty()) {
this->restartRecruiting.trigger();
}
}
return moveFutures;
}
// Include wiggled storage servers by setting their status from `WIGGLING`
// to `NONE`. The storage recruiter will recruit them as new storage servers
void includeStorageServersForWiggle() {
bool included = false;
for (auto& address : this->wiggle_addresses) {
if (!this->excludedServers.count(address) ||
this->excludedServers.get(address) != DDTeamCollection::Status::WIGGLING) {
continue;
}
included = true;
this->excludedServers.set(address, DDTeamCollection::Status::NONE);
}
this->wiggle_addresses.clear();
if (included) {
this->restartRecruiting.trigger();
}
}
// Track a team and issue RelocateShards when the level of degradation changes
// A badTeam can be unhealthy or just a redundantTeam removed by machineTeamRemover() or serverTeamRemover()
Future<Void> teamTracker(Reference<TCTeamInfo> team, bool badTeam, bool redundantTeam);
// Check the status of a storage server.
// Apply all requirements to the server and mark it as excluded if it fails to satisfies these requirements
Future<Void> storageServerTracker(Database cx,
TCServerInfo* server,
Promise<Void> errorOut,
Version addedVersion,
const DDEnabledState* ddEnabledState,
bool isTss);
Future<Void> removeWrongStoreType();
bool teamContainsFailedServer(Reference<TCTeamInfo> team);
// NOTE: this actor returns when the cluster is healthy and stable (no server is expected to be removed in a period)
// processingWiggle and processingUnhealthy indicate that some servers are going to be removed.
Future<Void> waitUntilHealthy(double extraDelay = 0, bool waitWiggle = false);
bool isCorrectDC(TCServerInfo* server) {
return (includedDCs.empty() ||
std::find(includedDCs.begin(), includedDCs.end(), server->lastKnownInterface.locality.dcId()) !=
includedDCs.end());
}
Future<Void> removeBadTeams();
Future<Void> zeroServerLeftLoggerActor(Reference<TCTeamInfo> team);
// Set the server's storeType; Error is catched by the caller
Future<Void> keyValueStoreTypeTracker(TCServerInfo* server);
Future<Void> storageServerFailureTracker(TCServerInfo* server,
Database cx,
ServerStatus* status,
Version addedVersion);
Future<Void> waitForAllDataRemoved(Database cx, UID serverID, Version addedVersion);
Future<Void> machineTeamRemover();
// Remove the server team whose members have the most number of process teams
// until the total number of server teams is no larger than the desired number
Future<Void> serverTeamRemover();
Future<Void> trackExcludedServers();
// Create a transaction reading the value of `wigglingStorageServerKey` and update it to the next Process ID
// according to a sorted PID set maintained by the data distributor. If now no storage server exists, the new
// Process ID is 0.
Future<Void> updateNextWigglingStoragePID();
// Iterate over each storage process to do storage wiggle. After initializing the first Process ID, it waits a
// signal from `perpetualStorageWiggler` indicating the wiggling of current process is finished. Then it writes the
// next Process ID to a system key: `wigglingStorageServerKey` to show the next process to wiggle.
Future<Void> perpetualStorageWiggleIterator(AsyncVar<bool>* stopSignal,
FutureStream<Void> finishStorageWiggleSignal);
// Watch the value change of `wigglingStorageServerKey`.
// Return the watch future and the current value of `wigglingStorageServerKey`.
Future<std::pair<Future<Void>, Value>> watchPerpetualStoragePIDChange();
// periodically check whether the cluster is healthy if we continue perpetual wiggle
Future<Void> clusterHealthCheckForPerpetualWiggle(int* extraTeamCount);
// Watches the value (pid) change of \xff/storageWigglePID, and adds storage servers held on process of which the
// Process Id is “pid” into excludeServers which prevent recruiting the wiggling storage servers and let teamTracker
// start to move data off the affected teams. The wiggling process of current storage servers will be paused if the
// cluster is unhealthy and restarted once the cluster is healthy again.
Future<Void> perpetualStorageWiggler(AsyncVar<bool>* stopSignal, PromiseStream<Void> finishStorageWiggleSignal);
// This coroutine sets a watch to monitor the value change of `perpetualStorageWiggleKey` which is controlled by
// command `configure perpetual_storage_wiggle=$value` if the value is 1, this actor start 2 actors,
// `perpetualStorageWiggleIterator` and `perpetualStorageWiggler`. Otherwise, it sends stop signal to them.
Future<Void> monitorPerpetualStorageWiggle();
// The serverList system keyspace keeps the StorageServerInterface for each serverID. Storage server's storeType
// and serverID are decided by the server's filename. By parsing storage server file's filename on each disk,
// process on each machine creates the TCServer with the correct serverID and StorageServerInterface.
Future<Void> waitServerListChange(FutureStream<Void> serverRemoved, const DDEnabledState* ddEnabledState);
Future<Void> waitHealthyZoneChange();
// Monitor whether or not storage servers are being recruited. If so, then a database cannot be considered quiet
Future<Void> monitorStorageServerRecruitment();
int numExistingSSOnAddr(const AddressExclusion& addr) {
int numExistingSS = 0;
for (auto& server : server_and_tss_info) {
const NetworkAddress& netAddr = server.second->lastKnownInterface.stableAddress();
AddressExclusion usedAddr(netAddr.ip, netAddr.port);
if (usedAddr == addr) {
++numExistingSS;
}
}
return numExistingSS;
}
Future<Void> initializeStorage(RecruitStorageReply candidateWorker,
DDEnabledState const* ddEnabledState,
bool recruitTss,
Reference<TSSPairState> tssState);
Future<Void> storageRecruiter(Reference<IAsyncListener<RequestStream<RecruitStorageRequest>>> recruitStorage,
DDEnabledState const* ddEnabledState);
Future<Void> updateReplicasKey(Optional<Key> dcId);
Future<Void> serverGetTeamRequests(TeamCollectionInterface tci);
Future<Void> monitorHealthyTeams();
// Find size of set intersection of excludeServerIDs and serverIDs on each team and see if the leftover team is
// valid
bool exclusionSafetyCheck(std::vector<UID>& excludeServerIDs) {
std::sort(excludeServerIDs.begin(), excludeServerIDs.end());
for (const auto& team : teams) {
std::vector<UID> teamServerIDs = team->getServerIDs();
std::sort(teamServerIDs.begin(), teamServerIDs.end());
TraceEvent(SevDebug, "DDExclusionSafetyCheck", distributorId)
.detail("Excluding", describe(excludeServerIDs))
.detail("Existing", team->getDesc());
// Find size of set intersection of both vectors and see if the leftover team is valid
std::vector<UID> intersectSet(teamServerIDs.size());
auto it = std::set_intersection(excludeServerIDs.begin(),
excludeServerIDs.end(),
teamServerIDs.begin(),
teamServerIDs.end(),
intersectSet.begin());
intersectSet.resize(it - intersectSet.begin());
if (teamServerIDs.size() - intersectSet.size() < SERVER_KNOBS->DD_EXCLUDE_MIN_REPLICAS) {
return false;
}
}
return true;
}
Future<UID> getClusterId();
};