Merge remote-tracking branch 'upstream/release-6.2' into fdbcli-tlsinfo

This commit is contained in:
Alex Miller 2020-03-16 12:33:50 -07:00
commit db5863145a
22 changed files with 123 additions and 37 deletions

View File

@ -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":{

View File

@ -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++;
}

View File

@ -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 );

View File

@ -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;

View File

@ -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)

View File

@ -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){

View File

@ -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":{

View File

@ -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;

View File

@ -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 ) ) {}

View File

@ -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

View File

@ -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 ) {

View File

@ -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];
}

View File

@ -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();

View File

@ -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;

View File

@ -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);

View File

@ -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"));

View File

@ -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...);
}

View File

@ -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));
}
}

View File

@ -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")

View File

@ -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"));

View File

@ -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.

View File

@ -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> {