From 3442ebd3b7a69cc601252d61a3b06682f3adb6be Mon Sep 17 00:00:00 2001 From: sfc-gh-tclinkenbeard Date: Sat, 24 Jul 2021 11:20:51 -0700 Subject: [PATCH] Fix more -Wreorder-ctor warnings across many files --- .../BackupContainerLocalDirectory.actor.cpp | 3 +- fdbclient/DatabaseBackupAgent.actor.cpp | 25 ++++++++------- fdbclient/MultiVersionTransaction.actor.cpp | 10 +++--- fdbclient/SpecialKeySpace.actor.cpp | 5 +-- fdbclient/TaskBucket.actor.cpp | 13 ++++---- fdbrpc/FlowTransport.actor.cpp | 20 ++++++------ fdbserver/BackupWorker.actor.cpp | 4 +-- fdbserver/ClusterController.actor.cpp | 32 +++++++++---------- fdbserver/ConfigBroadcaster.actor.cpp | 2 +- fdbserver/ConfigDatabaseUnitTests.actor.cpp | 9 +++--- fdbserver/DataDistributionQueue.actor.cpp | 16 +++++----- fdbserver/DataDistributionTracker.actor.cpp | 8 ++--- fdbserver/DiskQueue.actor.cpp | 16 +++++----- fdbserver/KeyValueStoreSQLite.actor.cpp | 12 +++---- fdbserver/LocalConfiguration.actor.cpp | 6 ++-- fdbserver/MetricLogger.actor.cpp | 2 +- fdbserver/Ratekeeper.actor.cpp | 10 +++--- fdbserver/SkipList.cpp | 4 +-- fdbserver/StorageCache.actor.cpp | 22 ++++++------- fdbserver/masterserver.actor.cpp | 16 +++++----- .../workloads/AsyncFileCorrectness.actor.cpp | 2 +- fdbserver/workloads/AsyncFileWrite.actor.cpp | 2 +- flow/Net2.actor.cpp | 19 +++++------ flow/Profiler.actor.cpp | 2 +- flow/Trace.cpp | 21 ++++++------ flow/Tracing.actor.cpp | 2 +- 26 files changed, 142 insertions(+), 141 deletions(-) diff --git a/fdbclient/BackupContainerLocalDirectory.actor.cpp b/fdbclient/BackupContainerLocalDirectory.actor.cpp index b89d085a64..0a397f40c8 100644 --- a/fdbclient/BackupContainerLocalDirectory.actor.cpp +++ b/fdbclient/BackupContainerLocalDirectory.actor.cpp @@ -31,7 +31,8 @@ namespace { class BackupFile : public IBackupFile, ReferenceCounted { public: BackupFile(const std::string& fileName, Reference file, const std::string& finalFullPath) - : IBackupFile(fileName), m_file(file), m_finalFullPath(finalFullPath), m_writeOffset(0), m_blockSize(CLIENT_KNOBS->BACKUP_LOCAL_FILE_WRITE_BLOCK) { + : IBackupFile(fileName), m_file(file), m_writeOffset(0), m_finalFullPath(finalFullPath), + m_blockSize(CLIENT_KNOBS->BACKUP_LOCAL_FILE_WRITE_BLOCK) { if (BUGGIFY) { m_blockSize = deterministicRandom()->randomInt(100, 20000); } diff --git a/fdbclient/DatabaseBackupAgent.actor.cpp b/fdbclient/DatabaseBackupAgent.actor.cpp index a8de6819dd..a7a9c7fbc3 100644 --- a/fdbclient/DatabaseBackupAgent.actor.cpp +++ b/fdbclient/DatabaseBackupAgent.actor.cpp @@ -44,28 +44,29 @@ const Key DatabaseBackupAgent::keyDatabasesInSync = LiteralStringRef("databases_ const int DatabaseBackupAgent::LATEST_DR_VERSION = 1; DatabaseBackupAgent::DatabaseBackupAgent() - : subspace(Subspace(databaseBackupPrefixRange.begin)), tagNames(subspace.get(BackupAgentBase::keyTagName)), - states(subspace.get(BackupAgentBase::keyStates)), config(subspace.get(BackupAgentBase::keyConfig)), - errors(subspace.get(BackupAgentBase::keyErrors)), ranges(subspace.get(BackupAgentBase::keyRanges)), + : subspace(Subspace(databaseBackupPrefixRange.begin)), states(subspace.get(BackupAgentBase::keyStates)), + config(subspace.get(BackupAgentBase::keyConfig)), errors(subspace.get(BackupAgentBase::keyErrors)), + ranges(subspace.get(BackupAgentBase::keyRanges)), tagNames(subspace.get(BackupAgentBase::keyTagName)), + sourceStates(subspace.get(BackupAgentBase::keySourceStates)), + sourceTagNames(subspace.get(BackupAgentBase::keyTagName)), taskBucket(new TaskBucket(subspace.get(BackupAgentBase::keyTasks), AccessSystemKeys::True, PriorityBatch::False, LockAware::True)), - futureBucket(new FutureBucket(subspace.get(BackupAgentBase::keyFutures), AccessSystemKeys::True, LockAware::True)), - sourceStates(subspace.get(BackupAgentBase::keySourceStates)), - sourceTagNames(subspace.get(BackupAgentBase::keyTagName)) {} + futureBucket(new FutureBucket(subspace.get(BackupAgentBase::keyFutures), AccessSystemKeys::True, LockAware::True)) { +} DatabaseBackupAgent::DatabaseBackupAgent(Database src) - : subspace(Subspace(databaseBackupPrefixRange.begin)), tagNames(subspace.get(BackupAgentBase::keyTagName)), - states(subspace.get(BackupAgentBase::keyStates)), config(subspace.get(BackupAgentBase::keyConfig)), - errors(subspace.get(BackupAgentBase::keyErrors)), ranges(subspace.get(BackupAgentBase::keyRanges)), + : subspace(Subspace(databaseBackupPrefixRange.begin)), states(subspace.get(BackupAgentBase::keyStates)), + config(subspace.get(BackupAgentBase::keyConfig)), errors(subspace.get(BackupAgentBase::keyErrors)), + ranges(subspace.get(BackupAgentBase::keyRanges)), tagNames(subspace.get(BackupAgentBase::keyTagName)), + sourceStates(subspace.get(BackupAgentBase::keySourceStates)), + sourceTagNames(subspace.get(BackupAgentBase::keyTagName)), taskBucket(new TaskBucket(subspace.get(BackupAgentBase::keyTasks), AccessSystemKeys::True, PriorityBatch::False, LockAware::True)), - futureBucket(new FutureBucket(subspace.get(BackupAgentBase::keyFutures), AccessSystemKeys::True, LockAware::True)), - sourceStates(subspace.get(BackupAgentBase::keySourceStates)), - sourceTagNames(subspace.get(BackupAgentBase::keyTagName)) { + futureBucket(new FutureBucket(subspace.get(BackupAgentBase::keyFutures), AccessSystemKeys::True, LockAware::True)) { taskBucket->src = src; } diff --git a/fdbclient/MultiVersionTransaction.actor.cpp b/fdbclient/MultiVersionTransaction.actor.cpp index afc92bfec0..acab668e7c 100644 --- a/fdbclient/MultiVersionTransaction.actor.cpp +++ b/fdbclient/MultiVersionTransaction.actor.cpp @@ -396,7 +396,7 @@ void loadClientFunction(T* fp, void* lib, std::string libPath, const char* funct } DLApi::DLApi(std::string fdbCPath, bool unlinkOnLoad) - : api(new FdbCApi()), fdbCPath(fdbCPath), unlinkOnLoad(unlinkOnLoad), networkSetup(false) {} + : fdbCPath(fdbCPath), api(new FdbCApi()), unlinkOnLoad(unlinkOnLoad), networkSetup(false) {} // Loads client API functions (definitions are in FdbCApi struct) void DLApi::init() { @@ -993,8 +993,8 @@ ThreadFuture MultiVersionDatabase::getServerProtocol(Optional

versionMonitorDb) - : clusterFilePath(clusterFilePath), versionMonitorDb(versionMonitorDb), - dbVar(new ThreadSafeAsyncVar>(Reference(nullptr))) {} + : dbVar(new ThreadSafeAsyncVar>(Reference(nullptr))), + clusterFilePath(clusterFilePath), versionMonitorDb(versionMonitorDb) {} // Adds a client (local or externally loaded) that can be used to connect to the cluster void MultiVersionDatabase::DatabaseState::addClient(Reference client) { @@ -1855,8 +1855,8 @@ void MultiVersionApi::loadEnvironmentVariableNetworkOptions() { } MultiVersionApi::MultiVersionApi() - : bypassMultiClientApi(false), networkStartSetup(false), networkSetup(false), callbackOnMainThread(true), - externalClient(false), localClientDisabled(false), apiVersion(0), envOptionsLoaded(false), threadCount(0) {} + : callbackOnMainThread(true), localClientDisabled(false), networkStartSetup(false), networkSetup(false), + bypassMultiClientApi(false), externalClient(false), apiVersion(0), threadCount(0), envOptionsLoaded(false) {} MultiVersionApi* MultiVersionApi::api = new MultiVersionApi(); diff --git a/fdbclient/SpecialKeySpace.actor.cpp b/fdbclient/SpecialKeySpace.actor.cpp index fc6bf0b2fe..441699df2d 100644 --- a/fdbclient/SpecialKeySpace.actor.cpp +++ b/fdbclient/SpecialKeySpace.actor.cpp @@ -248,8 +248,9 @@ ACTOR Future normalizeKeySelectorActor(SpecialKeySpace* sks, } SpecialKeySpace::SpecialKeySpace(KeyRef spaceStartKey, KeyRef spaceEndKey, bool testOnly) - : range(KeyRangeRef(spaceStartKey, spaceEndKey)), readImpls(nullptr, spaceEndKey), writeImpls(nullptr, spaceEndKey), - modules(testOnly ? SpecialKeySpace::MODULE::TESTONLY : SpecialKeySpace::MODULE::UNKNOWN, spaceEndKey) { + : readImpls(nullptr, spaceEndKey), + modules(testOnly ? SpecialKeySpace::MODULE::TESTONLY : SpecialKeySpace::MODULE::UNKNOWN, spaceEndKey), + writeImpls(nullptr, spaceEndKey), range(KeyRangeRef(spaceStartKey, spaceEndKey)) { // Default begin of KeyRangeMap is Key(), insert the range to update start key readImpls.insert(range, nullptr); writeImpls.insert(range, nullptr); diff --git a/fdbclient/TaskBucket.actor.cpp b/fdbclient/TaskBucket.actor.cpp index f7e4ed7e24..97c58efa9a 100644 --- a/fdbclient/TaskBucket.actor.cpp +++ b/fdbclient/TaskBucket.actor.cpp @@ -873,13 +873,14 @@ TaskBucket::TaskBucket(const Subspace& subspace, AccessSystemKeys sysAccess, PriorityBatch priorityBatch, LockAware lockAware) - : prefix(subspace), active(prefix.get(LiteralStringRef("ac"))), available(prefix.get(LiteralStringRef("av"))), + : cc("TaskBucket"), dbgid(deterministicRandom()->randomUniqueID()), + dispatchSlotChecksStarted("DispatchSlotChecksStarted", cc), dispatchErrors("DispatchErrors", cc), + dispatchDoTasks("DispatchDoTasks", cc), dispatchEmptyTasks("DispatchEmptyTasks", cc), + dispatchSlotChecksComplete("DispatchSlotChecksComplete", cc), prefix(subspace), + active(prefix.get(LiteralStringRef("ac"))), available(prefix.get(LiteralStringRef("av"))), available_prioritized(prefix.get(LiteralStringRef("avp"))), timeouts(prefix.get(LiteralStringRef("to"))), - pauseKey(prefix.pack(LiteralStringRef("pause"))), timeout(CLIENT_KNOBS->TASKBUCKET_TIMEOUT_VERSIONS), - system_access(sysAccess), priority_batch(priorityBatch), lockAware(lockAware), cc("TaskBucket"), - dbgid(deterministicRandom()->randomUniqueID()), dispatchSlotChecksStarted("DispatchSlotChecksStarted", cc), - dispatchErrors("DispatchErrors", cc), dispatchDoTasks("DispatchDoTasks", cc), - dispatchEmptyTasks("DispatchEmptyTasks", cc), dispatchSlotChecksComplete("DispatchSlotChecksComplete", cc) {} + timeout(CLIENT_KNOBS->TASKBUCKET_TIMEOUT_VERSIONS), pauseKey(prefix.pack(LiteralStringRef("pause"))), + system_access(sysAccess), priority_batch(priorityBatch), lockAware(lockAware) {} TaskBucket::~TaskBucket() {} diff --git a/fdbrpc/FlowTransport.actor.cpp b/fdbrpc/FlowTransport.actor.cpp index d44483da12..d6af0d3eb7 100644 --- a/fdbrpc/FlowTransport.actor.cpp +++ b/fdbrpc/FlowTransport.actor.cpp @@ -340,9 +340,8 @@ ACTOR Future pingLatencyLogger(TransportData* self) { } TransportData::TransportData(uint64_t transportId) - : endpoints(WLTOKEN_COUNTS), endpointNotFoundReceiver(endpoints), pingReceiver(endpoints), - warnAlwaysForLargePacket(true), lastIncompatibleMessage(0), transportId(transportId), - numIncompatibleConnections(0) { + : warnAlwaysForLargePacket(true), endpoints(WLTOKEN_COUNTS), endpointNotFoundReceiver(endpoints), + pingReceiver(endpoints), numIncompatibleConnections(0), lastIncompatibleMessage(0), transportId(transportId) { degraded = makeReference>(false); pingLogger = pingLatencyLogger(this); } @@ -795,13 +794,14 @@ ACTOR Future connectionKeeper(Reference self, } Peer::Peer(TransportData* transport, NetworkAddress const& destination) - : transport(transport), destination(destination), outgoingConnectionIdle(true), lastConnectTime(0.0), - 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), timeoutCount(0), lastLoggedTime(0.0), connectOutgoingCount(0), connectIncomingCount(0), - connectFailedCount(0), connectLatencies(destination.isPublic() ? FLOW_KNOBS->NETWORK_CONNECT_SAMPLE_AMOUNT : 1), - protocolVersion(Reference>>(new AsyncVar>())) { + : transport(transport), destination(destination), compatible(true), outgoingConnectionIdle(true), + lastConnectTime(0.0), reconnectionDelay(FLOW_KNOBS->INITIAL_RECONNECTION_TIME), peerReferences(-1), + outstandingReplies(0), pingLatencies(destination.isPublic() ? FLOW_KNOBS->PING_SAMPLE_AMOUNT : 1), + lastLoggedTime(0.0), lastLoggedBytesReceived(0), lastLoggedBytesSent(0), timeoutCount(0), + incompatibleProtocolVersionNewer(false), bytesReceived(0), bytesSent(0), lastDataPacketSentTime(now()), + protocolVersion(Reference>>(new AsyncVar>())), + connectOutgoingCount(0), connectIncomingCount(0), connectFailedCount(0), + connectLatencies(destination.isPublic() ? FLOW_KNOBS->NETWORK_CONNECT_SAMPLE_AMOUNT : 1) { IFailureMonitor::failureMonitor().setStatus(destination, FailureStatus(false)); } diff --git a/fdbserver/BackupWorker.actor.cpp b/fdbserver/BackupWorker.actor.cpp index d6bd6a0ebb..fb43bd18cc 100644 --- a/fdbserver/BackupWorker.actor.cpp +++ b/fdbserver/BackupWorker.actor.cpp @@ -241,8 +241,8 @@ struct BackupData { : myId(id), tag(req.routerTag), totalTags(req.totalTags), startVersion(req.startVersion), endVersion(req.endVersion), recruitedEpoch(req.recruitedEpoch), backupEpoch(req.backupEpoch), minKnownCommittedVersion(invalidVersion), savedVersion(req.startVersion - 1), popVersion(req.startVersion - 1), - cc("BackupWorker", myId.toString()), pulledVersion(0), paused(false), - lock(new FlowLock(SERVER_KNOBS->BACKUP_LOCK_BYTES)) { + pulledVersion(0), paused(false), lock(new FlowLock(SERVER_KNOBS->BACKUP_LOCK_BYTES)), + cc("BackupWorker", myId.toString()) { cx = openDBOnServer(db, TaskPriority::DefaultEndpoint, LockAware::True); specialCounter(cc, "SavedVersion", [this]() { return this->savedVersion; }); diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 97ef92b1e5..971eede909 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -128,15 +128,15 @@ public: std::map> clientStatus; DBInfo() - : masterRegistrationCount(0), recoveryStalled(false), forceRecovery(false), unfinishedRecoveries(0), - logGenerations(0), cachePopulated(false), clientInfo(new AsyncVar()), dbInfoCount(0), - serverInfo(new AsyncVar()), db(DatabaseContext::create(clientInfo, - Future(), - LocalityData(), - EnableLocalityLoadBalance::True, - TaskPriority::DefaultEndpoint, - LockAware::True)) // SOMEDAY: Locality! - {} + : clientInfo(new AsyncVar()), serverInfo(new AsyncVar()), + masterRegistrationCount(0), dbInfoCount(0), recoveryStalled(false), forceRecovery(false), + db(DatabaseContext::create(clientInfo, + Future(), + LocalityData(), + EnableLocalityLoadBalance::True, + TaskPriority::DefaultEndpoint, + LockAware::True)), // SOMEDAY: Locality! + unfinishedRecoveries(0), logGenerations(0), cachePopulated(false) {} void setDistributor(const DataDistributorInterface& interf) { auto newInfo = serverInfo->get(); @@ -1431,12 +1431,12 @@ public: bool degraded = false; RoleFitness(int bestFit, int worstFit, int count, ProcessClass::ClusterRole role) - : bestFit((ProcessClass::Fitness)bestFit), worstFit((ProcessClass::Fitness)worstFit), count(count), - role(role) {} + : bestFit((ProcessClass::Fitness)bestFit), worstFit((ProcessClass::Fitness)worstFit), role(role), + count(count) {} RoleFitness(int fitness, int count, ProcessClass::ClusterRole role) - : bestFit((ProcessClass::Fitness)fitness), worstFit((ProcessClass::Fitness)fitness), count(count), - role(role) {} + : bestFit((ProcessClass::Fitness)fitness), worstFit((ProcessClass::Fitness)fitness), role(role), + count(count) {} RoleFitness() : bestFit(ProcessClass::NeverAssign), worstFit(ProcessClass::NeverAssign), role(ProcessClass::NoRole), @@ -3059,9 +3059,9 @@ public: ClusterControllerData(ClusterControllerFullInterface const& ccInterface, LocalityData const& locality, ServerCoordinators const& coordinators) - : clusterControllerProcessId(locality.processId()), clusterControllerDcId(locality.dcId()), id(ccInterface.id()), - ac(false), outstandingRequestChecker(Void()), outstandingRemoteRequestChecker(Void()), gotProcessClasses(false), - gotFullyRecoveredConfig(false), startTime(now()), goodRecruitmentTime(Never()), + : gotProcessClasses(false), gotFullyRecoveredConfig(false), clusterControllerProcessId(locality.processId()), + clusterControllerDcId(locality.dcId()), id(ccInterface.id()), ac(false), outstandingRequestChecker(Void()), + outstandingRemoteRequestChecker(Void()), startTime(now()), goodRecruitmentTime(Never()), goodRemoteRecruitmentTime(Never()), datacenterVersionDifference(0), versionDifferenceUpdated(false), recruitingDistributor(false), recruitRatekeeper(false), clusterControllerMetrics("ClusterController", id.toString()), diff --git a/fdbserver/ConfigBroadcaster.actor.cpp b/fdbserver/ConfigBroadcaster.actor.cpp index e386bb4dc5..a5beb99e6a 100644 --- a/fdbserver/ConfigBroadcaster.actor.cpp +++ b/fdbserver/ConfigBroadcaster.actor.cpp @@ -203,7 +203,7 @@ class ConfigBroadcasterImpl { } ConfigBroadcasterImpl() - : id(deterministicRandom()->randomUniqueID()), lastCompactedVersion(0), mostRecentVersion(0), + : mostRecentVersion(0), lastCompactedVersion(0), id(deterministicRandom()->randomUniqueID()), cc("ConfigBroadcaster"), compactRequest("CompactRequest", cc), successfulChangeRequest("SuccessfulChangeRequest", cc), failedChangeRequest("FailedChangeRequest", cc), snapshotRequest("SnapshotRequest", cc) { diff --git a/fdbserver/ConfigDatabaseUnitTests.actor.cpp b/fdbserver/ConfigDatabaseUnitTests.actor.cpp index 39d7be0ac1..4bd32ad5ea 100644 --- a/fdbserver/ConfigDatabaseUnitTests.actor.cpp +++ b/fdbserver/ConfigDatabaseUnitTests.actor.cpp @@ -241,8 +241,8 @@ class BroadcasterToLocalConfigEnvironment { public: BroadcasterToLocalConfigEnvironment(std::string const& dataDir, std::string const& configPath) - : broadcaster(ConfigFollowerInterface{}), cbfi(makeReference>()), - readFrom(dataDir, configPath, {}) {} + : readFrom(dataDir, configPath, {}), cbfi(makeReference>()), + broadcaster(ConfigFollowerInterface{}) {} Future setup() { return setup(this); } @@ -371,8 +371,9 @@ class TransactionToLocalConfigEnvironment { public: TransactionToLocalConfigEnvironment(std::string const& dataDir, std::string const& configPath) - : writeTo(dataDir), readFrom(dataDir, configPath, {}), broadcaster(writeTo.getFollowerInterface()), - cbfi(makeReference>()) {} + : writeTo(dataDir), readFrom(dataDir, configPath, {}), + cbfi(makeReference>()), broadcaster(writeTo.getFollowerInterface()) { + } Future setup() { return setup(this); } diff --git a/fdbserver/DataDistributionQueue.actor.cpp b/fdbserver/DataDistributionQueue.actor.cpp index 6f55c39438..aa623f3361 100644 --- a/fdbserver/DataDistributionQueue.actor.cpp +++ b/fdbserver/DataDistributionQueue.actor.cpp @@ -50,7 +50,7 @@ struct RelocateData { TraceInterval interval; RelocateData() - : startTime(-1), priority(-1), boundaryPriority(-1), healthPriority(-1), workFactor(0), wantsNewServers(false), + : priority(-1), boundaryPriority(-1), healthPriority(-1), startTime(-1), workFactor(0), wantsNewServers(false), interval("QueuedRelocation") {} explicit RelocateData(RelocateShard const& rs) : keys(rs.keys), priority(rs.priority), boundaryPriority(isBoundaryPriority(rs.priority) ? rs.priority : -1), @@ -448,14 +448,14 @@ struct DDQueueData { FutureStream input, PromiseStream getShardMetrics, double* lastLimited) - : activeRelocations(0), queuedRelocations(0), bytesWritten(0), teamCollections(teamCollections), - shardsAffectedByTeamFailure(sABTF), getAverageShardBytes(getAverageShardBytes), distributorId(mid), lock(lock), - cx(cx), teamSize(teamSize), singleRegionTeamSize(singleRegionTeamSize), output(output), input(input), - getShardMetrics(getShardMetrics), startMoveKeysParallelismLock(SERVER_KNOBS->DD_MOVE_KEYS_PARALLELISM), + : distributorId(mid), lock(lock), cx(cx), teamCollections(teamCollections), shardsAffectedByTeamFailure(sABTF), + getAverageShardBytes(getAverageShardBytes), + startMoveKeysParallelismLock(SERVER_KNOBS->DD_MOVE_KEYS_PARALLELISM), finishMoveKeysParallelismLock(SERVER_KNOBS->DD_MOVE_KEYS_PARALLELISM), - fetchSourceLock(new FlowLock(SERVER_KNOBS->DD_FETCH_SOURCE_PARALLELISM)), lastLimited(lastLimited), - suppressIntervals(0), lastInterval(0), unhealthyRelocations(0), - rawProcessingUnhealthy(new AsyncVar(false)) {} + fetchSourceLock(new FlowLock(SERVER_KNOBS->DD_FETCH_SOURCE_PARALLELISM)), activeRelocations(0), + queuedRelocations(0), bytesWritten(0), teamSize(teamSize), singleRegionTeamSize(singleRegionTeamSize), + output(output), input(input), getShardMetrics(getShardMetrics), lastLimited(lastLimited), lastInterval(0), + suppressIntervals(0), rawProcessingUnhealthy(new AsyncVar(false)), unhealthyRelocations(0) {} void validate() { if (EXPENSIVE_VALIDATION) { diff --git a/fdbserver/DataDistributionTracker.actor.cpp b/fdbserver/DataDistributionTracker.actor.cpp index 40ca28aa08..e27dbf4e56 100644 --- a/fdbserver/DataDistributionTracker.actor.cpp +++ b/fdbserver/DataDistributionTracker.actor.cpp @@ -123,10 +123,10 @@ struct DataDistributionTracker { Reference> anyZeroHealthyTeams, KeyRangeMap& shards, bool& trackerCancelled) - : cx(cx), distributorId(distributorId), dbSizeEstimate(new AsyncVar()), systemSizeEstimate(0), - maxShardSize(new AsyncVar>()), sizeChanges(false), readyToStart(readyToStart), output(output), - shardsAffectedByTeamFailure(shardsAffectedByTeamFailure), anyZeroHealthyTeams(anyZeroHealthyTeams), - shards(shards), trackerCancelled(trackerCancelled) {} + : cx(cx), distributorId(distributorId), shards(shards), sizeChanges(false), systemSizeEstimate(0), + dbSizeEstimate(new AsyncVar()), maxShardSize(new AsyncVar>()), output(output), + shardsAffectedByTeamFailure(shardsAffectedByTeamFailure), readyToStart(readyToStart), + anyZeroHealthyTeams(anyZeroHealthyTeams), trackerCancelled(trackerCancelled) {} ~DataDistributionTracker() { trackerCancelled = true; diff --git a/fdbserver/DiskQueue.actor.cpp b/fdbserver/DiskQueue.actor.cpp index a7a5402374..372ad1a135 100644 --- a/fdbserver/DiskQueue.actor.cpp +++ b/fdbserver/DiskQueue.actor.cpp @@ -168,11 +168,11 @@ private: class RawDiskQueue_TwoFiles : public Tracked { public: RawDiskQueue_TwoFiles(std::string basename, std::string fileExtension, UID dbgid, int64_t fileSizeWarningLimit) - : basename(basename), fileExtension(fileExtension), onError(delayed(error.getFuture())), - onStopped(stopped.getFuture()), readingFile(-1), readingPage(-1), writingPos(-1), dbgid(dbgid), - dbg_file0BeginSeq(0), fileExtensionBytes(SERVER_KNOBS->DISK_QUEUE_FILE_EXTENSION_BYTES), - fileShrinkBytes(SERVER_KNOBS->DISK_QUEUE_FILE_SHRINK_BYTES), readingBuffer(dbgid), readyToPush(Void()), - fileSizeWarningLimit(fileSizeWarningLimit), lastCommit(Void()), isFirstCommit(true) { + : basename(basename), fileExtension(fileExtension), dbgid(dbgid), dbg_file0BeginSeq(0), + fileSizeWarningLimit(fileSizeWarningLimit), onError(delayed(error.getFuture())), onStopped(stopped.getFuture()), + readyToPush(Void()), lastCommit(Void()), isFirstCommit(true), readingBuffer(dbgid), readingFile(-1), + readingPage(-1), writingPos(-1), fileExtensionBytes(SERVER_KNOBS->DISK_QUEUE_FILE_EXTENSION_BYTES), + fileShrinkBytes(SERVER_KNOBS->DISK_QUEUE_FILE_SHRINK_BYTES) { if (BUGGIFY) fileExtensionBytes = _PAGE_SIZE * deterministicRandom()->randomSkewedUInt32(1, 10 << 10); if (BUGGIFY) @@ -878,9 +878,9 @@ public: DiskQueueVersion diskQueueVersion, int64_t fileSizeWarningLimit) : rawQueue(new RawDiskQueue_TwoFiles(basename, fileExtension, dbgid, fileSizeWarningLimit)), dbgid(dbgid), - diskQueueVersion(diskQueueVersion), anyPopped(false), nextPageSeq(0), poppedSeq(0), lastPoppedSeq(0), - nextReadLocation(-1), readBufPage(nullptr), readBufPos(0), pushed_page_buffer(nullptr), recovered(false), - initialized(false), lastCommittedSeq(-1), warnAlwaysForMemory(true) {} + diskQueueVersion(diskQueueVersion), anyPopped(false), warnAlwaysForMemory(true), nextPageSeq(0), poppedSeq(0), + lastPoppedSeq(0), lastCommittedSeq(-1), pushed_page_buffer(nullptr), recovered(false), initialized(false), + nextReadLocation(-1), readBufPage(nullptr), readBufPos(0) {} location push(StringRef contents) override { ASSERT(recovered); diff --git a/fdbserver/KeyValueStoreSQLite.actor.cpp b/fdbserver/KeyValueStoreSQLite.actor.cpp index 6e3043f3f3..e52ca14198 100644 --- a/fdbserver/KeyValueStoreSQLite.actor.cpp +++ b/fdbserver/KeyValueStoreSQLite.actor.cpp @@ -681,7 +681,7 @@ struct SQLiteTransaction { struct IntKeyCursor { SQLiteDB& db; BtCursor* cursor; - IntKeyCursor(SQLiteDB& db, int table, bool write) : cursor(0), db(db) { + IntKeyCursor(SQLiteDB& db, int table, bool write) : db(db), cursor(nullptr) { cursor = (BtCursor*)new char[sqlite3BtreeCursorSize()]; sqlite3BtreeCursorZero(cursor); db.checkError("BtreeCursor", sqlite3BtreeCursor(db.btree, table, write, nullptr, cursor)); @@ -705,7 +705,7 @@ struct RawCursor { operator bool() const { return valid; } - RawCursor(SQLiteDB& db, int table, bool write) : cursor(0), db(db), valid(false) { + RawCursor(SQLiteDB& db, int table, bool write) : db(db), cursor(nullptr), valid(false) { keyInfo.db = db.db; keyInfo.enc = db.db->aDb[0].pSchema->enc; keyInfo.aColl[0] = db.db->pDfltColl; @@ -1732,9 +1732,9 @@ private: volatile int64_t& freeListPages, UID dbgid, vector>* pReadThreads) - : kvs(kvs), conn(kvs->filename, isBtreeV2, isBtreeV2), commits(), setsThisCommit(), freeTableEmpty(false), - writesComplete(writesComplete), springCleaningStats(springCleaningStats), diskBytesUsed(diskBytesUsed), - freeListPages(freeListPages), cursor(nullptr), dbgid(dbgid), readThreads(*pReadThreads), + : kvs(kvs), conn(kvs->filename, isBtreeV2, isBtreeV2), cursor(nullptr), commits(), setsThisCommit(), + freeTableEmpty(false), writesComplete(writesComplete), springCleaningStats(springCleaningStats), + diskBytesUsed(diskBytesUsed), freeListPages(freeListPages), dbgid(dbgid), readThreads(*pReadThreads), checkAllChecksumsOnOpen(checkAllChecksumsOnOpen), checkIntegrityOnOpen(checkIntegrityOnOpen) {} ~Writer() override { TraceEvent("KVWriterDestroying", dbgid); @@ -2109,7 +2109,7 @@ KeyValueStoreSQLite::KeyValueStoreSQLite(std::string const& filename, KeyValueStoreType storeType, bool checkChecksums, bool checkIntegrity) - : type(storeType), filename(filename), logID(id), readThreads(CoroThreadPool::createThreadPool()), + : type(storeType), logID(id), filename(filename), readThreads(CoroThreadPool::createThreadPool()), writeThread(CoroThreadPool::createThreadPool()), readsRequested(0), writesRequested(0), writesComplete(0), diskBytesUsed(0), freeListPages(0) { TraceEvent(SevDebug, "KeyValueStoreSQLiteCreate").detail("Filename", filename); diff --git a/fdbserver/LocalConfiguration.actor.cpp b/fdbserver/LocalConfiguration.actor.cpp index 238db7041e..f375e6f85e 100644 --- a/fdbserver/LocalConfiguration.actor.cpp +++ b/fdbserver/LocalConfiguration.actor.cpp @@ -326,9 +326,9 @@ public: std::string const& configPath, std::map const& manualKnobOverrides, IsTest isTest) - : id(deterministicRandom()->randomUniqueID()), kvStore(dataFolder, id, "localconf-"), cc("LocalConfiguration"), - broadcasterChanges("BroadcasterChanges", cc), snapshots("Snapshots", cc), - changeRequestsFetched("ChangeRequestsFetched", cc), mutations("Mutations", cc), configKnobOverrides(configPath), + : id(deterministicRandom()->randomUniqueID()), kvStore(dataFolder, id, "localconf-"), + configKnobOverrides(configPath), cc("LocalConfiguration"), broadcasterChanges("BroadcasterChanges", cc), + snapshots("Snapshots", cc), changeRequestsFetched("ChangeRequestsFetched", cc), mutations("Mutations", cc), manualKnobOverrides(manualKnobOverrides) { if (isTest) { testKnobCollection = diff --git a/fdbserver/MetricLogger.actor.cpp b/fdbserver/MetricLogger.actor.cpp index aee9ea67d6..284c782626 100644 --- a/fdbserver/MetricLogger.actor.cpp +++ b/fdbserver/MetricLogger.actor.cpp @@ -29,7 +29,7 @@ struct MetricsRule { MetricsRule(bool enabled = false, int minLevel = 0, StringRef const& name = StringRef()) - : enabled(enabled), minLevel(minLevel), namePattern(name) {} + : namePattern(name), enabled(enabled), minLevel(minLevel) {} Standalone typePattern; Standalone namePattern; diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index a13f9583be..3fcb63ef4e 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -515,8 +515,7 @@ struct RatekeeperLimits { int64_t logSpringBytes, double maxVersionDifference, int64_t durabilityLagTargetVersions) - : priority(priority), tpsLimit(std::numeric_limits::infinity()), - tpsLimitMetric(StringRef("Ratekeeper.TPSLimit" + context)), + : tpsLimit(std::numeric_limits::infinity()), tpsLimitMetric(StringRef("Ratekeeper.TPSLimit" + context)), reasonMetric(StringRef("Ratekeeper.Reason" + context)), storageTargetBytes(storageTargetBytes), storageSpringBytes(storageSpringBytes), logTargetBytes(logTargetBytes), logSpringBytes(logSpringBytes), maxVersionDifference(maxVersionDifference), @@ -524,7 +523,8 @@ struct RatekeeperLimits { durabilityLagTargetVersions + SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS), // The read transaction life versions are expected to not // be durable on the storage servers - durabilityLagLimit(std::numeric_limits::infinity()), lastDurabilityLag(0), context(context) {} + lastDurabilityLag(0), durabilityLagLimit(std::numeric_limits::infinity()), priority(priority), + context(context) {} }; struct GrvProxyInfo { @@ -536,7 +536,7 @@ struct GrvProxyInfo { double lastTagPushTime; GrvProxyInfo() - : totalTransactions(0), batchTransactions(0), lastUpdateTime(0), lastThrottledTagChangeId(0), lastTagPushTime(0) { + : totalTransactions(0), batchTransactions(0), lastThrottledTagChangeId(0), lastUpdateTime(0), lastTagPushTime(0) { } }; @@ -577,7 +577,7 @@ struct RatekeeperData { smoothBatchReleasedTransactions(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothTotalDurableBytes(SERVER_KNOBS->SLOW_SMOOTHING_AMOUNT), actualTpsMetric(LiteralStringRef("Ratekeeper.ActualTPS")), lastWarning(0), lastSSListFetchedTimestamp(now()), - throttledTagChangeId(0), lastBusiestCommitTagPick(0), + lastBusiestCommitTagPick(0), throttledTagChangeId(0), normalLimits(TransactionPriority::DEFAULT, "", SERVER_KNOBS->TARGET_BYTES_PER_STORAGE_SERVER, diff --git a/fdbserver/SkipList.cpp b/fdbserver/SkipList.cpp index 86489f3850..d90e649b58 100644 --- a/fdbserver/SkipList.cpp +++ b/fdbserver/SkipList.cpp @@ -92,7 +92,7 @@ struct KeyInfo { KeyInfo() = default; KeyInfo(StringRef key, bool begin, bool write, int transaction, int* pIndex) - : key(key), begin(begin), write(write), transaction(transaction), pIndex(pIndex) {} + : key(key), pIndex(pIndex), begin(begin), write(write), transaction(transaction) {} }; force_inline int extra_ordering(const KeyInfo& ki) { @@ -343,7 +343,7 @@ public: StringRef value; Finger() = default; - Finger(Node* header, const StringRef& ptr) : value(ptr), x(header) {} + Finger(Node* header, const StringRef& ptr) : x(header), value(ptr) {} void init(const StringRef& value, Node* header) { this->value = value; diff --git a/fdbserver/StorageCache.actor.cpp b/fdbserver/StorageCache.actor.cpp index 8f44f054d6..7fa81738c6 100644 --- a/fdbserver/StorageCache.actor.cpp +++ b/fdbserver/StorageCache.actor.cpp @@ -224,14 +224,14 @@ public: // LatencyBands readLatencyBands; Counters(StorageCacheData* self) - : cc("StorageCacheServer", self->thisServerID.toString()), getKeyQueries("GetKeyQueries", cc), - getValueQueries("GetValueQueries", cc), getRangeQueries("GetRangeQueries", cc), - allQueries("QueryQueue", cc), finishedQueries("FinishedQueries", cc), rowsQueried("RowsQueried", cc), - bytesQueried("BytesQueried", cc), bytesInput("BytesInput", cc), bytesFetched("BytesFetched", cc), - mutationBytes("MutationBytes", cc), mutations("Mutations", cc), setMutations("SetMutations", cc), - clearRangeMutations("ClearRangeMutations", cc), atomicMutations("AtomicMutations", cc), - updateBatches("UpdateBatches", cc), updateVersions("UpdateVersions", cc), loops("Loops", cc), - readsRejected("ReadsRejected", cc) { + : cc("StorageCacheServer", self->thisServerID.toString()), allQueries("QueryQueue", cc), + getKeyQueries("GetKeyQueries", cc), getValueQueries("GetValueQueries", cc), + getRangeQueries("GetRangeQueries", cc), finishedQueries("FinishedQueries", cc), + rowsQueried("RowsQueried", cc), bytesQueried("BytesQueried", cc), bytesInput("BytesInput", cc), + bytesFetched("BytesFetched", cc), mutationBytes("MutationBytes", cc), mutations("Mutations", cc), + setMutations("SetMutations", cc), clearRangeMutations("ClearRangeMutations", cc), + atomicMutations("AtomicMutations", cc), updateBatches("UpdateBatches", cc), + updateVersions("UpdateVersions", cc), loops("Loops", cc), readsRejected("ReadsRejected", cc) { specialCounter(cc, "LastTLogVersion", [self]() { return self->lastTLogVersion; }); specialCounter(cc, "Version", [self]() { return self->version.get(); }); specialCounter(cc, "VersionLag", [self]() { return self->versionLag; }); @@ -1542,7 +1542,7 @@ ACTOR Future fetchKeys(StorageCacheData* data, AddingCacheRange* cacheRang }; AddingCacheRange::AddingCacheRange(StorageCacheData* server, KeyRangeRef const& keys) - : server(server), keys(keys), transferredVersion(invalidVersion), phase(WaitPrevious) { + : keys(keys), server(server), transferredVersion(invalidVersion), phase(WaitPrevious) { fetchClient = fetchKeys(server, this); } @@ -1704,9 +1704,9 @@ void cacheWarmup(StorageCacheData* data, const KeyRangeRef& keys, bool nowAssign class StorageCacheUpdater { public: StorageCacheUpdater() - : fromVersion(invalidVersion), currentVersion(invalidVersion), processedCacheStartKey(false) {} + : currentVersion(invalidVersion), fromVersion(invalidVersion), processedCacheStartKey(false) {} StorageCacheUpdater(Version currentVersion) - : fromVersion(currentVersion), currentVersion(currentVersion), processedCacheStartKey(false) {} + : currentVersion(invalidVersion), fromVersion(currentVersion), processedCacheStartKey(false) {} void applyMutation(StorageCacheData* data, MutationRef const& m, Version ver) { //TraceEvent("SCNewVersion", data->thisServerID).detail("VerWas", data->mutableData().latestVersion).detail("ChVer", ver); diff --git a/fdbserver/masterserver.actor.cpp b/fdbserver/masterserver.actor.cpp index 39cd0dd087..136ef3cb1e 100644 --- a/fdbserver/masterserver.actor.cpp +++ b/fdbserver/masterserver.actor.cpp @@ -95,8 +95,8 @@ public: ReusableCoordinatedState(ServerCoordinators const& coordinators, PromiseStream> const& addActor, UID const& dbgid) - : coordinators(coordinators), cstate(coordinators), addActor(addActor), dbgid(dbgid), finalWriteStarted(false), - previousWrite(Void()) {} + : finalWriteStarted(false), previousWrite(Void()), cstate(coordinators), coordinators(coordinators), + addActor(addActor), dbgid(dbgid) {} Future read() { return _read(this); } @@ -265,12 +265,12 @@ struct MasterData : NonCopyable, ReferenceCounted { : dbgid(myInterface.id()), lastEpochEnd(invalidVersion), recoveryTransactionVersion(invalidVersion), lastCommitTime(0), liveCommittedVersion(invalidVersion), databaseLocked(false), - minKnownCommittedVersion(invalidVersion), myInterface(myInterface), dbInfo(dbInfo), - cstate(coordinators, addActor, dbgid), coordinators(coordinators), clusterController(clusterController), - dbId(dbId), forceRecovery(forceRecovery), safeLocality(tagLocalityInvalid), primaryLocality(tagLocalityInvalid), - neverCreated(false), hasConfiguration(false), version(invalidVersion), lastVersionTime(0), - txnStateStore(nullptr), memoryLimit(2e9), registrationCount(0), addActor(addActor), - recruitmentStalled(makeReference>(false)), cc("Master", dbgid.toString()), + minKnownCommittedVersion(invalidVersion), hasConfiguration(false), coordinators(coordinators), + version(invalidVersion), lastVersionTime(0), txnStateStore(nullptr), memoryLimit(2e9), dbId(dbId), + myInterface(myInterface), clusterController(clusterController), cstate(coordinators, addActor, dbgid), + dbInfo(dbInfo), registrationCount(0), addActor(addActor), + recruitmentStalled(makeReference>(false)), forceRecovery(forceRecovery), neverCreated(false), + safeLocality(tagLocalityInvalid), primaryLocality(tagLocalityInvalid), cc("Master", dbgid.toString()), changeCoordinatorsRequests("ChangeCoordinatorsRequests", cc), getCommitVersionRequests("GetCommitVersionRequests", cc), backupWorkerDoneRequests("BackupWorkerDoneRequests", cc), diff --git a/fdbserver/workloads/AsyncFileCorrectness.actor.cpp b/fdbserver/workloads/AsyncFileCorrectness.actor.cpp index 4a7e8fd1ec..96dd3f2283 100644 --- a/fdbserver/workloads/AsyncFileCorrectness.actor.cpp +++ b/fdbserver/workloads/AsyncFileCorrectness.actor.cpp @@ -73,7 +73,7 @@ struct AsyncFileCorrectnessWorkload : public AsyncFileWorkload { PerfIntCounter numOperations; AsyncFileCorrectnessWorkload(WorkloadContext const& wcx) - : AsyncFileWorkload(wcx), success(true), numOperations("Num Operations"), memoryFile(nullptr) { + : AsyncFileWorkload(wcx), memoryFile(nullptr), success(true), numOperations("Num Operations") { maxOperationSize = getOption(options, LiteralStringRef("maxOperationSize"), 4096); numSimultaneousOperations = getOption(options, LiteralStringRef("numSimultaneousOperations"), 10); targetFileSize = getOption(options, LiteralStringRef("targetFileSize"), (uint64_t)163840); diff --git a/fdbserver/workloads/AsyncFileWrite.actor.cpp b/fdbserver/workloads/AsyncFileWrite.actor.cpp index 23659e832d..2848eaa9e3 100644 --- a/fdbserver/workloads/AsyncFileWrite.actor.cpp +++ b/fdbserver/workloads/AsyncFileWrite.actor.cpp @@ -46,7 +46,7 @@ struct AsyncFileWriteWorkload : public AsyncFileWorkload { PerfIntCounter bytesWritten; AsyncFileWriteWorkload(WorkloadContext const& wcx) - : AsyncFileWorkload(wcx), bytesWritten("Bytes Written"), writeBuffer(nullptr) { + : AsyncFileWorkload(wcx), writeBuffer(nullptr), bytesWritten("Bytes Written") { numParallelWrites = getOption(options, LiteralStringRef("numParallelWrites"), 0); writeSize = getOption(options, LiteralStringRef("writeSize"), _PAGE_SIZE); fileSize = getOption(options, LiteralStringRef("fileSize"), 10002432); diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index 44572113d4..6d377d9746 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -243,7 +243,7 @@ public: struct DelayedTask : OrderedTask { double at; DelayedTask(double at, int64_t priority, TaskPriority taskID, Task* task) - : at(at), OrderedTask(priority, taskID, task) {} + : OrderedTask(priority, taskID, task), at(at) {} bool operator<(DelayedTask const& rhs) const { return at > rhs.at; } // Ordering is reversed for priority_queue }; std::priority_queue> timers; @@ -1169,19 +1169,16 @@ struct PromiseTask : public Task, public FastAllocated { // 5MB for loading files into memory Net2::Net2(const TLSConfig& tlsConfig, bool useThreadPool, bool useMetrics) - : useThreadPool(useThreadPool), network(this), reactor(this), stopped(false), tasksIssued(0), - ready(FLOW_KNOBS->READY_QUEUE_RESERVED_SIZE), - // Until run() is called, yield() will always yield - tscBegin(0), tscEnd(0), taskBegin(0), currentTaskID(TaskPriority::DefaultYield), numYields(0), - lastPriorityStats(nullptr), tlsInitializedState(ETLSInitState::NONE), tlsConfig(tlsConfig), started(false) + : useThreadPool(useThreadPool), reactor(this), #ifndef TLS_DISABLED - , sslContextVar({ ReferencedObject::from( boost::asio::ssl::context(boost::asio::ssl::context::tls)) }), - sslPoolHandshakesInProgress(0), sslHandshakerThreadsStarted(0) + sslHandshakerThreadsStarted(0), sslPoolHandshakesInProgress(0), #endif - -{ + tlsConfig(tlsConfig), network(this), tscBegin(0), tscEnd(0), taskBegin(0), + currentTaskID(TaskPriority::DefaultYield), tasksIssued(0), stopped(false), started(false), numYields(0), + lastPriorityStats(nullptr), ready(FLOW_KNOBS->READY_QUEUE_RESERVED_SIZE), tlsInitializedState(ETLSInitState::NONE) { + // Until run() is called, yield() will always yield TraceEvent("Net2Starting"); // Set the global members @@ -1908,7 +1905,7 @@ void Net2::getDiskBytes(std::string const& directory, int64_t& free, int64_t& to #include #endif -ASIOReactor::ASIOReactor(Net2* net) : network(net), firstTimer(ios), do_not_stop(ios) { +ASIOReactor::ASIOReactor(Net2* net) : do_not_stop(ios), network(net), firstTimer(ios) { #ifdef __linux__ // Reactor flags are used only for experimentation, and are platform-specific if (FLOW_KNOBS->REACTOR_FLAGS & 1) { diff --git a/flow/Profiler.actor.cpp b/flow/Profiler.actor.cpp index 1275c5d410..0ee9dfce0e 100644 --- a/flow/Profiler.actor.cpp +++ b/flow/Profiler.actor.cpp @@ -128,7 +128,7 @@ struct Profiler { bool timerInitialized; Profiler(int period, std::string const& outfn, INetwork* network) - : environmentInfoWriter(Unversioned()), signalClosure(signal_handler_for_closure, this), network(network), + : signalClosure(signal_handler_for_closure, this), environmentInfoWriter(Unversioned()), network(network), timerInitialized(false) { actor = profile(this, period, outfn); } diff --git a/flow/Trace.cpp b/flow/Trace.cpp index e8655cf6cb..c099636644 100644 --- a/flow/Trace.cpp +++ b/flow/Trace.cpp @@ -205,7 +205,7 @@ public: WriterThread(Reference barriers, Reference logWriter, Reference formatter) - : barriers(barriers), logWriter(logWriter), formatter(formatter) {} + : logWriter(logWriter), formatter(formatter), barriers(barriers) {} void init() override {} @@ -277,8 +277,8 @@ public: }; TraceLog() - : bufferLength(0), loggedLength(0), opened(false), preopenOverflowCount(0), barriers(new BarrierList), - logTraceEventMetrics(false), formatter(new XmlTraceLogFormatter()), issues(new IssuesList) {} + : formatter(new XmlTraceLogFormatter()), loggedLength(0), bufferLength(0), opened(false), preopenOverflowCount(0), + logTraceEventMetrics(false), issues(new IssuesList), barriers(new BarrierList) {} bool isOpen() const { return opened; } @@ -835,28 +835,27 @@ Future pingTraceLogWriterThread() { } TraceEvent::TraceEvent(const char* type, UID id) - : id(id), type(type), severity(SevInfo), initialized(false), enabled(true), logged(false) { + : initialized(false), enabled(true), logged(false), severity(SevInfo), type(type), id(id) { setMaxFieldLength(0); setMaxEventLength(0); } TraceEvent::TraceEvent(Severity severity, const char* type, UID id) - : id(id), type(type), severity(severity), initialized(false), logged(false), - enabled(g_network == nullptr || FLOW_KNOBS->MIN_TRACE_SEVERITY <= severity) { + : initialized(false), enabled(g_network == nullptr || FLOW_KNOBS->MIN_TRACE_SEVERITY <= severity), logged(false), + severity(severity), type(type), id(id) { setMaxFieldLength(0); setMaxEventLength(0); } TraceEvent::TraceEvent(TraceInterval& interval, UID id) - : id(id), type(interval.type), severity(interval.severity), initialized(false), logged(false), - enabled(g_network == nullptr || FLOW_KNOBS->MIN_TRACE_SEVERITY <= interval.severity) { - + : initialized(false), enabled(g_network == nullptr || FLOW_KNOBS->MIN_TRACE_SEVERITY <= interval.severity), + logged(false), type(interval.type), id(id), severity(interval.severity) { setMaxFieldLength(0); setMaxEventLength(0); init(interval); } TraceEvent::TraceEvent(Severity severity, TraceInterval& interval, UID id) - : id(id), type(interval.type), severity(severity), initialized(false), logged(false), - enabled(g_network == nullptr || FLOW_KNOBS->MIN_TRACE_SEVERITY <= severity) { + : initialized(false), logged(false), enabled(g_network == nullptr || FLOW_KNOBS->MIN_TRACE_SEVERITY <= severity), + type(interval.type), id(id), severity(severity) { setMaxFieldLength(0); setMaxEventLength(0); diff --git a/flow/Tracing.actor.cpp b/flow/Tracing.actor.cpp index de62069a4f..f3bb438e14 100644 --- a/flow/Tracing.actor.cpp +++ b/flow/Tracing.actor.cpp @@ -276,7 +276,7 @@ ACTOR Future fastTraceLogger(int* unreadyMessages, int* failedMessages, in struct FastUDPTracer : public UDPTracer { FastUDPTracer() - : socket_fd_(-1), unready_socket_messages_(0), failed_messages_(0), total_messages_(0), send_error_(false) { + : unready_socket_messages_(0), failed_messages_(0), total_messages_(0), socket_fd_(-1), send_error_(false) { request_ = TraceRequest{ .buffer = std::make_unique(kTraceBufferSize), .data_size = 0, .buffer_size = kTraceBufferSize };