mirror of
https://github.com/apple/foundationdb.git
synced 2025-05-15 02:18:39 +08:00
Merge remote-tracking branch 'upstream/release-6.2' into fdbcli-tlsinfo
This commit is contained in:
commit
db5863145a
@ -188,6 +188,9 @@
|
||||
},
|
||||
"megabits_received":{
|
||||
"hz":0.0
|
||||
},
|
||||
"tls_policy_failures":{
|
||||
"hz":0.0
|
||||
}
|
||||
},
|
||||
"run_loop_busy":0.2 // fraction of time the run loop was busy
|
||||
@ -399,6 +402,7 @@
|
||||
},
|
||||
"required_logs":3,
|
||||
"missing_logs":"7f8d623d0cb9966e",
|
||||
"active_generations":1,
|
||||
"description":"Recovery complete."
|
||||
},
|
||||
"workload":{
|
||||
|
@ -107,7 +107,7 @@ struct DatabaseConfiguration {
|
||||
|
||||
int expectedLogSets( Optional<Key> dcId ) const {
|
||||
int result = 1;
|
||||
if(dcId.present() && getRegion(dcId.get()).satelliteTLogReplicationFactor > 0) {
|
||||
if(dcId.present() && getRegion(dcId.get()).satelliteTLogReplicationFactor > 0 && usableRegions > 1) {
|
||||
result++;
|
||||
}
|
||||
|
||||
|
@ -41,6 +41,10 @@ ClientKnobs::ClientKnobs(bool randomize) {
|
||||
init( CLIENT_FAILURE_TIMEOUT_DELAY, FAILURE_MIN_DELAY );
|
||||
init( FAILURE_EMERGENCY_DELAY, 30.0 );
|
||||
init( FAILURE_MAX_GENERATIONS, 10 );
|
||||
init( RECOVERY_DELAY_START_GENERATION, 70 );
|
||||
init( RECOVERY_DELAY_SECONDS_PER_GENERATION, 60.0 );
|
||||
init( MAX_GENERATIONS, 100 );
|
||||
init( MAX_GENERATIONS_OVERRIDE, 0 );
|
||||
|
||||
init( COORDINATOR_RECONNECTION_DELAY, 1.0 );
|
||||
init( CLIENT_EXAMPLE_AMOUNT, 20 );
|
||||
|
@ -40,6 +40,10 @@ public:
|
||||
double CLIENT_FAILURE_TIMEOUT_DELAY;
|
||||
double FAILURE_EMERGENCY_DELAY;
|
||||
double FAILURE_MAX_GENERATIONS;
|
||||
double RECOVERY_DELAY_START_GENERATION;
|
||||
double RECOVERY_DELAY_SECONDS_PER_GENERATION;
|
||||
double MAX_GENERATIONS;
|
||||
double MAX_GENERATIONS_OVERRIDE;
|
||||
|
||||
double COORDINATOR_RECONNECTION_DELAY;
|
||||
int CLIENT_EXAMPLE_AMOUNT;
|
||||
|
@ -224,7 +224,7 @@ template <> void delref( DatabaseContext* ptr ) { ptr->delref(); }
|
||||
ACTOR Future<Void> databaseLogger( DatabaseContext *cx ) {
|
||||
state double lastLogged = 0;
|
||||
loop {
|
||||
wait(delay(CLIENT_KNOBS->SYSTEM_MONITOR_INTERVAL, cx->taskID));
|
||||
wait(delay(CLIENT_KNOBS->SYSTEM_MONITOR_INTERVAL, TaskPriority::FlushTrace));
|
||||
TraceEvent ev("TransactionMetrics", cx->dbId);
|
||||
|
||||
ev.detail("Elapsed", (lastLogged == 0) ? 0 : now() - lastLogged)
|
||||
|
@ -1156,7 +1156,7 @@ Future<Version> ReadYourWritesTransaction::getReadVersion() {
|
||||
|
||||
Optional<Value> getValueFromJSON(StatusObject statusObj) {
|
||||
try {
|
||||
Value output = StringRef(json_spirit::write_string(json_spirit::mValue(statusObj), json_spirit::Output_options::raw_utf8).c_str());
|
||||
Value output = StringRef(json_spirit::write_string(json_spirit::mValue(statusObj), json_spirit::Output_options::none));
|
||||
return output;
|
||||
}
|
||||
catch (std::exception& e){
|
||||
|
@ -208,6 +208,9 @@ const KeyRef JSONSchemas::statusSchema = LiteralStringRef(R"statusSchema(
|
||||
},
|
||||
"megabits_received":{
|
||||
"hz":0.0
|
||||
},
|
||||
"tls_policy_failures":{
|
||||
"hz":0.0
|
||||
}
|
||||
},
|
||||
"run_loop_busy":0.2
|
||||
@ -425,6 +428,7 @@ const KeyRef JSONSchemas::statusSchema = LiteralStringRef(R"statusSchema(
|
||||
},
|
||||
"required_logs":3,
|
||||
"missing_logs":"7f8d623d0cb9966e",
|
||||
"active_generations":1,
|
||||
"description":"Recovery complete."
|
||||
},
|
||||
"workload":{
|
||||
|
@ -683,7 +683,7 @@ public:
|
||||
}
|
||||
|
||||
std::vector<WorkerDetails> satelliteLogs;
|
||||
if(region.satelliteTLogReplicationFactor > 0) {
|
||||
if(region.satelliteTLogReplicationFactor > 0 && req.configuration.usableRegions > 1) {
|
||||
satelliteLogs = getWorkersForSatelliteLogs( req.configuration, region, remoteRegion, id_used, result.satelliteFallback );
|
||||
for(int i = 0; i < satelliteLogs.size(); i++) {
|
||||
result.satelliteTLogs.push_back(satelliteLogs[i].interf);
|
||||
@ -718,7 +718,7 @@ public:
|
||||
|
||||
if( !goodRecruitmentTime.isReady() &&
|
||||
( RoleFitness(SERVER_KNOBS->EXPECTED_TLOG_FITNESS, req.configuration.getDesiredLogs(), ProcessClass::TLog).betterCount(RoleFitness(tlogs, ProcessClass::TLog)) ||
|
||||
( region.satelliteTLogReplicationFactor > 0 && RoleFitness(SERVER_KNOBS->EXPECTED_TLOG_FITNESS, req.configuration.getDesiredSatelliteLogs(dcId), ProcessClass::TLog).betterCount(RoleFitness(satelliteLogs, ProcessClass::TLog)) ) ||
|
||||
( region.satelliteTLogReplicationFactor > 0 && req.configuration.usableRegions > 1 && RoleFitness(SERVER_KNOBS->EXPECTED_TLOG_FITNESS, req.configuration.getDesiredSatelliteLogs(dcId), ProcessClass::TLog).betterCount(RoleFitness(satelliteLogs, ProcessClass::TLog)) ) ||
|
||||
RoleFitness(SERVER_KNOBS->EXPECTED_PROXY_FITNESS, req.configuration.getDesiredProxies(), ProcessClass::Proxy).betterCount(RoleFitness(proxies, ProcessClass::Proxy)) ||
|
||||
RoleFitness(SERVER_KNOBS->EXPECTED_RESOLVER_FITNESS, req.configuration.getDesiredResolvers(), ProcessClass::Resolver).betterCount(RoleFitness(resolvers, ProcessClass::Resolver)) ) ) {
|
||||
return operation_failed();
|
||||
@ -895,7 +895,7 @@ public:
|
||||
std::set<Optional<Key>> primaryDC;
|
||||
primaryDC.insert(regions[0].dcId);
|
||||
getWorkersForTlogs(db.config, db.config.tLogReplicationFactor, db.config.getDesiredLogs(), db.config.tLogPolicy, id_used, true, primaryDC);
|
||||
if(regions[0].satelliteTLogReplicationFactor > 0) {
|
||||
if(regions[0].satelliteTLogReplicationFactor > 0 && db.config.usableRegions > 1) {
|
||||
bool satelliteFallback = false;
|
||||
getWorkersForSatelliteLogs(db.config, regions[0], regions[1], id_used, satelliteFallback, true);
|
||||
}
|
||||
@ -1068,7 +1068,7 @@ public:
|
||||
|
||||
RoleFitness oldSatelliteTLogFit(satellite_tlogs, ProcessClass::TLog);
|
||||
bool newSatelliteFallback = false;
|
||||
auto newSatelliteTLogs = region.satelliteTLogReplicationFactor > 0 ? getWorkersForSatelliteLogs(db.config, region, remoteRegion, id_used, newSatelliteFallback, true) : satellite_tlogs;
|
||||
auto newSatelliteTLogs = (region.satelliteTLogReplicationFactor > 0 && db.config.usableRegions > 1) ? getWorkersForSatelliteLogs(db.config, region, remoteRegion, id_used, newSatelliteFallback, true) : satellite_tlogs;
|
||||
RoleFitness newSatelliteTLogFit(newSatelliteTLogs, ProcessClass::TLog);
|
||||
|
||||
std::map<Optional<Key>,int32_t> satellite_priority;
|
||||
|
@ -627,6 +627,9 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
|
||||
double medianAvailableSpace;
|
||||
double lastMedianAvailableSpaceUpdate;
|
||||
|
||||
int lowestUtilizationTeam;
|
||||
int highestUtilizationTeam;
|
||||
|
||||
void resetLocalitySet() {
|
||||
storageServerSet = Reference<LocalitySet>(new LocalityMap<UID>());
|
||||
LocalityMap<UID>* storageServerMap = (LocalityMap<UID>*) storageServerSet.getPtr();
|
||||
@ -671,7 +674,7 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
|
||||
optimalTeamCount(0), recruitingStream(0), restartRecruiting(SERVER_KNOBS->DEBOUNCE_RECRUITING_DELAY),
|
||||
unhealthyServers(0), includedDCs(includedDCs), otherTrackedDCs(otherTrackedDCs),
|
||||
zeroHealthyTeams(zeroHealthyTeams), zeroOptimalTeams(true), primary(primary), medianAvailableSpace(SERVER_KNOBS->MIN_AVAILABLE_SPACE_RATIO),
|
||||
lastMedianAvailableSpaceUpdate(0), processingUnhealthy(processingUnhealthy) {
|
||||
lastMedianAvailableSpaceUpdate(0), processingUnhealthy(processingUnhealthy), lowestUtilizationTeam(0), highestUtilizationTeam(0) {
|
||||
if(!primary || configuration.usableRegions == 1) {
|
||||
TraceEvent("DDTrackerStarting", distributorId)
|
||||
.detail( "State", "Inactive" )
|
||||
@ -809,18 +812,29 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
|
||||
|
||||
if( req.wantsTrueBest ) {
|
||||
ASSERT( !bestOption.present() );
|
||||
auto &startIndex = req.preferLowerUtilization ? self->lowestUtilizationTeam : self->highestUtilizationTeam;
|
||||
if(startIndex >= self->teams.size()) {
|
||||
startIndex = 0;
|
||||
}
|
||||
|
||||
int bestIndex = startIndex;
|
||||
for( int i = 0; i < self->teams.size(); i++ ) {
|
||||
if (self->teams[i]->isHealthy() &&
|
||||
(!req.preferLowerUtilization || self->teams[i]->hasHealthyAvailableSpace(self->medianAvailableSpace)) &&
|
||||
(!req.teamMustHaveShards || self->shardsAffectedByTeamFailure->getShardsFor(ShardsAffectedByTeamFailure::Team(self->teams[i]->getServerIDs(), self->primary)).size() > 0))
|
||||
int currentIndex = (startIndex + i) % self->teams.size();
|
||||
if (self->teams[currentIndex]->isHealthy() &&
|
||||
(!req.preferLowerUtilization || self->teams[currentIndex]->hasHealthyAvailableSpace(self->medianAvailableSpace)))
|
||||
{
|
||||
int64_t loadBytes = self->teams[currentIndex]->getLoadBytes(true, req.inflightPenalty);
|
||||
if((!bestOption.present() || (req.preferLowerUtilization && loadBytes < bestLoadBytes) || (!req.preferLowerUtilization && loadBytes > bestLoadBytes)) &&
|
||||
(!req.teamMustHaveShards || self->shardsAffectedByTeamFailure->hasShards(ShardsAffectedByTeamFailure::Team(self->teams[currentIndex]->getServerIDs(), self->primary))))
|
||||
{
|
||||
int64_t loadBytes = self->teams[i]->getLoadBytes(true, req.inflightPenalty);
|
||||
if( !bestOption.present() || ( req.preferLowerUtilization && loadBytes < bestLoadBytes ) || ( !req.preferLowerUtilization && loadBytes > bestLoadBytes ) ) {
|
||||
bestLoadBytes = loadBytes;
|
||||
bestOption = self->teams[i];
|
||||
bestOption = self->teams[currentIndex];
|
||||
bestIndex = currentIndex;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
startIndex = bestIndex;
|
||||
}
|
||||
else {
|
||||
int nTries = 0;
|
||||
@ -828,8 +842,7 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
|
||||
Reference<IDataDistributionTeam> dest = deterministicRandom()->randomChoice(self->teams);
|
||||
|
||||
bool ok = dest->isHealthy() &&
|
||||
(!req.preferLowerUtilization || dest->hasHealthyAvailableSpace(self->medianAvailableSpace)) &&
|
||||
(!req.teamMustHaveShards || self->shardsAffectedByTeamFailure->getShardsFor(ShardsAffectedByTeamFailure::Team(dest->getServerIDs(), self->primary)).size() > 0);
|
||||
(!req.preferLowerUtilization || dest->hasHealthyAvailableSpace(self->medianAvailableSpace));
|
||||
|
||||
for(int i=0; ok && i<randomTeams.size(); i++) {
|
||||
if (randomTeams[i]->getServerIDs() == dest->getServerIDs()) {
|
||||
@ -838,6 +851,8 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
|
||||
}
|
||||
}
|
||||
|
||||
ok = ok && (!req.teamMustHaveShards || self->shardsAffectedByTeamFailure->hasShards(ShardsAffectedByTeamFailure::Team(dest->getServerIDs(), self->primary)));
|
||||
|
||||
if (ok)
|
||||
randomTeams.push_back( dest );
|
||||
else
|
||||
@ -2746,7 +2761,7 @@ ACTOR Future<Void> serverTeamRemover(DDTeamCollection* self) {
|
||||
ACTOR Future<Void> teamTracker(DDTeamCollection* self, Reference<TCTeamInfo> team, bool badTeam, bool redundantTeam) {
|
||||
state int lastServersLeft = team->size();
|
||||
state bool lastAnyUndesired = false;
|
||||
state bool logTeamEvents = g_network->isSimulated() || !badTeam;
|
||||
state bool logTeamEvents = g_network->isSimulated() || !badTeam || team->size() <= self->configuration.storageTeamSize;
|
||||
state bool lastReady = false;
|
||||
state bool lastHealthy;
|
||||
state bool lastOptimal;
|
||||
@ -2789,6 +2804,10 @@ ACTOR Future<Void> teamTracker(DDTeamCollection* self, Reference<TCTeamInfo> tea
|
||||
}
|
||||
}
|
||||
|
||||
if(serversLeft == 0) {
|
||||
logTeamEvents = true;
|
||||
}
|
||||
|
||||
// Failed server should not trigger DD if SS failures are set to be ignored
|
||||
if (!badTeam && self->healthyZone.get().present() && (self->healthyZone.get().get() == ignoreSSFailuresZoneString)) {
|
||||
ASSERT_WE_THINK(serversLeft == self->configuration.storageTeamSize);
|
||||
@ -2917,7 +2936,7 @@ ACTOR Future<Void> teamTracker(DDTeamCollection* self, Reference<TCTeamInfo> tea
|
||||
// t is the team in primary DC or the remote DC
|
||||
auto& t = j < teams.first.size() ? teams.first[j] : teams.second[j-teams.first.size()];
|
||||
if( !t.servers.size() ) {
|
||||
maxPriority = SERVER_KNOBS->PRIORITY_TEAM_0_LEFT;
|
||||
maxPriority = std::max( maxPriority, SERVER_KNOBS->PRIORITY_POPULATE_REGION );
|
||||
break;
|
||||
}
|
||||
|
||||
@ -3946,7 +3965,7 @@ ACTOR Future<Void> dataDistributionTeamCollection(
|
||||
.detail("StorageTeamSize", self->configuration.storageTeamSize)
|
||||
.detail("HighestPriority", highestPriority)
|
||||
.trackLatest(self->primary ? "TotalDataInFlight" : "TotalDataInFlightRemote");
|
||||
loggingTrigger = delay( SERVER_KNOBS->DATA_DISTRIBUTION_LOGGING_INTERVAL );
|
||||
loggingTrigger = delay( SERVER_KNOBS->DATA_DISTRIBUTION_LOGGING_INTERVAL, TaskPriority::FlushTrace );
|
||||
}
|
||||
when( wait( self->serverTrackerErrorOut.getFuture() ) ) {} // Propagate errors from storageServerTracker
|
||||
when( wait( error ) ) {}
|
||||
|
@ -134,6 +134,7 @@ public:
|
||||
|
||||
int getNumberOfShards( UID ssID );
|
||||
vector<KeyRange> getShardsFor( Team team );
|
||||
bool hasShards(Team team);
|
||||
|
||||
//The first element of the pair is either the source for non-moving shards or the destination team for in-flight shards
|
||||
//The second element of the pair is all previous sources for in-flight shards
|
||||
|
@ -1491,7 +1491,7 @@ ACTOR Future<Void> dataDistributionQueue(
|
||||
Promise<int64_t> req;
|
||||
getAverageShardBytes.send( req );
|
||||
|
||||
recordMetrics = delay(SERVER_KNOBS->DD_QUEUE_LOGGING_INTERVAL);
|
||||
recordMetrics = delay(SERVER_KNOBS->DD_QUEUE_LOGGING_INTERVAL, TaskPriority::FlushTrace);
|
||||
|
||||
int highestPriorityRelocation = 0;
|
||||
for( auto it = self.priority_relocations.begin(); it != self.priority_relocations.end(); ++it ) {
|
||||
|
@ -764,7 +764,7 @@ ACTOR Future<Void> dataDistributionTracker(
|
||||
.detail("SystemSizeBytes", self.systemSizeEstimate)
|
||||
.trackLatest( "DDTrackerStats" );
|
||||
|
||||
loggingTrigger = delay(SERVER_KNOBS->DATA_DISTRIBUTION_LOGGING_INTERVAL);
|
||||
loggingTrigger = delay(SERVER_KNOBS->DATA_DISTRIBUTION_LOGGING_INTERVAL, TaskPriority::FlushTrace);
|
||||
}
|
||||
when( GetMetricsRequest req = waitNext( getShardMetrics.getFuture() ) ) {
|
||||
self.sizeChanges.add( fetchShardMetrics( &self, req ) );
|
||||
@ -784,6 +784,11 @@ vector<KeyRange> ShardsAffectedByTeamFailure::getShardsFor( Team team ) {
|
||||
return r;
|
||||
}
|
||||
|
||||
bool ShardsAffectedByTeamFailure::hasShards(Team team) {
|
||||
auto it = team_shards.lower_bound(std::pair<Team, KeyRange>(team, KeyRangeRef()));
|
||||
return it != team_shards.end() && it->first == team;
|
||||
}
|
||||
|
||||
int ShardsAffectedByTeamFailure::getNumberOfShards( UID ssID ) {
|
||||
return storageServerShards[ssID];
|
||||
}
|
||||
|
@ -773,6 +773,10 @@ ACTOR static Future<JsonBuilderObject> processStatusFetcher(
|
||||
megabits_received.setKeyRawNumber("hz", processMetrics.getValue("MbpsReceived"));
|
||||
networkObj["megabits_received"] = megabits_received;
|
||||
|
||||
JsonBuilderObject tls_policy_failures;
|
||||
tls_policy_failures.setKeyRawNumber("hz", processMetrics.getValue("TLSPolicyFailures"));
|
||||
networkObj["tls_policy_failures"] = tls_policy_failures;
|
||||
|
||||
statusObj["network"] = networkObj;
|
||||
|
||||
memoryObj.setKeyRawNumber("used_bytes", processMetrics.getValue("Memory"));
|
||||
@ -961,8 +965,9 @@ ACTOR static Future<JsonBuilderObject> recoveryStateStatusFetcher(WorkerDetails
|
||||
state JsonBuilderObject message;
|
||||
|
||||
try {
|
||||
state Future<TraceEventFields> activeGens = timeoutError(mWorker.interf.eventLogRequest.getReply( EventLogRequest( LiteralStringRef("MasterRecoveryGenerations") ) ), 1.0);
|
||||
TraceEventFields md = wait( timeoutError(mWorker.interf.eventLogRequest.getReply( EventLogRequest( LiteralStringRef("MasterRecoveryState") ) ), 1.0) );
|
||||
state int mStatusCode = md.getInt("StatusCode");
|
||||
int mStatusCode = md.getInt("StatusCode");
|
||||
if (mStatusCode < 0 || mStatusCode >= RecoveryStatus::END)
|
||||
throw attribute_not_found();
|
||||
|
||||
@ -986,6 +991,12 @@ ACTOR static Future<JsonBuilderObject> recoveryStateStatusFetcher(WorkerDetails
|
||||
// TODO: time_in_recovery: 0.5
|
||||
// time_in_state: 0.1
|
||||
|
||||
TraceEventFields md = wait(activeGens);
|
||||
if(md.size()) {
|
||||
int activeGenerations = md.getInt("ActiveGenerations");
|
||||
message["active_generations"] = activeGenerations;
|
||||
}
|
||||
|
||||
} catch (Error &e){
|
||||
if (e.code() == error_code_actor_cancelled)
|
||||
throw;
|
||||
@ -1573,7 +1584,7 @@ static int getExtraTLogEligibleZones(const vector<WorkerDetails>& workers, const
|
||||
for(auto& region : configuration.regions) {
|
||||
int eligible = dcId_zone[region.dcId].size() - std::max(configuration.remoteTLogReplicationFactor, std::max(configuration.tLogReplicationFactor, configuration.storageTeamSize) );
|
||||
//FIXME: does not take into account fallback satellite policies
|
||||
if(region.satelliteTLogReplicationFactor > 0) {
|
||||
if(region.satelliteTLogReplicationFactor > 0 && configuration.usableRegions > 1) {
|
||||
int totalSatelliteEligible = 0;
|
||||
for(auto& sat : region.satellites) {
|
||||
totalSatelliteEligible += dcId_zone[sat.dcId].size();
|
||||
|
@ -1997,7 +1997,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
||||
oldLogSystem->recruitmentID = logSystem->recruitmentID;
|
||||
|
||||
if(configuration.usableRegions > 1) {
|
||||
logSystem->logRouterTags = recr.tLogs.size() * std::max<int>(1, configuration.desiredLogRouterCount / std::max<int>(1,recr.tLogs.size()));
|
||||
logSystem->logRouterTags = std::max(recr.satelliteTLogs.size(), recr.tLogs.size()) * std::max<int>(1, configuration.desiredLogRouterCount / std::max<int>(1,std::max(recr.satelliteTLogs.size(), recr.tLogs.size())));
|
||||
logSystem->expectedLogSets++;
|
||||
logSystem->addPseudoLocality(tagLocalityLogRouterMapped);
|
||||
}
|
||||
@ -2019,7 +2019,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
||||
needsOldTxs = needsOldTxs || it.tLogs[0]->tLogVersion < TLogVersion::V4;
|
||||
}
|
||||
|
||||
if(region.satelliteTLogReplicationFactor > 0) {
|
||||
if(region.satelliteTLogReplicationFactor > 0 && configuration.usableRegions > 1) {
|
||||
logSystem->tLogs.emplace_back(new LogSet());
|
||||
if(recr.satelliteFallback) {
|
||||
logSystem->tLogs[1]->tLogWriteAntiQuorum = region.satelliteTLogWriteAntiQuorumFallback;
|
||||
@ -2167,7 +2167,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
||||
|
||||
state std::vector<Future<Void>> recoveryComplete;
|
||||
|
||||
if(region.satelliteTLogReplicationFactor > 0) {
|
||||
if(region.satelliteTLogReplicationFactor > 0 && configuration.usableRegions > 1) {
|
||||
state vector<Future<TLogInterface>> satelliteInitializationReplies;
|
||||
vector< InitializeTLogRequest > sreqs( recr.satelliteTLogs.size() );
|
||||
std::vector<Tag> satelliteTags;
|
||||
|
@ -1161,6 +1161,10 @@ ACTOR Future<Void> trackTlogRecovery( Reference<MasterData> self, Reference<Asyn
|
||||
.detail("StatusCode", RecoveryStatus::fully_recovered)
|
||||
.detail("Status", RecoveryStatus::names[RecoveryStatus::fully_recovered])
|
||||
.trackLatest("MasterRecoveryState");
|
||||
|
||||
TraceEvent("MasterRecoveryGenerations", self->dbgid)
|
||||
.detail("ActiveGenerations", 1)
|
||||
.trackLatest("MasterRecoveryGenerations");
|
||||
} else if( !newState.oldTLogData.size() && self->recoveryState < RecoveryState::STORAGE_RECOVERED ) {
|
||||
self->recoveryState = RecoveryState::STORAGE_RECOVERED;
|
||||
TraceEvent("MasterRecoveryState", self->dbgid)
|
||||
@ -1245,10 +1249,27 @@ ACTOR Future<Void> masterCore( Reference<MasterData> self ) {
|
||||
.detail("StatusCode", RecoveryStatus::locking_coordinated_state)
|
||||
.detail("Status", RecoveryStatus::names[RecoveryStatus::locking_coordinated_state])
|
||||
.detail("TLogs", self->cstate.prevDBState.tLogs.size())
|
||||
.detail("ActiveGenerations", self->cstate.myDBState.oldTLogData.size() + 1)
|
||||
.detail("MyRecoveryCount", self->cstate.prevDBState.recoveryCount+2)
|
||||
.detail("ForceRecovery", self->forceRecovery)
|
||||
.trackLatest("MasterRecoveryState");
|
||||
|
||||
TraceEvent("MasterRecoveryGenerations", self->dbgid)
|
||||
.detail("ActiveGenerations", self->cstate.myDBState.oldTLogData.size() + 1)
|
||||
.trackLatest("MasterRecoveryGenerations");
|
||||
|
||||
if (self->cstate.myDBState.oldTLogData.size() > CLIENT_KNOBS->MAX_GENERATIONS_OVERRIDE) {
|
||||
if (self->cstate.myDBState.oldTLogData.size() >= CLIENT_KNOBS->MAX_GENERATIONS) {
|
||||
TraceEvent(SevError, "RecoveryStoppedTooManyOldGenerations").detail("OldGenerations", self->cstate.myDBState.oldTLogData.size())
|
||||
.detail("Reason", "Recovery stopped because too many recoveries have happened since the last time the cluster was fully_recovered. Set --knob_max_generations_override on your server processes to a value larger than OldGenerations to resume recovery once the underlying problem has been fixed.");
|
||||
wait(Future<Void>(Never()));
|
||||
} else if (self->cstate.myDBState.oldTLogData.size() > CLIENT_KNOBS->RECOVERY_DELAY_START_GENERATION) {
|
||||
TraceEvent(SevError, "RecoveryDelayedTooManyOldGenerations").detail("OldGenerations", self->cstate.myDBState.oldTLogData.size())
|
||||
.detail("Reason", "Recovery is delayed because too many recoveries have happened since the last time the cluster was fully_recovered. Set --knob_max_generations_override on your server processes to a value larger than OldGenerations to resume recovery once the underlying problem has been fixed.");
|
||||
wait(delay(CLIENT_KNOBS->RECOVERY_DELAY_SECONDS_PER_GENERATION*(self->cstate.myDBState.oldTLogData.size() - CLIENT_KNOBS->RECOVERY_DELAY_START_GENERATION)));
|
||||
}
|
||||
}
|
||||
|
||||
state Reference<AsyncVar<Reference<ILogSystem>>> oldLogSystems( new AsyncVar<Reference<ILogSystem>> );
|
||||
state Future<Void> recoverAndEndEpoch = ILogSystem::recoverAndEndEpoch(oldLogSystems, self->dbgid, self->cstate.prevDBState, self->myInterface.tlogRejoin.getFuture(), self->myInterface.locality, &self->forceRecovery);
|
||||
|
||||
|
@ -222,6 +222,7 @@ public:
|
||||
Int64MetricHandle countYieldCallsTrue;
|
||||
Int64MetricHandle countASIOEvents;
|
||||
Int64MetricHandle countSlowTaskSignals;
|
||||
Int64MetricHandle countTLSPolicyFailures;
|
||||
Int64MetricHandle priorityMetric;
|
||||
DoubleMetricHandle countLaunchTime;
|
||||
DoubleMetricHandle countReactTime;
|
||||
@ -914,7 +915,7 @@ ACTOR static Future<Void> watchFileForChanges( std::string filename, AsyncTrigge
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> reloadCertificatesOnChange( TLSConfig config, AsyncVar<Reference<ReferencedObject<boost::asio::ssl::context>>>* contextVar ) {
|
||||
ACTOR static Future<Void> reloadCertificatesOnChange( TLSConfig config, std::function<void()> onPolicyFailure, AsyncVar<Reference<ReferencedObject<boost::asio::ssl::context>>>* contextVar ) {
|
||||
if (FLOW_KNOBS->TLS_CERT_REFRESH_DELAY_SECONDS <= 0) {
|
||||
return Void();
|
||||
}
|
||||
@ -938,7 +939,7 @@ ACTOR static Future<Void> reloadCertificatesOnChange( TLSConfig config, AsyncVar
|
||||
try {
|
||||
LoadedTLSConfig loaded = wait( config.loadAsync() );
|
||||
boost::asio::ssl::context context(boost::asio::ssl::context::tls);
|
||||
ConfigureSSLContext(loaded, &context);
|
||||
ConfigureSSLContext(loaded, &context, onPolicyFailure);
|
||||
TraceEvent(SevInfo, "TLSCertificateRefreshSucceeded");
|
||||
mismatches = 0;
|
||||
contextVar->set(ReferencedObject<boost::asio::ssl::context>::from(std::move(context)));
|
||||
@ -961,9 +962,10 @@ void Net2::initTLS() {
|
||||
#ifndef TLS_DISABLED
|
||||
try {
|
||||
boost::asio::ssl::context newContext(boost::asio::ssl::context::tls);
|
||||
ConfigureSSLContext( tlsConfig.loadSync(), &newContext );
|
||||
auto onPolicyFailure = [this]() { this->countTLSPolicyFailures++; };
|
||||
ConfigureSSLContext( tlsConfig.loadSync(), &newContext, onPolicyFailure );
|
||||
sslContextVar.set(ReferencedObject<boost::asio::ssl::context>::from(std::move(newContext)));
|
||||
backgroundCertRefresh = reloadCertificatesOnChange( tlsConfig, &sslContextVar );
|
||||
backgroundCertRefresh = reloadCertificatesOnChange( tlsConfig, onPolicyFailure, &sslContextVar );
|
||||
} catch (Error& e) {
|
||||
TraceEvent("Net2TLSInitError").error(e);
|
||||
throw tls_error();
|
||||
@ -999,6 +1001,7 @@ void Net2::initMetrics() {
|
||||
countASIOEvents.init(LiteralStringRef("Net2.CountASIOEvents"));
|
||||
countYieldCallsTrue.init(LiteralStringRef("Net2.CountYieldCallsTrue"));
|
||||
countSlowTaskSignals.init(LiteralStringRef("Net2.CountSlowTaskSignals"));
|
||||
countTLSPolicyFailures.init(LiteralStringRef("Net2.CountTLSPolicyFailures"));
|
||||
priorityMetric.init(LiteralStringRef("Net2.Priority"));
|
||||
awakeMetric.init(LiteralStringRef("Net2.Awake"));
|
||||
slowTaskMetric.init(LiteralStringRef("Net2.SlowTask"));
|
||||
|
@ -78,7 +78,10 @@ public:
|
||||
void deserialize(FileIdentifier file_identifier, Items&... items) {
|
||||
const uint8_t* data = static_cast<ReaderImpl*>(this)->data();
|
||||
LoadContext<ReaderImpl> context(static_cast<ReaderImpl*>(this));
|
||||
ASSERT(read_file_identifier(data) == file_identifier);
|
||||
if(read_file_identifier(data) != file_identifier) {
|
||||
TraceEvent(SevError, "MismatchedFileIdentifier").detail("Expected", file_identifier).detail("Read", read_file_identifier(data));
|
||||
ASSERT(false);
|
||||
}
|
||||
load_members(data, context, items...);
|
||||
}
|
||||
|
||||
|
@ -95,6 +95,6 @@ ACTOR Future<Void> traceCounters(std::string traceEventName, UID traceEventID, d
|
||||
}
|
||||
|
||||
last_interval = now();
|
||||
wait(delay(interval));
|
||||
wait(delay(interval, TaskPriority::FlushTrace));
|
||||
}
|
||||
}
|
||||
|
@ -101,6 +101,7 @@ SystemStatistics customSystemMonitor(std::string eventName, StatisticsState *sta
|
||||
.detail("ConnectionsEstablished", (double) (netData.countConnEstablished - statState->networkState.countConnEstablished) / currentStats.elapsed)
|
||||
.detail("ConnectionsClosed", ((netData.countConnClosedWithError - statState->networkState.countConnClosedWithError) + (netData.countConnClosedWithoutError - statState->networkState.countConnClosedWithoutError)) / currentStats.elapsed)
|
||||
.detail("ConnectionErrors", (netData.countConnClosedWithError - statState->networkState.countConnClosedWithError) / currentStats.elapsed)
|
||||
.detail("TLSPolicyFailures", (netData.countTLSPolicyFailures - statState->networkState.countTLSPolicyFailures) / currentStats.elapsed)
|
||||
.trackLatest(eventName);
|
||||
|
||||
TraceEvent("MemoryMetrics")
|
||||
|
@ -80,6 +80,7 @@ struct NetworkData {
|
||||
int64_t countConnEstablished;
|
||||
int64_t countConnClosedWithError;
|
||||
int64_t countConnClosedWithoutError;
|
||||
int64_t countTLSPolicyFailures;
|
||||
double countLaunchTime;
|
||||
double countReactTime;
|
||||
|
||||
@ -107,6 +108,7 @@ struct NetworkData {
|
||||
countConnEstablished = Int64Metric::getValueOrDefault(LiteralStringRef("Net2.CountConnEstablished"));
|
||||
countConnClosedWithError = Int64Metric::getValueOrDefault(LiteralStringRef("Net2.CountConnClosedWithError"));
|
||||
countConnClosedWithoutError = Int64Metric::getValueOrDefault(LiteralStringRef("Net2.CountConnClosedWithoutError"));
|
||||
countTLSPolicyFailures = Int64Metric::getValueOrDefault(LiteralStringRef("Net2.CountTLSPolicyFailures"));
|
||||
countLaunchTime = DoubleMetric::getValueOrDefault(LiteralStringRef("Net2.CountLaunchTime"));
|
||||
countReactTime = DoubleMetric::getValueOrDefault(LiteralStringRef("Net2.CountReactTime"));
|
||||
countFileLogicalWrites = Int64Metric::getValueOrDefault(LiteralStringRef("AsyncFile.CountLogicalWrites"));
|
||||
|
@ -121,7 +121,7 @@ void LoadedTLSConfig::print(FILE* fp) {
|
||||
X509_STORE_CTX_free(store_ctx);
|
||||
}
|
||||
|
||||
void ConfigureSSLContext( const LoadedTLSConfig& loaded, boost::asio::ssl::context* context ) {
|
||||
void ConfigureSSLContext( const LoadedTLSConfig& loaded, boost::asio::ssl::context* context, std::function<void()> onPolicyFailure ) {
|
||||
try {
|
||||
context->set_options(boost::asio::ssl::context::default_workarounds);
|
||||
context->set_verify_mode(boost::asio::ssl::context::verify_peer | boost::asio::ssl::verify_fail_if_no_peer_cert);
|
||||
@ -130,8 +130,12 @@ void ConfigureSSLContext( const LoadedTLSConfig& loaded, boost::asio::ssl::conte
|
||||
Reference<TLSPolicy> tlsPolicy = Reference<TLSPolicy>(new TLSPolicy(loaded.getEndpointType()));
|
||||
tlsPolicy->set_verify_peers({ loaded.getVerifyPeers() });
|
||||
|
||||
context->set_verify_callback([policy=tlsPolicy](bool preverified, boost::asio::ssl::verify_context& ctx) {
|
||||
return policy->verify_peer(preverified, ctx.native_handle());
|
||||
context->set_verify_callback([policy=tlsPolicy, onPolicyFailure](bool preverified, boost::asio::ssl::verify_context& ctx) {
|
||||
bool success = policy->verify_peer(preverified, ctx.native_handle());
|
||||
if (!success) {
|
||||
onPolicyFailure();
|
||||
}
|
||||
return success;
|
||||
});
|
||||
} else {
|
||||
// Insecurely always except if TLS is not enabled.
|
||||
|
@ -222,7 +222,7 @@ PRIVATE_EXCEPT_FOR_TLSCONFIG_CPP:
|
||||
|
||||
#ifndef TLS_DISABLED
|
||||
namespace boost { namespace asio { namespace ssl { struct context; }}}
|
||||
void ConfigureSSLContext(const LoadedTLSConfig& loaded, boost::asio::ssl::context* context);
|
||||
void ConfigureSSLContext(const LoadedTLSConfig& loaded, boost::asio::ssl::context* context, std::function<void()> onPolicyFailure);
|
||||
#endif
|
||||
|
||||
class TLSPolicy : ReferenceCounted<TLSPolicy> {
|
||||
|
Loading…
x
Reference in New Issue
Block a user