Merge branch 'release-6.2' into release-6.3

This commit is contained in:
David Youngworth 2020-11-16 10:15:23 -08:00
commit d0391db862
34 changed files with 770 additions and 239 deletions

3
.gitignore vendored
View File

@ -95,3 +95,6 @@ flow/coveragetool/obj
.DS_Store
temp/
/versions.target
/compile_commands.json
/.ccls-cache
.clangd/

View File

@ -180,6 +180,10 @@ std::string describe( Reference<T> const& item ) {
return item->toString();
}
static std::string describe(UID const& item) {
return item.shortString();
}
template <class T>
std::string describe( T const& item ) {
return item.toString();

View File

@ -260,18 +260,36 @@ ACTOR Future<Void> pingLatencyLogger(TransportData* self) {
if(!peer) {
TraceEvent(SevWarnAlways, "MissingNetworkAddress").suppressFor(10.0).detail("PeerAddr", lastAddress);
}
if (peer->lastLoggedTime <= 0.0) {
peer->lastLoggedTime = peer->lastConnectTime;
}
if(peer && peer->pingLatencies.getPopulationSize() >= 10) {
TraceEvent("PingLatency")
.detail("PeerAddr", lastAddress)
.detail("MinLatency", peer->pingLatencies.min())
.detail("MaxLatency", peer->pingLatencies.max())
.detail("MeanLatency", peer->pingLatencies.mean())
.detail("MedianLatency", peer->pingLatencies.median())
.detail("P90Latency", peer->pingLatencies.percentile(0.90))
.detail("Count", peer->pingLatencies.getPopulationSize())
.detail("BytesReceived", peer->bytesReceived - peer->lastLoggedBytesReceived)
.detail("BytesSent", peer->bytesSent - peer->lastLoggedBytesSent);
.detail("Elapsed", now() - peer->lastLoggedTime)
.detail("PeerAddr", lastAddress)
.detail("MinLatency", peer->pingLatencies.min())
.detail("MaxLatency", peer->pingLatencies.max())
.detail("MeanLatency", peer->pingLatencies.mean())
.detail("MedianLatency", peer->pingLatencies.median())
.detail("P90Latency", peer->pingLatencies.percentile(0.90))
.detail("Count", peer->pingLatencies.getPopulationSize())
.detail("BytesReceived", peer->bytesReceived - peer->lastLoggedBytesReceived)
.detail("BytesSent", peer->bytesSent - peer->lastLoggedBytesSent)
.detail("ConnectOutgoingCount", peer->connectOutgoingCount)
.detail("ConnectIncomingCount", peer->connectIncomingCount)
.detail("ConnectFailedCount", peer->connectFailedCount)
.detail("ConnectMinLatency", peer->connectLatencies.min())
.detail("ConnectMaxLatency", peer->connectLatencies.max())
.detail("ConnectMeanLatency", peer->connectLatencies.mean())
.detail("ConnectMedianLatency", peer->connectLatencies.median())
.detail("ConnectP90Latency", peer->connectLatencies.percentile(0.90));
peer->lastLoggedTime = now();
peer->connectOutgoingCount = 0;
peer->connectIncomingCount = 0;
peer->connectFailedCount = 0;
peer->pingLatencies.clear();
peer->connectLatencies.clear();
peer->lastLoggedBytesReceived = peer->bytesReceived;
peer->lastLoggedBytesSent = peer->bytesSent;
wait(delay(FLOW_KNOBS->PING_LOGGING_INTERVAL));
@ -567,6 +585,7 @@ ACTOR Future<Void> connectionKeeper( Reference<Peer> self,
.detail("FailureStatus", IFailureMonitor::failureMonitor().getState(self->destination).isAvailable()
? "OK"
: "FAILED");
++self->connectOutgoingCount;
try {
choose {
@ -574,6 +593,10 @@ ACTOR Future<Void> connectionKeeper( Reference<Peer> self,
wait(INetworkConnections::net()->connect(self->destination))) {
conn = _conn;
wait(conn->connectHandshake());
self->connectLatencies.addSample(now() - self->lastConnectTime);
if (FlowTransport::isClient()) {
IFailureMonitor::failureMonitor().setStatus(self->destination, FailureStatus(false));
}
if (self->unsent.empty()) {
delayedHealthUpdateF = delayedHealthUpdate(self->destination);
choose {
@ -597,8 +620,9 @@ ACTOR Future<Void> connectionKeeper( Reference<Peer> self,
throw connection_failed();
}
}
} catch (Error& e) {
if (e.code() != error_code_connection_failed) {
} catch(Error &e) {
++self->connectFailedCount;
if(e.code() != error_code_connection_failed) {
throw;
}
TraceEvent("ConnectionTimedOut", conn ? conn->getDebugID() : UID())
@ -728,7 +752,8 @@ Peer::Peer(TransportData* transport, NetworkAddress const& destination)
reconnectionDelay(FLOW_KNOBS->INITIAL_RECONNECTION_TIME), compatible(true), outstandingReplies(0),
incompatibleProtocolVersionNewer(false), peerReferences(-1), bytesReceived(0), lastDataPacketSentTime(now()),
pingLatencies(destination.isPublic() ? FLOW_KNOBS->PING_SAMPLE_AMOUNT : 1), lastLoggedBytesReceived(0),
bytesSent(0), lastLoggedBytesSent(0) {
bytesSent(0), lastLoggedBytesSent(0), lastLoggedTime(0.0), connectOutgoingCount(0), connectIncomingCount(0),
connectFailedCount(0), connectLatencies(destination.isPublic() ? FLOW_KNOBS->NETWORK_CONNECT_SAMPLE_AMOUNT : 1) {
IFailureMonitor::failureMonitor().setStatus(destination, FailureStatus(false));
}
@ -779,6 +804,7 @@ void Peer::discardUnreliablePackets() {
void Peer::onIncomingConnection( Reference<Peer> self, Reference<IConnection> conn, Future<Void> reader ) {
// In case two processes are trying to connect to each other simultaneously, the process with the larger canonical NetworkAddress
// gets to keep its outgoing connection.
++self->connectIncomingCount;
if ( !destination.isPublic() && !outgoingConnectionIdle ) throw address_in_use();
NetworkAddress compatibleAddr = transport->localAddresses.address;
if(transport->localAddresses.secondaryAddress.present() && transport->localAddresses.secondaryAddress.get().isTLS() == destination.isTLS()) {

View File

@ -145,8 +145,14 @@ struct Peer : public ReferenceCounted<Peer> {
double lastDataPacketSentTime;
int outstandingReplies;
ContinuousSample<double> pingLatencies;
double lastLoggedTime;
int64_t lastLoggedBytesReceived;
int64_t lastLoggedBytesSent;
// Cleared every time stats are logged for this peer.
int connectOutgoingCount;
int connectIncomingCount;
int connectFailedCount;
ContinuousSample<double> connectLatencies;
explicit Peer(TransportData* transport, NetworkAddress const& destination);

View File

@ -87,7 +87,9 @@ void CounterCollection::logToTraceEvent(TraceEvent &te) const {
}
}
ACTOR Future<Void> traceCounters(std::string traceEventName, UID traceEventID, double interval, CounterCollection* counters, std::string trackLatestName) {
ACTOR Future<Void> traceCounters(std::string traceEventName, UID traceEventID, double interval,
CounterCollection* counters, std::string trackLatestName,
std::function<void(TraceEvent&)> decorator) {
wait(delay(0)); // Give an opportunity for all members used in special counters to be initialized
for (ICounter* c : counters->counters)
@ -100,6 +102,7 @@ ACTOR Future<Void> traceCounters(std::string traceEventName, UID traceEventID, d
te.detail("Elapsed", now() - last_interval);
counters->logToTraceEvent(te);
decorator(te);
if (!trackLatestName.empty()) {
te.trackLatest(trackLatestName);

View File

@ -146,7 +146,9 @@ struct SpecialCounter : ICounter, FastAllocated<SpecialCounter<F>>, NonCopyable
template <class F>
static void specialCounter(CounterCollection& collection, std::string const& name, F && f) { new SpecialCounter<F>(collection, name, std::move(f)); }
Future<Void> traceCounters(std::string const& traceEventName, UID const& traceEventID, double const& interval, CounterCollection* const& counters, std::string const& trackLatestName = std::string());
Future<Void> traceCounters(std::string const& traceEventName, UID const& traceEventID, double const& interval,
CounterCollection* const& counters, std::string const& trackLatestName = std::string(),
std::function<void(TraceEvent&)> const& decorator = [](TraceEvent& te) {});
class LatencyBands {
public:

View File

@ -87,8 +87,6 @@ void ISimulator::displayWorkers() const
const UID TOKEN_ENDPOINT_NOT_FOUND(-1, -1);
ISimulator* g_pSimulator = 0;
thread_local ISimulator::ProcessInfo* ISimulator::currentProcess = 0;
int openCount = 0;
struct SimClogging {

View File

@ -23,6 +23,7 @@
#pragma once
#include "flow/flow.h"
#include "flow/Histogram.h"
#include "fdbrpc/FailureMonitor.h"
#include "fdbrpc/Locality.h"
#include "fdbrpc/IAsyncFile.h"
@ -54,6 +55,7 @@ public:
LocalityData locality;
ProcessClass startingClass;
TDMetricCollection tdmetrics;
HistogramRegistry histograms;
std::map<NetworkAddress, Reference<IListener>> listenerMap;
bool failed;
bool excluded;

View File

@ -122,8 +122,10 @@ public:
vector<Reference<TCMachineInfo>> machines;
vector<Standalone<StringRef>> machineIDs;
vector<Reference<TCTeamInfo>> serverTeams;
UID id;
explicit TCMachineTeamInfo(vector<Reference<TCMachineInfo>> const& machines) : machines(machines) {
explicit TCMachineTeamInfo(vector<Reference<TCMachineInfo>> const& machines)
: machines(machines), id(deterministicRandom()->randomUniqueID()) {
machineIDs.reserve(machines.size());
for (int i = 0; i < machines.size(); i++) {
machineIDs.push_back(machines[i]->machineID);
@ -158,13 +160,15 @@ class TCTeamInfo : public ReferenceCounted<TCTeamInfo>, public IDataDistribution
bool healthy;
bool wrongConfiguration; //True if any of the servers in the team have the wrong configuration
int priority;
UID id;
public:
Reference<TCMachineTeamInfo> machineTeam;
Future<Void> tracker;
explicit TCTeamInfo(vector<Reference<TCServerInfo>> const& servers)
: servers(servers), healthy(true), priority(SERVER_KNOBS->PRIORITY_TEAM_HEALTHY), wrongConfiguration(false) {
: servers(servers), healthy(true), priority(SERVER_KNOBS->PRIORITY_TEAM_HEALTHY), wrongConfiguration(false),
id(deterministicRandom()->randomUniqueID()) {
if (servers.empty()) {
TraceEvent(SevInfo, "ConstructTCTeamFromEmptyServers");
}
@ -174,6 +178,8 @@ public:
}
}
std::string getTeamID() override { return id.shortString(); }
vector<StorageServerInterface> getLastKnownServerInterfaces() const override {
vector<StorageServerInterface> v;
v.reserve(servers.size());
@ -616,6 +622,7 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
int highestUtilizationTeam;
AsyncTrigger printDetailedTeamsInfo;
PromiseStream<GetMetricsRequest> getShardMetrics;
void resetLocalitySet() {
storageServerSet = Reference<LocalitySet>(new LocalityMap<UID>());
@ -647,7 +654,7 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
DatabaseConfiguration configuration, std::vector<Optional<Key>> includedDCs,
Optional<std::vector<Optional<Key>>> otherTrackedDCs, Future<Void> readyToStart,
Reference<AsyncVar<bool>> zeroHealthyTeams, bool primary,
Reference<AsyncVar<bool>> processingUnhealthy)
Reference<AsyncVar<bool>> processingUnhealthy, PromiseStream<GetMetricsRequest> getShardMetrics)
: cx(cx), distributorId(distributorId), lock(lock), output(output),
shardsAffectedByTeamFailure(shardsAffectedByTeamFailure), doBuildTeams(true), lastBuildTeamsFailed(false),
teamBuilder(Void()), badTeamRemover(Void()), checkInvalidLocalities(Void()), wrongStoreTypeRemover(Void()), configuration(configuration),
@ -659,8 +666,10 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
initializationDoneActor(logOnCompletion(readyToStart && initialFailureReactionDelay, this)),
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), lowestUtilizationTeam(0), highestUtilizationTeam(0) {
zeroHealthyTeams(zeroHealthyTeams), zeroOptimalTeams(true), primary(primary),
medianAvailableSpace(SERVER_KNOBS->MIN_AVAILABLE_SPACE_RATIO), lastMedianAvailableSpaceUpdate(0),
processingUnhealthy(processingUnhealthy), lowestUtilizationTeam(0), highestUtilizationTeam(0),
getShardMetrics(getShardMetrics) {
if(!primary || configuration.usableRegions == 1) {
TraceEvent("DDTrackerStarting", distributorId)
.detail( "State", "Inactive" )
@ -1395,7 +1404,8 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
.detail("TeamIndex", i++)
.detail("Healthy", team->isHealthy())
.detail("TeamSize", team->size())
.detail("MemberIDs", team->getServerIDsStr());
.detail("MemberIDs", team->getServerIDsStr())
.detail("TeamID", team->getTeamID());
}
}
@ -2151,7 +2161,7 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
.detail("Primary", primary)
.detail("AddedTeams", 0)
.detail("TeamsToBuild", 0)
.detail("CurrentTeams", teams.size())
.detail("CurrentServerTeams", teams.size())
.detail("DesiredTeams", desiredServerTeams)
.detail("MaxTeams", maxServerTeams)
.detail("StorageTeamSize", configuration.storageTeamSize)
@ -2200,11 +2210,11 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
}
}
uniqueMachines = machines.size();
TraceEvent("BuildTeams")
.detail("ServerCount", self->server_info.size())
.detail("UniqueMachines", uniqueMachines)
.detail("Primary", self->primary)
.detail("StorageTeamSize", self->configuration.storageTeamSize);
TraceEvent("BuildTeams", self->distributorId)
.detail("ServerCount", self->server_info.size())
.detail("UniqueMachines", uniqueMachines)
.detail("Primary", self->primary)
.detail("StorageTeamSize", self->configuration.storageTeamSize);
// If there are too few machines to even build teams or there are too few represented datacenters, build no new teams
if( uniqueMachines >= self->configuration.storageTeamSize ) {
@ -2231,11 +2241,11 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
.detail("TeamsToBuild", teamsToBuild)
.detail("DesiredTeams", desiredTeams)
.detail("MaxTeams", maxTeams)
.detail("BadTeams", self->badTeams.size())
.detail("BadServerTeams", self->badTeams.size())
.detail("UniqueMachines", uniqueMachines)
.detail("TeamSize", self->configuration.storageTeamSize)
.detail("Servers", serverCount)
.detail("CurrentTrackedTeams", self->teams.size())
.detail("CurrentTrackedServerTeams", self->teams.size())
.detail("HealthyTeamCount", teamCount)
.detail("TotalTeamCount", totalTeamCount)
.detail("MachineTeamCount", self->machineTeams.size())
@ -2252,9 +2262,9 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
int addedTeams = self->addTeamsBestOf(teamsToBuild, desiredTeams, maxTeams);
if (addedTeams <= 0 && self->teams.size() == 0) {
TraceEvent(SevWarn, "NoTeamAfterBuildTeam")
.detail("TeamNum", self->teams.size())
.detail("Debug", "Check information below");
TraceEvent(SevWarn, "NoTeamAfterBuildTeam", self->distributorId)
.detail("ServerTeamNum", self->teams.size())
.detail("Debug", "Check information below");
// Debug: set true for traceAllInfo() to print out more information
self->traceAllInfo();
}
@ -2272,7 +2282,7 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
.detail("Primary", self->primary)
.detail("AddedTeams", 0)
.detail("TeamsToBuild", teamsToBuild)
.detail("CurrentTeams", self->teams.size())
.detail("CurrentServerTeams", self->teams.size())
.detail("DesiredTeams", desiredTeams)
.detail("MaxTeams", maxTeams)
.detail("StorageTeamSize", self->configuration.storageTeamSize)
@ -2313,9 +2323,9 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
}
TraceEvent(SevWarn, "NoHealthyTeams", distributorId)
.detail("CurrentTeamCount", teams.size())
.detail("ServerCount", server_info.size())
.detail("NonFailedServerCount", desiredServerSet.size());
.detail("CurrentServerTeamCount", teams.size())
.detail("ServerCount", server_info.size())
.detail("NonFailedServerCount", desiredServerSet.size());
}
bool shouldHandleServer(const StorageServerInterface &newServer) {
@ -2343,7 +2353,7 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
}
bool removeTeam( Reference<TCTeamInfo> team ) {
TraceEvent("RemovedTeam", distributorId).detail("Team", team->getDesc());
TraceEvent("RemovedServerTeam", distributorId).detail("Team", team->getDesc());
bool found = false;
for(int t=0; t<teams.size(); t++) {
if( teams[t] == team ) {
@ -2537,9 +2547,10 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
int removedCount = 0;
for (int t = 0; t < teams.size(); t++) {
if ( std::count( teams[t]->getServerIDs().begin(), teams[t]->getServerIDs().end(), removedServer ) ) {
TraceEvent("TeamRemoved")
TraceEvent("ServerTeamRemoved")
.detail("Primary", primary)
.detail("TeamServerIDs", teams[t]->getServerIDsStr());
.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--;
@ -2612,8 +2623,8 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
restartTeamBuilder.trigger();
TraceEvent("DataDistributionTeamCollectionUpdate", distributorId)
.detail("Teams", teams.size())
.detail("BadTeams", badTeams.size())
.detail("ServerTeams", teams.size())
.detail("BadServerTeams", badTeams.size())
.detail("Servers", allServers.size())
.detail("Machines", machine_info.size())
.detail("MachineTeams", machineTeams.size())
@ -2911,7 +2922,7 @@ ACTOR Future<Void> removeBadTeams(DDTeamCollection* self) {
wait(self->initialFailureReactionDelay);
wait(waitUntilHealthy(self));
wait(self->addSubsetComplete.getFuture());
TraceEvent("DDRemovingBadTeams", self->distributorId).detail("Primary", self->primary);
TraceEvent("DDRemovingBadServerTeams", self->distributorId).detail("Primary", self->primary);
for(auto it : self->badTeams) {
it->tracker.cancel();
}
@ -3025,9 +3036,9 @@ ACTOR Future<Void> machineTeamRemover(DDTeamCollection* self) {
// Check if a server will have 0 team after the team is removed
for (auto& s : team->getServers()) {
if (s->teams.size() == 0) {
TraceEvent(SevError, "TeamRemoverTooAggressive")
TraceEvent(SevError, "MachineTeamRemoverTooAggressive", self->distributorId)
.detail("Server", s->id)
.detail("Team", team->getServerIDsStr());
.detail("ServerTeam", team->getDesc());
self->traceAllInfo(true);
}
}
@ -3050,6 +3061,7 @@ ACTOR Future<Void> machineTeamRemover(DDTeamCollection* self) {
}
TraceEvent("MachineTeamRemover", self->distributorId)
.detail("MachineTeamIDToRemove", mt->id.shortString())
.detail("MachineTeamToRemove", mt->getMachineIDsStr())
.detail("NumProcessTeamsOnTheMachineTeam", minNumProcessTeams)
.detail("CurrentMachineTeams", self->machineTeams.size())
@ -3065,7 +3077,7 @@ ACTOR Future<Void> machineTeamRemover(DDTeamCollection* self) {
} else {
if (numMachineTeamRemoved > 0) {
// Only trace the information when we remove a machine team
TraceEvent("TeamRemoverDone")
TraceEvent("MachineTeamRemoverDone", self->distributorId)
.detail("HealthyMachines", healthyMachineCount)
// .detail("CurrentHealthyMachineTeams", currentHealthyMTCount)
.detail("CurrentMachineTeams", self->machineTeams.size())
@ -3129,6 +3141,7 @@ ACTOR Future<Void> serverTeamRemover(DDTeamCollection* self) {
TraceEvent("ServerTeamRemover", self->distributorId)
.detail("ServerTeamToRemove", st->getServerIDsStr())
.detail("ServerTeamID", st->getTeamID())
.detail("NumProcessTeamsOnTheServerTeam", maxNumProcessTeams)
.detail("CurrentServerTeams", self->teams.size())
.detail("DesiredServerTeams", desiredServerTeams);
@ -3148,25 +3161,33 @@ ACTOR Future<Void> serverTeamRemover(DDTeamCollection* self) {
}
}
bool teamContainsFailedServer(DDTeamCollection* self, Reference<TCTeamInfo> team) {
auto ssis = team->getLastKnownServerInterfaces();
for (const auto &ssi : ssis) {
AddressExclusion addr(ssi.address().ip, ssi.address().port);
AddressExclusion ipaddr(ssi.address().ip);
if (self->excludedServers.get(addr) == DDTeamCollection::Status::FAILED ||
self->excludedServers.get(ipaddr) == DDTeamCollection::Status::FAILED) {
return true;
}
if(ssi.secondaryAddress().present()) {
AddressExclusion saddr(ssi.secondaryAddress().get().ip, ssi.secondaryAddress().get().port);
AddressExclusion sipaddr(ssi.secondaryAddress().get().ip);
if (self->excludedServers.get(saddr) == DDTeamCollection::Status::FAILED ||
self->excludedServers.get(sipaddr) == DDTeamCollection::Status::FAILED) {
return true;
}
}
ACTOR Future<Void> zeroServerLeftLogger_impl(DDTeamCollection* self, Reference<TCTeamInfo> team) {
wait(delay(SERVER_KNOBS->DD_TEAM_ZERO_SERVER_LEFT_LOG_DELAY));
state vector<KeyRange> shards = self->shardsAffectedByTeamFailure->getShardsFor(
ShardsAffectedByTeamFailure::Team(team->getServerIDs(), self->primary));
state std::vector<Future<StorageMetrics>> sizes;
sizes.reserve(shards.size());
for (auto const& shard : shards) {
sizes.emplace_back(brokenPromiseToNever(self->getShardMetrics.getReply(GetMetricsRequest(shard))));
TraceEvent(SevWarnAlways, "DDShardLost", self->distributorId)
.detail("ServerTeamID", team->getTeamID())
.detail("ShardBegin", shard.begin)
.detail("ShardEnd", shard.end);
}
return false;
wait(waitForAll(sizes));
int64_t bytesLost = 0;
for (auto const& size : sizes) {
bytesLost += size.get().bytes;
}
TraceEvent(SevWarnAlways, "DDZeroServerLeftInTeam", self->distributorId)
.detail("Team", team->getDesc())
.detail("TotalBytesLost", bytesLost);
return Void();
}
// Track a team and issue RelocateShards when the level of degradation changes
@ -3183,18 +3204,22 @@ ACTOR Future<Void> teamTracker(DDTeamCollection* self, Reference<TCTeamInfo> tea
state bool lastZeroHealthy = self->zeroHealthyTeams->get();
state bool firstCheck = true;
state Future<Void> zeroServerLeftLogger;
if(logTeamEvents) {
TraceEvent("TeamTrackerStarting", self->distributorId).detail("Reason", "Initial wait complete (sc)").detail("Team", team->getDesc());
TraceEvent("ServerTeamTrackerStarting", self->distributorId)
.detail("Reason", "Initial wait complete (sc)")
.detail("ServerTeam", team->getDesc());
}
self->priority_teams[team->getPriority()]++;
try {
loop {
if(logTeamEvents) {
TraceEvent("TeamHealthChangeDetected", self->distributorId)
.detail("Team", team->getDesc())
.detail("Primary", self->primary)
.detail("IsReady", self->initialFailureReactionDelay.isReady());
TraceEvent("ServerTeamHealthChangeDetected", self->distributorId)
.detail("ServerTeam", team->getDesc())
.detail("Primary", self->primary)
.detail("IsReady", self->initialFailureReactionDelay.isReady());
self->traceTeamCollectionInfo();
}
// Check if the number of degraded machines has changed
@ -3270,10 +3295,13 @@ ACTOR Future<Void> teamTracker(DDTeamCollection* self, Reference<TCTeamInfo> tea
if (serversLeft != lastServersLeft || anyUndesired != lastAnyUndesired ||
anyWrongConfiguration != lastWrongConfiguration || recheck) { // NOTE: do not check wrongSize
if(logTeamEvents) {
TraceEvent("TeamHealthChanged", self->distributorId)
.detail("Team", team->getDesc()).detail("ServersLeft", serversLeft)
.detail("LastServersLeft", lastServersLeft).detail("ContainsUndesiredServer", anyUndesired)
.detail("HealthyTeamsCount", self->healthyTeamCount).detail("IsWrongConfiguration", anyWrongConfiguration);
TraceEvent("ServerTeamHealthChanged", self->distributorId)
.detail("ServerTeam", team->getDesc())
.detail("ServersLeft", serversLeft)
.detail("LastServersLeft", lastServersLeft)
.detail("ContainsUndesiredServer", anyUndesired)
.detail("HealthyTeamsCount", self->healthyTeamCount)
.detail("IsWrongConfiguration", anyWrongConfiguration);
}
team->setWrongConfiguration( anyWrongConfiguration );
@ -3295,18 +3323,18 @@ ACTOR Future<Void> teamTracker(DDTeamCollection* self, Reference<TCTeamInfo> tea
self->zeroHealthyTeams->set(self->healthyTeamCount == 0);
if( self->healthyTeamCount == 0 ) {
TraceEvent(SevWarn, "ZeroTeamsHealthySignalling", self->distributorId)
.detail("SignallingTeam", team->getDesc())
.detail("Primary", self->primary);
TraceEvent(SevWarn, "ZeroServerTeamsHealthySignalling", self->distributorId)
.detail("SignallingTeam", team->getDesc())
.detail("Primary", self->primary);
}
if(logTeamEvents) {
TraceEvent("TeamHealthDifference", self->distributorId)
.detail("Team", team->getDesc())
.detail("LastOptimal", lastOptimal)
.detail("LastHealthy", lastHealthy)
.detail("Optimal", optimal)
.detail("OptimalTeamCount", self->optimalTeamCount);
TraceEvent("ServerTeamHealthDifference", self->distributorId)
.detail("ServerTeam", team->getDesc())
.detail("LastOptimal", lastOptimal)
.detail("LastHealthy", lastHealthy)
.detail("Optimal", optimal)
.detail("OptimalTeamCount", self->optimalTeamCount);
}
}
@ -3343,12 +3371,24 @@ ACTOR Future<Void> teamTracker(DDTeamCollection* self, Reference<TCTeamInfo> tea
if(lastPriority != team->getPriority()) {
self->priority_teams[lastPriority]--;
self->priority_teams[team->getPriority()]++;
if (lastPriority == SERVER_KNOBS->PRIORITY_TEAM_0_LEFT &&
team->getPriority() < SERVER_KNOBS->PRIORITY_TEAM_0_LEFT) {
zeroServerLeftLogger = Void();
}
if (logTeamEvents) {
int dataLoss = team->getPriority() == SERVER_KNOBS->PRIORITY_TEAM_0_LEFT;
Severity severity = dataLoss ? SevWarnAlways : SevInfo;
TraceEvent(severity, "ServerTeamPriorityChange", self->distributorId)
.detail("Priority", team->getPriority())
.detail("Info", team->getDesc())
.detail("ZeroHealthyServerTeams", self->zeroHealthyTeams->get());
if (team->getPriority() == SERVER_KNOBS->PRIORITY_TEAM_0_LEFT) {
// 0 servers left in this team, data might be lost.
zeroServerLeftLogger = zeroServerLeftLogger_impl(self, team);
}
}
}
if(logTeamEvents) {
TraceEvent("TeamPriorityChange", self->distributorId).detail("Priority", team->getPriority())
.detail("Info", team->getDesc()).detail("ZeroHealthyTeams", self->zeroHealthyTeams->get());
}
lastZeroHealthy = self->zeroHealthyTeams->get(); //set this again in case it changed from this teams health changing
if ((self->initialFailureReactionDelay.isReady() && !self->zeroHealthyTeams->get()) || containsFailed) {
@ -3417,17 +3457,19 @@ ACTOR Future<Void> teamTracker(DDTeamCollection* self, Reference<TCTeamInfo> tea
self->output.send(rs);
TraceEvent("SendRelocateToDDQueue", self->distributorId)
.suppressFor(1.0)
.detail("Primary", self->primary)
.detail("Team", team->getDesc())
.detail("ServerPrimary", self->primary)
.detail("ServerTeam", team->getDesc())
.detail("KeyBegin", rs.keys.begin)
.detail("KeyEnd", rs.keys.end)
.detail("Priority", rs.priority)
.detail("TeamFailedMachines", team->size() - serversLeft)
.detail("TeamOKMachines", serversLeft);
.detail("ServerTeamFailedMachines", team->size() - serversLeft)
.detail("ServerTeamOKMachines", serversLeft);
}
} else {
if(logTeamEvents) {
TraceEvent("TeamHealthNotReady", self->distributorId).detail("HealthyTeamCount", self->healthyTeamCount);
TraceEvent("ServerTeamHealthNotReady", self->distributorId)
.detail("HealthyServerTeamCount", self->healthyTeamCount)
.detail("ServerTeamID", team->getTeamID());
}
}
}
@ -3439,7 +3481,7 @@ ACTOR Future<Void> teamTracker(DDTeamCollection* self, Reference<TCTeamInfo> tea
} catch(Error& e) {
if(logTeamEvents) {
TraceEvent("TeamTrackerStopping", self->distributorId)
.detail("Primary", self->primary)
.detail("ServerPrimary", self->primary)
.detail("Team", team->getDesc())
.detail("Priority", team->getPriority());
}
@ -3450,8 +3492,8 @@ ACTOR Future<Void> teamTracker(DDTeamCollection* self, Reference<TCTeamInfo> tea
if( self->healthyTeamCount == 0 ) {
TraceEvent(SevWarn, "ZeroTeamsHealthySignalling", self->distributorId)
.detail("Primary", self->primary)
.detail("SignallingTeam", team->getDesc());
.detail("ServerPrimary", self->primary)
.detail("SignallingServerTeam", team->getDesc());
self->zeroHealthyTeams->set(true);
}
}
@ -4889,10 +4931,16 @@ ACTOR Future<Void> dataDistribution(Reference<DataDistributorData> self, Promise
"DDQueue", self->ddId, &normalDDQueueErrors()));
vector<DDTeamCollection*> teamCollectionsPtrs;
primaryTeamCollection = Reference<DDTeamCollection>( new DDTeamCollection(cx, self->ddId, lock, output, shardsAffectedByTeamFailure, configuration, primaryDcId, configuration.usableRegions > 1 ? remoteDcIds : std::vector<Optional<Key>>(), readyToStart.getFuture(), zeroHealthyTeams[0], true, processingUnhealthy) );
primaryTeamCollection = Reference<DDTeamCollection>(new DDTeamCollection(
cx, self->ddId, lock, output, shardsAffectedByTeamFailure, configuration, primaryDcId,
configuration.usableRegions > 1 ? remoteDcIds : std::vector<Optional<Key>>(), readyToStart.getFuture(),
zeroHealthyTeams[0], true, processingUnhealthy, getShardMetrics));
teamCollectionsPtrs.push_back(primaryTeamCollection.getPtr());
if (configuration.usableRegions > 1) {
remoteTeamCollection = Reference<DDTeamCollection>( new DDTeamCollection(cx, self->ddId, lock, output, shardsAffectedByTeamFailure, configuration, remoteDcIds, Optional<std::vector<Optional<Key>>>(), readyToStart.getFuture() && remoteRecovered(self->dbInfo), zeroHealthyTeams[1], false, processingUnhealthy) );
remoteTeamCollection = Reference<DDTeamCollection>(new DDTeamCollection(
cx, self->ddId, lock, output, shardsAffectedByTeamFailure, configuration, remoteDcIds,
Optional<std::vector<Optional<Key>>>(), readyToStart.getFuture() && remoteRecovered(self->dbInfo),
zeroHealthyTeams[1], false, processingUnhealthy, getShardMetrics));
teamCollectionsPtrs.push_back(remoteTeamCollection.getPtr());
remoteTeamCollection->teamCollections = teamCollectionsPtrs;
actors.push_back( reportErrorsExcept( dataDistributionTeamCollection( remoteTeamCollection, initData, tcis[1], self->dbInfo ), "DDTeamCollectionSecondary", self->ddId, &normalDDQueueErrors() ) );
@ -5209,20 +5257,11 @@ DDTeamCollection* testTeamCollection(int teamSize, Reference<IReplicationPolicy>
conf.storageTeamSize = teamSize;
conf.storagePolicy = policy;
DDTeamCollection* collection = new DDTeamCollection(
database,
UID(0, 0),
MoveKeysLock(),
PromiseStream<RelocateShard>(),
Reference<ShardsAffectedByTeamFailure>(new ShardsAffectedByTeamFailure()),
conf,
{},
{},
Future<Void>(Void()),
Reference<AsyncVar<bool>>( new AsyncVar<bool>(true) ),
true,
Reference<AsyncVar<bool>>( new AsyncVar<bool>(false) )
);
DDTeamCollection* collection =
new DDTeamCollection(database, UID(0, 0), MoveKeysLock(), PromiseStream<RelocateShard>(),
Reference<ShardsAffectedByTeamFailure>(new ShardsAffectedByTeamFailure()), conf, {}, {},
Future<Void>(Void()), Reference<AsyncVar<bool>>(new AsyncVar<bool>(true)), true,
Reference<AsyncVar<bool>>(new AsyncVar<bool>(false)), PromiseStream<GetMetricsRequest>());
for (int id = 1; id <= processCount; ++id) {
UID uid(id, 0);
@ -5250,9 +5289,8 @@ DDTeamCollection* testMachineTeamCollection(int teamSize, Reference<IReplication
DDTeamCollection* collection =
new DDTeamCollection(database, UID(0, 0), MoveKeysLock(), PromiseStream<RelocateShard>(),
Reference<ShardsAffectedByTeamFailure>(new ShardsAffectedByTeamFailure()), conf, {}, {},
Future<Void>(Void()),
Reference<AsyncVar<bool>>(new AsyncVar<bool>(true)), true,
Reference<AsyncVar<bool>>(new AsyncVar<bool>(false)));
Future<Void>(Void()), Reference<AsyncVar<bool>>(new AsyncVar<bool>(true)), true,
Reference<AsyncVar<bool>>(new AsyncVar<bool>(false)), PromiseStream<GetMetricsRequest>());
for (int id = 1; id <= processCount; id++) {
UID uid(id, 0);

View File

@ -58,10 +58,12 @@ struct IDataDistributionTeam {
virtual bool isWrongConfiguration() const = 0;
virtual void setWrongConfiguration(bool) = 0;
virtual void addServers(const vector<UID> &servers) = 0;
virtual std::string getTeamID() = 0;
std::string getDesc() const {
const auto& servers = getLastKnownServerInterfaces();
std::string s = format("Size %d; ", servers.size());
std::string s = format("TeamID:%s", getTeamID().c_str());
s += format("Size %d; ", servers.size());
for(int i=0; i<servers.size(); i++) {
if (i) s += ", ";
s += servers[i].address().toString() + " " + servers[i].id().shortString();

View File

@ -83,6 +83,13 @@ struct RelocateData {
class ParallelTCInfo : public ReferenceCounted<ParallelTCInfo>, public IDataDistributionTeam {
vector<Reference<IDataDistributionTeam>> teams;
vector<UID> tempServerIDs;
ParallelTCInfo() {}
void addTeam(Reference<IDataDistributionTeam> team) {
teams.push_back(team);
}
int64_t sum(std::function<int64_t(IDataDistributionTeam const&)> func) const {
int64_t result = 0;
@ -234,6 +241,15 @@ public:
ASSERT(!teams.empty());
teams[0]->addServers(servers);
}
std::string getTeamID() override {
std::string id;
for (int i = 0; i < teams.size(); i++) {
auto const& team = teams[i];
id += (i == teams.size() - 1) ? team->getTeamID() : format("%s, ", team->getTeamID().c_str());
}
return id;
}
};
struct Busyness {
@ -1036,6 +1052,9 @@ ACTOR Future<Void> dataDistributionRelocator( DDQueueData *self, RelocateData rd
} else {
TraceEvent(relocateShardInterval.severity, "RelocateShardHasDestination", distributorId)
.detail("PairId", relocateShardInterval.pairID)
.detail("KeyBegin", rd.keys.begin)
.detail("KeyEnd", rd.keys.end)
.detail("SourceServers", describe(rd.src))
.detail("DestinationTeam", describe(destIds))
.detail("ExtraIds", describe(extraIds));
}

View File

@ -701,8 +701,10 @@ ACTOR Future<Void> shardTracker(
wait( delay(0, TaskPriority::DataDistribution) );
}
} catch (Error& e) {
if (e.code() != error_code_actor_cancelled)
self->output.sendError(e); // Propagate failure to dataDistributionTracker
// If e is broken_promise then self may have already been deleted
if (e.code() != error_code_actor_cancelled && e.code() != error_code_broken_promise) {
self->output.sendError(e); // Propagate failure to dataDistributionTracker
}
throw e;
}
}

View File

@ -246,6 +246,7 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
init( DD_SS_STUCK_TIME_LIMIT, 300.0 ); if( randomize && BUGGIFY ) { DD_SS_STUCK_TIME_LIMIT = 200.0 + deterministicRandom()->random01() * 100.0; }
init( DD_TEAMS_INFO_PRINT_INTERVAL, 60 ); if( randomize && BUGGIFY ) DD_TEAMS_INFO_PRINT_INTERVAL = 10;
init( DD_TEAMS_INFO_PRINT_YIELD_COUNT, 100 ); if( randomize && BUGGIFY ) DD_TEAMS_INFO_PRINT_YIELD_COUNT = deterministicRandom()->random01() * 1000 + 1;
init( DD_TEAM_ZERO_SERVER_LEFT_LOG_DELAY, 120 ); if( randomize && BUGGIFY ) DD_TEAM_ZERO_SERVER_LEFT_LOG_DELAY = 5;
// TeamRemover
init( TR_FLAG_DISABLE_MACHINE_TEAM_REMOVER, false ); if( randomize && BUGGIFY ) TR_FLAG_DISABLE_MACHINE_TEAM_REMOVER = deterministicRandom()->random01() < 0.1 ? true : false; // false by default. disable the consistency check when it's true

View File

@ -193,6 +193,7 @@ public:
double DD_SS_STUCK_TIME_LIMIT; // If a storage server is not getting new versions for this amount of time, then it becomes undesired.
int DD_TEAMS_INFO_PRINT_INTERVAL;
int DD_TEAMS_INFO_PRINT_YIELD_COUNT;
int DD_TEAM_ZERO_SERVER_LEFT_LOG_DELAY;
// TeamRemover to remove redundant teams
bool TR_FLAG_DISABLE_MACHINE_TEAM_REMOVER; // disable the machineTeamRemover actor

View File

@ -30,6 +30,8 @@
#include "fdbserver/ApplyMetadataMutation.h"
#include "fdbserver/RecoveryState.h"
#include "fdbclient/Atomic.h"
#include "flow/Arena.h"
#include "flow/Histogram.h"
#include "flow/TDMetric.actor.h"
#include "flow/actorcompiler.h" // This must be the last #include.
@ -75,6 +77,7 @@ struct LogRouterData {
const UID dbgid;
Reference<AsyncVar<Reference<ILogSystem>>> logSystem;
Optional<UID> primaryPeekLocation;
NotifiedVersion version;
NotifiedVersion minPopped;
const Version startVersion;
@ -89,6 +92,8 @@ struct LogRouterData {
double maxWaitForVersionTime = 0;
double getMoreTime = 0;
double maxGetMoreTime = 0;
int64_t generation = -1;
Reference<Histogram> peekLatencyDist;
struct PeekTrackerData {
std::map<int, Promise<std::pair<Version, bool>>> sequence_version;
@ -119,9 +124,14 @@ struct LogRouterData {
return newTagData;
}
LogRouterData(UID dbgid, const InitializeLogRouterRequest& req) : dbgid(dbgid), routerTag(req.routerTag), logSystem(new AsyncVar<Reference<ILogSystem>>()),
version(req.startVersion-1), minPopped(0), startVersion(req.startVersion), allowPops(false), minKnownCommittedVersion(0), poppedVersion(0), foundEpochEnd(false),
cc("LogRouter", dbgid.toString()), getMoreCount("GetMoreCount", cc), getMoreBlockedCount("GetMoreBlockedCount", cc) {
LogRouterData(UID dbgid, const InitializeLogRouterRequest& req)
: dbgid(dbgid), routerTag(req.routerTag), logSystem(new AsyncVar<Reference<ILogSystem>>()),
version(req.startVersion - 1), minPopped(0), generation(req.recoveryCount), startVersion(req.startVersion),
allowPops(false), minKnownCommittedVersion(0), poppedVersion(0), foundEpochEnd(false),
cc("LogRouter", dbgid.toString()), getMoreCount("GetMoreCount", cc),
getMoreBlockedCount("GetMoreBlockedCount", cc),
peekLatencyDist(Histogram::getHistogram(LiteralStringRef("LogRouter"), LiteralStringRef("PeekTLogLatency"),
Histogram::Unit::microseconds)) {
//setup just enough of a logSet to be able to call getPushLocations
logSet.logServers.resize(req.tLogLocalities.size());
logSet.tLogPolicy = req.tLogPolicy;
@ -148,7 +158,12 @@ struct LogRouterData {
specialCounter(cc, "WaitForVersionMaxMS", [this](){ double val = this->maxWaitForVersionTime; this->maxWaitForVersionTime = 0; return 1000*val; });
specialCounter(cc, "GetMoreMS", [this](){ double val = this->getMoreTime; this->getMoreTime = 0; return 1000*val; });
specialCounter(cc, "GetMoreMaxMS", [this](){ double val = this->maxGetMoreTime; this->maxGetMoreTime = 0; return 1000*val; });
logger = traceCounters("LogRouterMetrics", dbgid, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "LogRouterMetrics");
specialCounter(cc, "Generation", [this]() { return this->generation; });
logger = traceCounters("LogRouterMetrics", dbgid, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc,
"LogRouterMetrics", [this](TraceEvent& te) {
te.detail("PrimaryPeekLocation", this->primaryPeekLocation);
te.detail("RouterTag", this->routerTag.toString());
});
}
};
@ -255,13 +270,16 @@ ACTOR Future<Void> pullAsyncData( LogRouterData *self ) {
state double startTime = now();
choose {
when(wait( getMoreF ) ) {
self->getMoreTime += now() - startTime;
self->maxGetMoreTime = std::max(self->maxGetMoreTime, now() - startTime);
double peekTime = now() - startTime;
self->peekLatencyDist->sampleSeconds(peekTime);
self->getMoreTime += peekTime;
self->maxGetMoreTime = std::max(self->maxGetMoreTime, peekTime);
break;
}
when( wait( dbInfoChange ) ) { //FIXME: does this actually happen?
if( self->logSystem->get() ) {
r = self->logSystem->get()->peekLogRouter( self->dbgid, tagAt, self->routerTag );
self->primaryPeekLocation = r->getPrimaryPeekLocation();
TraceEvent("LogRouterPeekLocation", self->dbgid).detail("LogID", r->getPrimaryPeekLocation()).trackLatest(self->eventCacheHolder->trackingKey);
} else {
r = Reference<ILogSystem::IPeekCursor>();

View File

@ -139,8 +139,20 @@ ACTOR Future<Void> resetChecker( ILogSystem::ServerPeekCursor* self, NetworkAddr
self->unknownReplies = 0;
self->fastReplies = 0;
wait(delay(SERVER_KNOBS->PEEK_STATS_INTERVAL));
TraceEvent("SlowPeekStats").detail("PeerAddress", addr).detail("SlowReplies", self->slowReplies).detail("FastReplies", self->fastReplies).detail("UnknownReplies", self->unknownReplies);
if(self->slowReplies >= SERVER_KNOBS->PEEK_STATS_SLOW_AMOUNT && self->slowReplies/double(self->slowReplies+self->fastReplies) >= SERVER_KNOBS->PEEK_STATS_SLOW_RATIO) {
TraceEvent("SlowPeekStats", self->randomID)
.detail("PeerAddress", addr)
.detail("SlowReplies", self->slowReplies)
.detail("FastReplies", self->fastReplies)
.detail("UnknownReplies", self->unknownReplies);
if (self->slowReplies >= SERVER_KNOBS->PEEK_STATS_SLOW_AMOUNT &&
self->slowReplies / double(self->slowReplies + self->fastReplies) >= SERVER_KNOBS->PEEK_STATS_SLOW_RATIO) {
TraceEvent("ConnectionResetSlowPeek", self->randomID)
.detail("PeerAddress", addr)
.detail("SlowReplies", self->slowReplies)
.detail("FastReplies", self->fastReplies)
.detail("UnknownReplies", self->unknownReplies);
FlowTransport::transport().resetConnection(addr);
self->lastReset = now();
}

View File

@ -864,7 +864,10 @@ ACTOR Future<Void> commitBatch(
g_traceBatch.addEvent("CommitDebug", debugID.get().first(), "MasterProxyServer.commitBatch.Before");
if(localBatchNumber-self->latestLocalCommitBatchResolving.get()>SERVER_KNOBS->RESET_MASTER_BATCHES && now()-self->lastMasterReset>SERVER_KNOBS->RESET_MASTER_DELAY) {
TraceEvent(SevWarnAlways, "ResetMasterNetwork").detail("CurrentBatch", localBatchNumber).detail("InProcessBatch", self->latestLocalCommitBatchResolving.get());
TraceEvent(SevWarnAlways, "ConnectionResetMaster", self->dbgid)
.detail("PeerAddress", self->master.address())
.detail("CurrentBatch", localBatchNumber)
.detail("InProcessBatch", self->latestLocalCommitBatchResolving.get());
FlowTransport::transport().resetConnection(self->master.address());
self->lastMasterReset=now();
}
@ -929,9 +932,14 @@ ACTOR Future<Void> commitBatch(
std::move(requests.txReadConflictRangeIndexMap); // used to report conflicting keys
state Future<Void> releaseFuture = releaseResolvingAfter(self, releaseDelay, localBatchNumber);
if(localBatchNumber-self->latestLocalCommitBatchLogging.get()>SERVER_KNOBS->RESET_RESOLVER_BATCHES && now()-self->lastResolverReset>SERVER_KNOBS->RESET_RESOLVER_DELAY) {
TraceEvent(SevWarnAlways, "ResetResolverNetwork").detail("CurrentBatch", localBatchNumber).detail("InProcessBatch", self->latestLocalCommitBatchLogging.get());
if (localBatchNumber - self->latestLocalCommitBatchLogging.get() > SERVER_KNOBS->RESET_RESOLVER_BATCHES &&
now() - self->lastResolverReset > SERVER_KNOBS->RESET_RESOLVER_DELAY) {
for (int r = 0; r<self->resolvers.size(); r++) {
TraceEvent(SevWarnAlways, "ConnectionResetResolver", self->dbgid)
.detail("PeerAddr", self->resolvers[r].address())
.detail("CurrentBatch", localBatchNumber)
.detail("InProcessBatch", self->latestLocalCommitBatchLogging.get());
FlowTransport::transport().resetConnection(self->resolvers[r].address());
}
self->lastResolverReset=now();

View File

@ -587,6 +587,7 @@ struct LogData : NonCopyable, public ReferenceCounted<LogData> {
specialCounter(cc, "QueueDiskBytesTotal", [tLogData](){ return tLogData->rawPersistentQueue->getStorageBytes().total; });
specialCounter(cc, "PeekMemoryReserved", [tLogData]() { return tLogData->peekMemoryLimiter.activePermits(); });
specialCounter(cc, "PeekMemoryRequestsStalled", [tLogData]() { return tLogData->peekMemoryLimiter.waiters(); });
specialCounter(cc, "Geneartion", [this]() { return this->recoveryCount; });
}
~LogData() {

View File

@ -485,8 +485,10 @@ ACTOR Future<Void> dumpDatabase( Database cx, std::string outputFilename, KeyRan
void memoryTest();
void skipListTest();
Future<Void> startSystemMonitor(std::string dataFolder, Optional<Standalone<StringRef>> zoneId, Optional<Standalone<StringRef>> machineId) {
initializeSystemMonitorMachineState(SystemMonitorMachineState(dataFolder, zoneId, machineId, g_network->getLocalAddress().ip));
Future<Void> startSystemMonitor(std::string dataFolder, Optional<Standalone<StringRef>> dcId,
Optional<Standalone<StringRef>> zoneId, Optional<Standalone<StringRef>> machineId) {
initializeSystemMonitorMachineState(
SystemMonitorMachineState(dataFolder, dcId, zoneId, machineId, g_network->getLocalAddress().ip));
systemMonitor();
return recurring( &systemMonitor, 5.0, TaskPriority::FlushTrace );
@ -1896,14 +1898,14 @@ int main(int argc, char* argv[]) {
g_network->run();
} else if (role == Test) {
setupRunLoopProfiler();
auto m = startSystemMonitor(opts.dataFolder, opts.zoneId, opts.zoneId);
auto m = startSystemMonitor(opts.dataFolder, opts.dcId opts.zoneId, opts.zoneId);
f = stopAfter(runTests(opts.connectionFile, TEST_TYPE_FROM_FILE, TEST_HERE, 1, opts.testFile, StringRef(),
opts.localities));
g_network->run();
} else if (role == ConsistencyCheck) {
setupRunLoopProfiler();
auto m = startSystemMonitor(opts.dataFolder, opts.zoneId, opts.zoneId);
auto m = startSystemMonitor(opts.dataFolder, opts.dcId, opts.zoneId, opts.zoneId);
f = stopAfter(runTests(opts.connectionFile, TEST_TYPE_CONSISTENCY_CHECK, TEST_HERE, 1, opts.testFile,
StringRef(), opts.localities));
g_network->run();

View File

@ -3621,7 +3621,10 @@ ACTOR Future<Void> metricsCore( StorageServer* self, StorageServerInterface ssi
wait( self->byteSampleRecovery );
self->actors.add(traceCounters("StorageMetrics", self->thisServerID, SERVER_KNOBS->STORAGE_LOGGING_DELAY, &self->counters.cc, self->thisServerID.toString() + "/StorageMetrics"));
Tag tag = self->tag;
self->actors.add(traceCounters("StorageMetrics", self->thisServerID, SERVER_KNOBS->STORAGE_LOGGING_DELAY,
&self->counters.cc, self->thisServerID.toString() + "/StorageMetrics",
[tag](TraceEvent& te) { te.detail("Tag", tag.toString()); }));
loop {
choose {

View File

@ -978,7 +978,8 @@ ACTOR Future<Void> workerServer(
filesClosed.add(stopping.getFuture());
initializeSystemMonitorMachineState(SystemMonitorMachineState(folder, locality.zoneId(), locality.machineId(), g_network->getLocalAddress().ip));
initializeSystemMonitorMachineState(SystemMonitorMachineState(
folder, locality.dcId(), locality.zoneId(), locality.machineId(), g_network->getLocalAddress().ip));
{
auto recruited = interf; //ghetto! don't we all love a good #define

View File

@ -28,9 +28,9 @@
extern IKeyValueStore *makeDummyKeyValueStore();
template <class T>
class Histogram {
class TestHistogram {
public:
Histogram(int minSamples = 100) : minSamples(minSamples) { reset(); }
TestHistogram(int minSamples = 100) : minSamples(minSamples) { reset(); }
void reset() {
N = 0;
@ -145,7 +145,7 @@ struct KVTest {
}
};
ACTOR Future<Void> testKVRead( KVTest* test, Key key, Histogram<float>* latency, PerfIntCounter* count ) {
ACTOR Future<Void> testKVRead(KVTest* test, Key key, TestHistogram<float>* latency, PerfIntCounter* count) {
// state Version s1 = test->lastCommit;
state Version s2 = test->lastDurable;
@ -163,7 +163,7 @@ ACTOR Future<Void> testKVRead( KVTest* test, Key key, Histogram<float>* latency,
return Void();
}
ACTOR Future<Void> testKVReadSaturation( KVTest* test, Histogram<float>* latency, PerfIntCounter* count ) {
ACTOR Future<Void> testKVReadSaturation(KVTest* test, TestHistogram<float>* latency, PerfIntCounter* count) {
while (true) {
state double begin = timer();
Optional<Value> val = wait(test->store->readValue(test->randomKey()));
@ -173,7 +173,7 @@ ACTOR Future<Void> testKVReadSaturation( KVTest* test, Histogram<float>* latency
}
}
ACTOR Future<Void> testKVCommit( KVTest* test, Histogram<float>* latency, PerfIntCounter* count ) {
ACTOR Future<Void> testKVCommit(KVTest* test, TestHistogram<float>* latency, PerfIntCounter* count) {
state Version v = test->lastSet;
test->lastCommit = v;
state double begin = timer();
@ -194,7 +194,7 @@ struct KVStoreTestWorkload : TestWorkload {
bool doSetup, doClear, doCount;
std::string filename;
PerfIntCounter reads, sets, commits;
Histogram<float> readLatency, commitLatency;
TestHistogram<float> readLatency, commitLatency;
double setupTook;
std::string storeType;
@ -222,7 +222,7 @@ struct KVStoreTestWorkload : TestWorkload {
return Void();
}
virtual Future<bool> check(Database const& cx) { return true; }
void metricsFromHistogram(vector<PerfMetric>& m, std::string name, Histogram<float>& h) {
void metricsFromHistogram(vector<PerfMetric>& m, std::string name, TestHistogram<float>& h) {
m.push_back(PerfMetric("Min " + name, 1000.0 * h.min(), true));
m.push_back(PerfMetric("Average " + name, 1000.0 * h.mean(), true));
m.push_back(PerfMetric("Median " + name, 1000.0 * h.medianEstimate(), true));

View File

@ -24,6 +24,8 @@ set(FLOW_SRCS
FileTraceLogWriter.h
Hash3.c
Hash3.h
Histogram.cpp
Histogram.h
IDispatched.h
IRandom.h
IThreadPool.cpp

171
flow/Histogram.cpp Normal file
View File

@ -0,0 +1,171 @@
/*
* Histogram.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.
*/
#include <flow/Histogram.h>
#include <flow/flow.h>
#include <flow/UnitTest.h>
// TODO: remove dependency on fdbrpc.
// we need to be able to check if we're in simulation so that the histograms are properly
// scoped to the right "machine".
// either we pull g_simulator into flow, or flow (and the I/O path) will be unable to log performance
// metrics.
#include <fdbrpc/simulator.h>
// pull in some global pointers too: These types are implemented in fdbrpc/sim2.actor.cpp, which is not available here.
// Yuck. If you're not using the simulator, these will remain null, and all should be well.
// TODO: create a execution context abstraction that allows independent flow instances within a process.
// The simulator would be the main user of it, and histogram would be the only other user (for now).
ISimulator* g_pSimulator = nullptr;
thread_local ISimulator::ProcessInfo* ISimulator::currentProcess = nullptr;
// Fallback registry when we're not in simulation -- if we had execution contexts we wouldn't need to check if
// we have a simulated contex here; we'd just use the current context regardless.
static HistogramRegistry* globalHistograms = nullptr;
HistogramRegistry& GetHistogramRegistry() {
ISimulator::ProcessInfo* info = g_simulator.getCurrentProcess();
if (info) {
// in simulator; scope histograms to simulated process
return info->histograms;
}
// avoid link order issues where the registry hasn't been initialized, but we're
// instantiating a histogram
if (globalHistograms == nullptr) {
// Note: This will show up as a leak on shutdown, but we're OK with that.
globalHistograms = new HistogramRegistry();
}
return *globalHistograms;
}
void HistogramRegistry::registerHistogram(Histogram* h) {
if (histograms.find(h->name()) != histograms.end()) {
TraceEvent(SevError, "HistogramDoubleRegistered").detail("group", h->group).detail("op", h->op);
ASSERT(false);
}
histograms.insert(std::pair<std::string, Histogram*>(h->name(), h));
}
void HistogramRegistry::unregisterHistogram(Histogram* h) {
std::string name = h->name();
if (histograms.find(name) == histograms.end()) {
TraceEvent(SevError, "HistogramNotRegistered").detail("group", h->group).detail("op", h->op);
}
int count = histograms.erase(name);
ASSERT(count == 1);
}
Histogram* HistogramRegistry::lookupHistogram(std::string name) {
auto h = histograms.find(name);
if (h == histograms.end()) {
return nullptr;
}
return h->second;
}
void HistogramRegistry::logReport() {
for (auto& i : histograms) {
i.second->writeToLog();
i.second->clear();
}
}
void Histogram::writeToLog() {
bool active = false;
for (uint32_t i = 0; i < 32; i++) {
if (buckets[i]) {
active = true;
break;
}
}
if (!active) {
return;
}
TraceEvent e(SevInfo, "Histogram");
e.detail("Group", group).detail("Op", op);
for (uint32_t i = 0; i < 32; i++) {
if (buckets[i]) {
switch (unit) {
case Unit::microseconds: {
uint32_t usec = ((uint32_t)1) << (i + 1);
e.detail(format("LessThan%u.%03u", usec / 1000, usec % 1000), buckets[i]);
break;
}
case Unit::bytes:
e.detail(format("LessThan%u", ((uint32_t)1) << (i + 1)), buckets[i]);
break;
default:
ASSERT(false);
}
}
}
}
TEST_CASE("/flow/histogram/smoke_test") {
{
Reference<Histogram> h =
Histogram::getHistogram(LiteralStringRef("smoke_test"), LiteralStringRef("counts"), Histogram::Unit::bytes);
h->sample(0);
ASSERT(h->buckets[0] == 1);
h->sample(1);
ASSERT(h->buckets[0] == 2);
h->sample(2);
ASSERT(h->buckets[1] == 1);
GetHistogramRegistry().logReport();
ASSERT(h->buckets[0] == 0);
h->sample(0);
ASSERT(h->buckets[0] == 1);
h = Histogram::getHistogram(LiteralStringRef("smoke_test"), LiteralStringRef("counts2"),
Histogram::Unit::bytes);
// confirm that old h was deallocated.
h = Histogram::getHistogram(LiteralStringRef("smoke_test"), LiteralStringRef("counts"), Histogram::Unit::bytes);
ASSERT(h->buckets[0] == 0);
h = Histogram::getHistogram(LiteralStringRef("smoke_test"), LiteralStringRef("times"),
Histogram::Unit::microseconds);
h->sampleSeconds(0.000000);
h->sampleSeconds(0.0000019);
ASSERT(h->buckets[0] == 2);
h->sampleSeconds(0.0000021);
ASSERT(h->buckets[1] == 1);
h->sampleSeconds(0.000015);
ASSERT(h->buckets[3] == 1);
h->sampleSeconds(4400.0);
ASSERT(h->buckets[31] == 1);
GetHistogramRegistry().logReport();
}
// h has been deallocated. Does this crash?
GetHistogramRegistry().logReport();
return Void();
}

131
flow/Histogram.h Normal file
View File

@ -0,0 +1,131 @@
/*
* Histogram.h
*
* 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.
*/
#ifndef FLOW_HISTOGRAM_H
#define FLOW_HISTOGRAM_H
#pragma once
#include <flow/Arena.h>
#include <string>
#include <map>
#ifdef _WIN32
#include <intrin.h>
#pragma intrinsic(_BitScanReverse)
#endif
class Histogram;
class HistogramRegistry {
public:
void registerHistogram(Histogram* h);
void unregisterHistogram(Histogram* h);
Histogram* lookupHistogram(std::string name);
void logReport();
private:
// This map is ordered by key so that ops within the same group end up
// next to each other in the trace log.
std::map<std::string, Histogram*> histograms;
};
HistogramRegistry& GetHistogramRegistry();
/*
* A fast histogram with power-of-two spaced buckets.
*
* For more information about this technique, see:
* https://www.fsl.cs.stonybrook.edu/project-osprof.html
*/
class Histogram sealed : public ReferenceCounted<Histogram> {
public:
enum class Unit { microseconds, bytes };
private:
Histogram(std::string group, std::string op, Unit unit, HistogramRegistry& registry)
: group(group), op(op), unit(unit), registry(registry), ReferenceCounted<Histogram>() {
clear();
}
static std::string generateName(std::string group, std::string op) { return group + ":" + op; }
public:
~Histogram() { registry.unregisterHistogram(this); }
static Reference<Histogram> getHistogram(StringRef group, StringRef op, Unit unit) {
std::string group_str = group.toString();
std::string op_str = op.toString();
std::string name = generateName(group_str, op_str);
HistogramRegistry& registry = GetHistogramRegistry();
Histogram* h = registry.lookupHistogram(name);
if (!h) {
h = new Histogram(group_str, op_str, unit, registry);
registry.registerHistogram(h);
return Reference<Histogram>(h);
} else {
return Reference<Histogram>::addRef(h);
}
}
// This histogram buckets samples into powers of two.
inline void sample(uint32_t sample) {
size_t idx;
#ifdef _WIN32
unsigned long index;
// _BitScanReverse sets index to the position of the first non-zero bit, so
// _BitScanReverse(sample) ~= log_2(sample). _BitScanReverse returns false if
// sample is zero.
idx = _BitScanReverse(&index, sample) ? index : 0;
#else
// __builtin_clz counts the leading zeros in its uint32_t argument. So, 31-clz ~= log_2(sample).
// __builtin_clz(0) is undefined.
idx = sample ? (31 - __builtin_clz(sample)) : 0;
#endif
ASSERT(idx < 32);
buckets[idx]++;
}
inline void sampleSeconds(double delta) {
uint64_t delta_usec = (delta * 1000000);
if (delta_usec > UINT32_MAX) {
sample(UINT32_MAX);
} else {
sample((uint32_t)(delta * 1000000)); // convert to microseconds and truncate to integer
}
}
void clear() {
for (uint32_t& i : buckets) {
i = 0;
}
}
void writeToLog();
std::string name() { return generateName(this->group, this->op); }
std::string const group;
std::string const op;
Unit const unit;
HistogramRegistry& registry;
uint32_t buckets[32];
};
#endif // FLOW_HISTOGRAM_H

View File

@ -84,6 +84,7 @@ void FlowKnobs::initialize(bool randomize, bool isSimulated) {
init( INCOMPATIBLE_PEER_DELAY_BEFORE_LOGGING, 5.0 );
init( PING_LOGGING_INTERVAL, 3.0 );
init( PING_SAMPLE_AMOUNT, 100 );
init( NETWORK_CONNECT_SAMPLE_AMOUNT, 100 );
init( TLS_CERT_REFRESH_DELAY_SECONDS, 12*60*60 );
init( TLS_SERVER_CONNECTION_THROTTLE_TIMEOUT, 9.0 );

View File

@ -98,6 +98,7 @@ public:
double INCOMPATIBLE_PEER_DELAY_BEFORE_LOGGING;
double PING_LOGGING_INTERVAL;
int PING_SAMPLE_AMOUNT;
int NETWORK_CONNECT_SAMPLE_AMOUNT;
int TLS_CERT_REFRESH_DELAY_SECONDS;
double TLS_SERVER_CONNECTION_THROTTLE_TIMEOUT;

View File

@ -153,6 +153,8 @@ void UnsentPacketQueue::sent(int bytes) {
bytes -= b->bytes_written - b->bytes_sent;
b->bytes_sent = b->bytes_written;
ASSERT(b->bytes_written <= b->size());
double queue_time = now() - b->enqueue_time;
sendQueueLatencyHistogram->sampleSeconds(queue_time);
unsent_first = b->nextPacketBuffer();
if (!unsent_first) unsent_last = NULL;
b->delref();

View File

@ -23,6 +23,7 @@
#pragma once
#include "flow/flow.h"
#include "flow/Histogram.h"
// PacketWriter and PacketBuffer are in serialize.h because they are needed by the SerializeSource<> template
@ -40,8 +41,17 @@ struct ReliablePacket : FastAllocated<ReliablePacket> {
class UnsentPacketQueue : NonCopyable {
public:
UnsentPacketQueue() : unsent_first(0), unsent_last(0) {}
~UnsentPacketQueue() { discardAll(); }
UnsentPacketQueue()
: unsent_first(0), unsent_last(0),
sendQueueLatencyHistogram(Histogram::getHistogram(
LiteralStringRef("UnsentPacketQueue"), LiteralStringRef("QueueWait"), Histogram::Unit::microseconds)) {}
~UnsentPacketQueue() {
discardAll();
unsent_first = (PacketBuffer*)0xDEADBEEF;
unsent_last = (PacketBuffer*)0xCAFEBABE;
sendQueueLatencyHistogram = Reference<Histogram>(nullptr);
}
// Get a PacketBuffer to write new packets into
PacketBuffer* getWriteBuffer(size_t sizeHint = 0) {
@ -70,6 +80,7 @@ public:
private:
PacketBuffer *unsent_first, *unsent_last; // Both NULL, or inclusive range of PacketBuffers that haven't been sent. The last one may have space for more packets to be written.
Reference<Histogram> sendQueueLatencyHistogram;
};
class ReliablePacketList : NonCopyable {

View File

@ -19,6 +19,7 @@
*/
#include "flow/flow.h"
#include "flow/Histogram.h"
#include "flow/Platform.h"
#include "flow/TDMetric.actor.h"
#include "flow/SystemMonitor.h"
@ -59,88 +60,119 @@ SystemStatistics customSystemMonitor(std::string eventName, StatisticsState *sta
netData.init();
if (!g_network->isSimulated() && currentStats.initialized) {
{
GetHistogramRegistry().logReport();
TraceEvent(eventName.c_str())
.detail("Elapsed", currentStats.elapsed)
.detail("CPUSeconds", currentStats.processCPUSeconds)
.detail("MainThreadCPUSeconds", currentStats.mainThreadCPUSeconds)
.detail("UptimeSeconds", now() - machineState.monitorStartTime)
.detail("Memory", currentStats.processMemory)
.detail("ResidentMemory", currentStats.processResidentMemory)
.detail("UnusedAllocatedMemory", getTotalUnusedAllocatedMemory())
.detail("MbpsSent", ((netData.bytesSent - statState->networkState.bytesSent) * 8e-6) / currentStats.elapsed)
.detail("MbpsReceived", ((netData.bytesReceived - statState->networkState.bytesReceived) * 8e-6) / currentStats.elapsed)
.detail("DiskTotalBytes", currentStats.processDiskTotalBytes)
.detail("DiskFreeBytes", currentStats.processDiskFreeBytes)
.detail("DiskQueueDepth", currentStats.processDiskQueueDepth)
.detail("DiskIdleSeconds", currentStats.processDiskIdleSeconds)
.detail("DiskReads", currentStats.processDiskRead)
.detail("DiskWrites", currentStats.processDiskWrite)
.detail("DiskReadsCount", currentStats.processDiskReadCount)
.detail("DiskWritesCount", currentStats.processDiskWriteCount)
.detail("DiskWriteSectors", currentStats.processDiskWriteSectors)
.detail("DiskReadSectors", currentStats.processDiskReadSectors)
.detail("FileWrites", netData.countFileLogicalWrites - statState->networkState.countFileLogicalWrites)
.detail("FileReads", netData.countFileLogicalReads - statState->networkState.countFileLogicalReads)
.detail("CacheReadBytes", netData.countFileCacheReadBytes - statState->networkState.countFileCacheReadBytes)
.detail("CacheFinds", netData.countFileCacheFinds - statState->networkState.countFileCacheFinds)
.detail("CacheWritesBlocked", netData.countFileCacheWritesBlocked - statState->networkState.countFileCacheWritesBlocked)
.detail("CacheReadsBlocked", netData.countFileCacheReadsBlocked - statState->networkState.countFileCacheReadsBlocked)
.detail("CachePageReadsMerged", netData.countFileCachePageReadsMerged - statState->networkState.countFileCachePageReadsMerged)
.detail("CacheWrites", netData.countFileCacheWrites - statState->networkState.countFileCacheWrites)
.detail("CacheReads", netData.countFileCacheReads - statState->networkState.countFileCacheReads)
.detail("CacheHits", netData.countFilePageCacheHits - statState->networkState.countFilePageCacheHits)
.detail("CacheMisses", netData.countFilePageCacheMisses - statState->networkState.countFilePageCacheMisses)
.detail("CacheEvictions", netData.countFilePageCacheEvictions - statState->networkState.countFilePageCacheEvictions)
.detail("ZoneID", machineState.zoneId)
.detail("MachineID", machineState.machineId)
.detail("AIOSubmitCount", netData.countAIOSubmit - statState->networkState.countAIOSubmit)
.detail("AIOCollectCount", netData.countAIOCollect - statState->networkState.countAIOCollect)
.detail("AIOSubmitLag", (g_network->networkInfo.metrics.secSquaredSubmit - statState->networkMetricsState.secSquaredSubmit) / currentStats.elapsed)
.detail("AIODiskStall", (g_network->networkInfo.metrics.secSquaredDiskStall - statState->networkMetricsState.secSquaredDiskStall) / currentStats.elapsed)
.detail("CurrentConnections", netData.countConnEstablished - netData.countConnClosedWithError - netData.countConnClosedWithoutError)
.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);
.detail("Elapsed", currentStats.elapsed)
.detail("CPUSeconds", currentStats.processCPUSeconds)
.detail("MainThreadCPUSeconds", currentStats.mainThreadCPUSeconds)
.detail("UptimeSeconds", now() - machineState.monitorStartTime)
.detail("Memory", currentStats.processMemory)
.detail("ResidentMemory", currentStats.processResidentMemory)
.detail("UnusedAllocatedMemory", getTotalUnusedAllocatedMemory())
.detail("MbpsSent",
((netData.bytesSent - statState->networkState.bytesSent) * 8e-6) / currentStats.elapsed)
.detail("MbpsReceived",
((netData.bytesReceived - statState->networkState.bytesReceived) * 8e-6) / currentStats.elapsed)
.detail("DiskTotalBytes", currentStats.processDiskTotalBytes)
.detail("DiskFreeBytes", currentStats.processDiskFreeBytes)
.detail("DiskQueueDepth", currentStats.processDiskQueueDepth)
.detail("DiskIdleSeconds", currentStats.processDiskIdleSeconds)
.detail("DiskReads", currentStats.processDiskRead)
.detail("DiskWrites", currentStats.processDiskWrite)
.detail("DiskReadsCount", currentStats.processDiskReadCount)
.detail("DiskWritesCount", currentStats.processDiskWriteCount)
.detail("DiskWriteSectors", currentStats.processDiskWriteSectors)
.detail("DiskReadSectors", currentStats.processDiskReadSectors)
.detail("FileWrites", netData.countFileLogicalWrites - statState->networkState.countFileLogicalWrites)
.detail("FileReads", netData.countFileLogicalReads - statState->networkState.countFileLogicalReads)
.detail("CacheReadBytes",
netData.countFileCacheReadBytes - statState->networkState.countFileCacheReadBytes)
.detail("CacheFinds", netData.countFileCacheFinds - statState->networkState.countFileCacheFinds)
.detail("CacheWritesBlocked",
netData.countFileCacheWritesBlocked - statState->networkState.countFileCacheWritesBlocked)
.detail("CacheReadsBlocked",
netData.countFileCacheReadsBlocked - statState->networkState.countFileCacheReadsBlocked)
.detail("CachePageReadsMerged",
netData.countFileCachePageReadsMerged - statState->networkState.countFileCachePageReadsMerged)
.detail("CacheWrites", netData.countFileCacheWrites - statState->networkState.countFileCacheWrites)
.detail("CacheReads", netData.countFileCacheReads - statState->networkState.countFileCacheReads)
.detail("CacheHits", netData.countFilePageCacheHits - statState->networkState.countFilePageCacheHits)
.detail("CacheMisses",
netData.countFilePageCacheMisses - statState->networkState.countFilePageCacheMisses)
.detail("CacheEvictions",
netData.countFilePageCacheEvictions - statState->networkState.countFilePageCacheEvictions)
.detail("DCID", machineState.dcId)
.detail("ZoneID", machineState.zoneId)
.detail("MachineID", machineState.machineId)
.detail("AIOSubmitCount", netData.countAIOSubmit - statState->networkState.countAIOSubmit)
.detail("AIOCollectCount", netData.countAIOCollect - statState->networkState.countAIOCollect)
.detail("AIOSubmitLag", (g_network->networkInfo.metrics.secSquaredSubmit -
statState->networkMetricsState.secSquaredSubmit) /
currentStats.elapsed)
.detail("AIODiskStall", (g_network->networkInfo.metrics.secSquaredDiskStall -
statState->networkMetricsState.secSquaredDiskStall) /
currentStats.elapsed)
.detail("CurrentConnections", netData.countConnEstablished - netData.countConnClosedWithError -
netData.countConnClosedWithoutError)
.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")
.DETAILALLOCATORMEMUSAGE(16)
.DETAILALLOCATORMEMUSAGE(32)
.DETAILALLOCATORMEMUSAGE(64)
.DETAILALLOCATORMEMUSAGE(96)
.DETAILALLOCATORMEMUSAGE(128)
.DETAILALLOCATORMEMUSAGE(256)
.DETAILALLOCATORMEMUSAGE(512)
.DETAILALLOCATORMEMUSAGE(1024)
.DETAILALLOCATORMEMUSAGE(2048)
.DETAILALLOCATORMEMUSAGE(4096)
.DETAILALLOCATORMEMUSAGE(8192)
.detail("HugeArenaMemory", g_hugeArenaMemory.load());
.DETAILALLOCATORMEMUSAGE(16)
.DETAILALLOCATORMEMUSAGE(32)
.DETAILALLOCATORMEMUSAGE(64)
.DETAILALLOCATORMEMUSAGE(96)
.DETAILALLOCATORMEMUSAGE(128)
.DETAILALLOCATORMEMUSAGE(256)
.DETAILALLOCATORMEMUSAGE(512)
.DETAILALLOCATORMEMUSAGE(1024)
.DETAILALLOCATORMEMUSAGE(2048)
.DETAILALLOCATORMEMUSAGE(4096)
.DETAILALLOCATORMEMUSAGE(8192)
.detail("HugeArenaMemory", g_hugeArenaMemory.load())
.detail("DCID", machineState.dcId)
.detail("ZoneID", machineState.zoneId)
.detail("MachineID", machineState.machineId);
TraceEvent n("NetworkMetrics");
n
.detail("Elapsed", currentStats.elapsed)
.detail("CantSleep", netData.countCantSleep - statState->networkState.countCantSleep)
.detail("WontSleep", netData.countWontSleep - statState->networkState.countWontSleep)
.detail("Yields", netData.countYields - statState->networkState.countYields)
.detail("YieldCalls", netData.countYieldCalls - statState->networkState.countYieldCalls)
.detail("YieldCallsTrue", netData.countYieldCallsTrue - statState->networkState.countYieldCallsTrue)
.detail("RunLoopProfilingSignals", netData.countRunLoopProfilingSignals - statState->networkState.countRunLoopProfilingSignals)
.detail("YieldBigStack", netData.countYieldBigStack - statState->networkState.countYieldBigStack)
.detail("RunLoopIterations", netData.countRunLoop - statState->networkState.countRunLoop)
.detail("TimersExecuted", netData.countTimers - statState->networkState.countTimers)
.detail("TasksExecuted", netData.countTasks - statState->networkState.countTasks)
.detail("ASIOEventsProcessed", netData.countASIOEvents - statState->networkState.countASIOEvents)
.detail("ReadCalls", netData.countReads - statState->networkState.countReads)
.detail("WriteCalls", netData.countWrites - statState->networkState.countWrites)
.detail("ReadProbes", netData.countReadProbes - statState->networkState.countReadProbes)
.detail("WriteProbes", netData.countWriteProbes - statState->networkState.countWriteProbes)
.detail("PacketsRead", netData.countPacketsReceived - statState->networkState.countPacketsReceived)
.detail("PacketsGenerated", netData.countPacketsGenerated - statState->networkState.countPacketsGenerated)
.detail("WouldBlock", netData.countWouldBlock - statState->networkState.countWouldBlock)
.detail("LaunchTime", netData.countLaunchTime - statState->networkState.countLaunchTime)
.detail("ReactTime", netData.countReactTime - statState->networkState.countReactTime);
n.detail("Elapsed", currentStats.elapsed)
.detail("CantSleep", netData.countCantSleep - statState->networkState.countCantSleep)
.detail("WontSleep", netData.countWontSleep - statState->networkState.countWontSleep)
.detail("Yields", netData.countYields - statState->networkState.countYields)
.detail("YieldCalls", netData.countYieldCalls - statState->networkState.countYieldCalls)
.detail("YieldCallsTrue", netData.countYieldCallsTrue - statState->networkState.countYieldCallsTrue)
.detail("RunLoopProfilingSignals",
netData.countRunLoopProfilingSignals - statState->networkState.countRunLoopProfilingSignals)
.detail("YieldBigStack", netData.countYieldBigStack - statState->networkState.countYieldBigStack)
.detail("RunLoopIterations", netData.countRunLoop - statState->networkState.countRunLoop)
.detail("TimersExecuted", netData.countTimers - statState->networkState.countTimers)
.detail("TasksExecuted", netData.countTasks - statState->networkState.countTasks)
.detail("ASIOEventsProcessed", netData.countASIOEvents - statState->networkState.countASIOEvents)
.detail("ReadCalls", netData.countReads - statState->networkState.countReads)
.detail("WriteCalls", netData.countWrites - statState->networkState.countWrites)
.detail("ReadProbes", netData.countReadProbes - statState->networkState.countReadProbes)
.detail("WriteProbes", netData.countWriteProbes - statState->networkState.countWriteProbes)
.detail("PacketsRead", netData.countPacketsReceived - statState->networkState.countPacketsReceived)
.detail("PacketsGenerated",
netData.countPacketsGenerated - statState->networkState.countPacketsGenerated)
.detail("WouldBlock", netData.countWouldBlock - statState->networkState.countWouldBlock)
.detail("LaunchTime", netData.countLaunchTime - statState->networkState.countLaunchTime)
.detail("ReactTime", netData.countReactTime - statState->networkState.countReactTime)
.detail("DCID", machineState.dcId)
.detail("ZoneID", machineState.zoneId)
.detail("MachineID", machineState.machineId);
for (int i = 0; i<NetworkMetrics::SLOW_EVENT_BINS; i++) {
if (int c = g_network->networkInfo.metrics.countSlowEvents[i] - statState->networkMetricsState.countSlowEvents[i]) {
@ -194,18 +226,20 @@ SystemStatistics customSystemMonitor(std::string eventName, StatisticsState *sta
}
if(machineMetrics) {
TraceEvent("MachineMetrics").detail("Elapsed", currentStats.elapsed)
.detail("MbpsSent", currentStats.machineMegabitsSent / currentStats.elapsed)
.detail("MbpsReceived", currentStats.machineMegabitsReceived / currentStats.elapsed)
.detail("OutSegs", currentStats.machineOutSegs)
.detail("RetransSegs", currentStats.machineRetransSegs)
.detail("CPUSeconds", currentStats.machineCPUSeconds)
.detail("TotalMemory", currentStats.machineTotalRAM)
.detail("CommittedMemory", currentStats.machineCommittedRAM)
.detail("AvailableMemory", currentStats.machineAvailableRAM)
.detail("ZoneID", machineState.zoneId)
.detail("MachineID", machineState.machineId)
.trackLatest("MachineMetrics");
TraceEvent("MachineMetrics")
.detail("Elapsed", currentStats.elapsed)
.detail("MbpsSent", currentStats.machineMegabitsSent / currentStats.elapsed)
.detail("MbpsReceived", currentStats.machineMegabitsReceived / currentStats.elapsed)
.detail("OutSegs", currentStats.machineOutSegs)
.detail("RetransSegs", currentStats.machineRetransSegs)
.detail("CPUSeconds", currentStats.machineCPUSeconds)
.detail("TotalMemory", currentStats.machineTotalRAM)
.detail("CommittedMemory", currentStats.machineCommittedRAM)
.detail("AvailableMemory", currentStats.machineAvailableRAM)
.detail("DCID", machineState.dcId)
.detail("ZoneID", machineState.zoneId)
.detail("MachineID", machineState.machineId)
.trackLatest("MachineMetrics");
}
}

View File

@ -27,6 +27,7 @@
struct SystemMonitorMachineState {
Optional<std::string> folder;
Optional<Standalone<StringRef>> dcId;
Optional<Standalone<StringRef>> zoneId;
Optional<Standalone<StringRef>> machineId;
Optional<IPAddress> ip;
@ -35,9 +36,10 @@ struct SystemMonitorMachineState {
SystemMonitorMachineState() : monitorStartTime(0) {}
explicit SystemMonitorMachineState(const IPAddress& ip) : ip(ip), monitorStartTime(0) {}
SystemMonitorMachineState(std::string folder, Optional<Standalone<StringRef>> zoneId,
Optional<Standalone<StringRef>> machineId, const IPAddress& ip)
: folder(folder), zoneId(zoneId), machineId(machineId), ip(ip), monitorStartTime(0) {}
SystemMonitorMachineState(std::string folder, Optional<Standalone<StringRef>> dcId,
Optional<Standalone<StringRef>> zoneId, Optional<Standalone<StringRef>> machineId,
const IPAddress& ip)
: folder(folder), dcId(dcId), zoneId(zoneId), machineId(machineId), ip(ip), monitorStartTime(0) {}
};
void initializeSystemMonitorMachineState(SystemMonitorMachineState machineState);

View File

@ -783,6 +783,13 @@ TraceEvent::TraceEvent(TraceEvent &&ev) {
tmpEventMetric = ev.tmpEventMetric;
trackingKey = ev.trackingKey;
type = ev.type;
timeIndex = ev.timeIndex;
for (int i = 0; i < 5; i++) {
eventCounts[i] = ev.eventCounts[i];
}
networkThread = ev.networkThread;
ev.initialized = true;
ev.enabled = false;
@ -791,6 +798,7 @@ TraceEvent::TraceEvent(TraceEvent &&ev) {
}
TraceEvent& TraceEvent::operator=(TraceEvent &&ev) {
// Note: still broken if ev and this are the same memory address.
enabled = ev.enabled;
err = ev.err;
fields = std::move(ev.fields);
@ -803,6 +811,13 @@ TraceEvent& TraceEvent::operator=(TraceEvent &&ev) {
tmpEventMetric = ev.tmpEventMetric;
trackingKey = ev.trackingKey;
type = ev.type;
timeIndex = ev.timeIndex;
for (int i = 0; i < 5; i++) {
eventCounts[i] = ev.eventCounts[i];
}
networkThread = ev.networkThread;
ev.initialized = true;
ev.enabled = false;

View File

@ -30,8 +30,9 @@
#ifndef TLS_DISABLED
#include "boost/asio/ssl.hpp"
#endif
#include "flow/serialize.h"
#include "flow/Arena.h"
#include "flow/IRandom.h"
#include "flow/Trace.h"
enum class TaskPriority {
Max = 1000000,
@ -120,8 +121,6 @@ inline TaskPriority incrementPriorityIfEven(TaskPriority p) {
class Void;
template<class T> class Optional;
struct IPAddress {
typedef boost::asio::ip::address_v6::bytes_type IPAddressStore;
static_assert(std::is_same<IPAddressStore, std::array<uint8_t, 16>>::value,
@ -369,6 +368,12 @@ public:
virtual Future<int64_t> read() = 0;
};
struct SendBuffer {
uint8_t const* data;
SendBuffer* next;
int bytes_written, bytes_sent;
};
class IConnection {
public:
// IConnection is reference-counted (use Reference<IConnection>), but the caller must explicitly call close()

View File

@ -30,6 +30,7 @@
#include "flow/Arena.h"
#include "flow/FileIdentifier.h"
#include "flow/ObjectSerializer.h"
#include "flow/network.h"
#include <algorithm>
// Though similar, is_binary_serializable cannot be replaced by std::is_pod, as doing so would prefer
@ -684,17 +685,20 @@ struct SendBuffer {
struct PacketBuffer : SendBuffer {
private:
static constexpr size_t PACKET_BUFFER_OVERHEAD = 40;
int reference_count;
uint32_t size_;
uint32_t const size_;
static constexpr size_t PACKET_BUFFER_MIN_SIZE = 16384;
static constexpr size_t PACKET_BUFFER_OVERHEAD = 32;
public:
double const enqueue_time;
uint8_t* data() { return const_cast<uint8_t*>(static_cast<SendBuffer*>(this)->data); }
size_t size() { return size_; }
private:
explicit PacketBuffer(size_t size) : reference_count(1), size_(size) {
explicit PacketBuffer(size_t size) : reference_count(1), size_(size), enqueue_time(g_network->now()) {
next = 0;
bytes_written = bytes_sent = 0;
((SendBuffer*)this)->data = reinterpret_cast<uint8_t*>(this + 1);