Merge pull request #5264 from sfc-gh-tclinkenbeard/fix-more-clang-warnings
Enable more warnings for `clang`
This commit is contained in:
commit
353efe7db2
|
@ -36,8 +36,8 @@ const Subspace DirectoryLayer::DEFAULT_CONTENT_SUBSPACE = Subspace();
|
|||
const StringRef DirectoryLayer::PARTITION_LAYER = LiteralStringRef("partition");
|
||||
|
||||
DirectoryLayer::DirectoryLayer(Subspace nodeSubspace, Subspace contentSubspace, bool allowManualPrefixes)
|
||||
: nodeSubspace(nodeSubspace), contentSubspace(contentSubspace), allowManualPrefixes(allowManualPrefixes),
|
||||
rootNode(nodeSubspace.get(nodeSubspace.key())), allocator(rootNode.get(HIGH_CONTENTION_KEY)) {}
|
||||
: rootNode(nodeSubspace.get(nodeSubspace.key())), nodeSubspace(nodeSubspace), contentSubspace(contentSubspace),
|
||||
allocator(rootNode.get(HIGH_CONTENTION_KEY)), allowManualPrefixes(allowManualPrefixes) {}
|
||||
|
||||
Subspace DirectoryLayer::nodeWithPrefix(StringRef const& prefix) const {
|
||||
return nodeSubspace.get(prefix);
|
||||
|
|
|
@ -167,9 +167,9 @@ struct RangeResultRef : VectorRef<KeyValueRef> {
|
|||
|
||||
RangeResultRef() : more(false), readToBegin(false), readThroughEnd(false) {}
|
||||
RangeResultRef(Arena& p, const RangeResultRef& toCopy)
|
||||
: more(toCopy.more), readToBegin(toCopy.readToBegin), readThroughEnd(toCopy.readThroughEnd),
|
||||
: VectorRef<KeyValueRef>(p, toCopy), more(toCopy.more),
|
||||
readThrough(toCopy.readThrough.present() ? KeyRef(p, toCopy.readThrough.get()) : Optional<KeyRef>()),
|
||||
VectorRef<KeyValueRef>(p, toCopy) {}
|
||||
readToBegin(toCopy.readToBegin), readThroughEnd(toCopy.readThroughEnd) {}
|
||||
RangeResultRef(const VectorRef<KeyValueRef>& value, bool more, Optional<KeyRef> readThrough = Optional<KeyRef>())
|
||||
: VectorRef<KeyValueRef>(value), more(more), readThrough(readThrough), readToBegin(false), readThroughEnd(false) {
|
||||
}
|
||||
|
|
|
@ -287,13 +287,10 @@ else()
|
|||
-Wshift-sign-overflow
|
||||
# Here's the current set of warnings we need to explicitly disable to compile warning-free with clang 11
|
||||
-Wno-comment
|
||||
-Wno-dangling-else
|
||||
-Wno-delete-non-virtual-dtor
|
||||
-Wno-format
|
||||
-Wno-mismatched-tags
|
||||
-Wno-missing-field-initializers
|
||||
-Wno-reorder
|
||||
-Wno-reorder-ctor
|
||||
-Wno-sign-compare
|
||||
-Wno-tautological-pointer-compare
|
||||
-Wno-undefined-var-template
|
||||
|
|
|
@ -298,7 +298,7 @@ class DecodeProgress {
|
|||
public:
|
||||
DecodeProgress() = default;
|
||||
template <class U>
|
||||
DecodeProgress(const LogFile& file, U&& values) : file(file), keyValues(std::forward<U>(values)) {}
|
||||
DecodeProgress(const LogFile& file, U&& values) : keyValues(std::forward<U>(values)), file(file) {}
|
||||
|
||||
// If there are no more mutations to pull from the file.
|
||||
// However, we could have unfinished version in the buffer when EOF is true,
|
||||
|
|
|
@ -368,8 +368,8 @@ public:
|
|||
DatabaseBackupAgent(DatabaseBackupAgent&& r) noexcept
|
||||
: subspace(std::move(r.subspace)), states(std::move(r.states)), config(std::move(r.config)),
|
||||
errors(std::move(r.errors)), ranges(std::move(r.ranges)), tagNames(std::move(r.tagNames)),
|
||||
taskBucket(std::move(r.taskBucket)), futureBucket(std::move(r.futureBucket)),
|
||||
sourceStates(std::move(r.sourceStates)), sourceTagNames(std::move(r.sourceTagNames)) {}
|
||||
sourceStates(std::move(r.sourceStates)), sourceTagNames(std::move(r.sourceTagNames)),
|
||||
taskBucket(std::move(r.taskBucket)), futureBucket(std::move(r.futureBucket)) {}
|
||||
|
||||
void operator=(DatabaseBackupAgent&& r) noexcept {
|
||||
subspace = std::move(r.subspace);
|
||||
|
|
|
@ -31,7 +31,8 @@ namespace {
|
|||
class BackupFile : public IBackupFile, ReferenceCounted<BackupFile> {
|
||||
public:
|
||||
BackupFile(const std::string& fileName, Reference<IAsyncFile> 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);
|
||||
}
|
||||
|
|
|
@ -244,7 +244,7 @@ struct GetReadVersionRequest : TimedRequest {
|
|||
uint32_t flags = 0,
|
||||
TransactionTagMap<uint32_t> tags = TransactionTagMap<uint32_t>(),
|
||||
Optional<UID> debugID = Optional<UID>())
|
||||
: spanContext(spanContext), transactionCount(transactionCount), priority(priority), flags(flags), tags(tags),
|
||||
: spanContext(spanContext), transactionCount(transactionCount), flags(flags), priority(priority), tags(tags),
|
||||
debugID(debugID) {
|
||||
flags = flags & ~FLAG_PRIORITY_MASK;
|
||||
switch (priority) {
|
||||
|
@ -313,7 +313,7 @@ struct GetKeyServerLocationsRequest {
|
|||
int limit,
|
||||
bool reverse,
|
||||
Arena const& arena)
|
||||
: spanContext(spanContext), begin(begin), end(end), limit(limit), reverse(reverse), arena(arena) {}
|
||||
: arena(arena), spanContext(spanContext), begin(begin), end(end), limit(limit), reverse(reverse) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -51,7 +51,7 @@ public:
|
|||
private:
|
||||
DatabaseContext* cx;
|
||||
StorageServerInfo(DatabaseContext* cx, StorageServerInterface const& interf, LocalityData const& locality)
|
||||
: cx(cx), ReferencedInterface<StorageServerInterface>(interf, locality) {}
|
||||
: ReferencedInterface<StorageServerInterface>(interf, locality), cx(cx) {}
|
||||
};
|
||||
|
||||
struct LocationInfo : MultiInterface<ReferencedInterface<StorageServerInterface>>, FastAllocated<LocationInfo> {
|
||||
|
|
|
@ -655,9 +655,9 @@ struct RangeResultRef : VectorRef<KeyValueRef> {
|
|||
|
||||
RangeResultRef() : more(false), readToBegin(false), readThroughEnd(false) {}
|
||||
RangeResultRef(Arena& p, const RangeResultRef& toCopy)
|
||||
: more(toCopy.more), readToBegin(toCopy.readToBegin), readThroughEnd(toCopy.readThroughEnd),
|
||||
: VectorRef<KeyValueRef>(p, toCopy), more(toCopy.more),
|
||||
readThrough(toCopy.readThrough.present() ? KeyRef(p, toCopy.readThrough.get()) : Optional<KeyRef>()),
|
||||
VectorRef<KeyValueRef>(p, toCopy) {}
|
||||
readToBegin(toCopy.readToBegin), readThroughEnd(toCopy.readThroughEnd) {}
|
||||
RangeResultRef(const VectorRef<KeyValueRef>& value, bool more, Optional<KeyRef> readThrough = Optional<KeyRef>())
|
||||
: VectorRef<KeyValueRef>(value), more(more), readThrough(readThrough), readToBegin(false), readThroughEnd(false) {
|
||||
}
|
||||
|
|
|
@ -58,7 +58,7 @@ struct MonitorLeaderInfo {
|
|||
|
||||
MonitorLeaderInfo() : hasConnected(false) {}
|
||||
explicit MonitorLeaderInfo(Reference<ClusterConnectionFile> intermediateConnFile)
|
||||
: intermediateConnFile(intermediateConnFile), hasConnected(false) {}
|
||||
: hasConnected(false), intermediateConnFile(intermediateConnFile) {}
|
||||
};
|
||||
|
||||
// Monitors the given coordination group's leader election process and provides a best current guess
|
||||
|
|
|
@ -281,7 +281,7 @@ template <class S, class T>
|
|||
class FlatMapSingleAssignmentVar final : public ThreadSingleAssignmentVar<T>, ThreadCallback {
|
||||
public:
|
||||
FlatMapSingleAssignmentVar(ThreadFuture<S> source, std::function<ErrorOr<ThreadFuture<T>>(ErrorOr<S>)> mapValue)
|
||||
: source(source), mapValue(mapValue), cancelled(false), released(false) {
|
||||
: source(source), cancelled(false), released(false), mapValue(mapValue) {
|
||||
ThreadSingleAssignmentVar<T>::addref();
|
||||
|
||||
int userParam;
|
||||
|
|
|
@ -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() {
|
||||
|
@ -1014,8 +1014,8 @@ ThreadFuture<ProtocolVersion> MultiVersionDatabase::getServerProtocol(Optional<P
|
|||
}
|
||||
|
||||
MultiVersionDatabase::DatabaseState::DatabaseState(std::string clusterFilePath, Reference<IDatabase> versionMonitorDb)
|
||||
: clusterFilePath(clusterFilePath), versionMonitorDb(versionMonitorDb),
|
||||
dbVar(new ThreadSafeAsyncVar<Reference<IDatabase>>(Reference<IDatabase>(nullptr))), closed(false) {}
|
||||
: dbVar(new ThreadSafeAsyncVar<Reference<IDatabase>>(Reference<IDatabase>(nullptr))),
|
||||
clusterFilePath(clusterFilePath), versionMonitorDb(versionMonitorDb), closed(false) {}
|
||||
|
||||
// Adds a client (local or externally loaded) that can be used to connect to the cluster
|
||||
void MultiVersionDatabase::DatabaseState::addClient(Reference<ClientInfo> client) {
|
||||
|
@ -1912,8 +1912,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();
|
||||
|
||||
|
|
|
@ -116,10 +116,10 @@ TLSConfig tlsConfig(TLSEndpointType::CLIENT);
|
|||
|
||||
// The default values, TRACE_DEFAULT_ROLL_SIZE and TRACE_DEFAULT_MAX_LOGS_SIZE are located in Trace.h.
|
||||
NetworkOptions::NetworkOptions()
|
||||
: localAddress(""), clusterFile(""), traceDirectory(Optional<std::string>()), traceRollSize(TRACE_DEFAULT_ROLL_SIZE),
|
||||
traceMaxLogsSize(TRACE_DEFAULT_MAX_LOGS_SIZE), traceLogGroup("default"), traceFormat("xml"),
|
||||
traceClockSource("now"), runLoopProfilingEnabled(false),
|
||||
supportedVersions(new ReferencedObject<Standalone<VectorRef<ClientVersionRef>>>()) {}
|
||||
: traceRollSize(TRACE_DEFAULT_ROLL_SIZE), traceMaxLogsSize(TRACE_DEFAULT_MAX_LOGS_SIZE), traceLogGroup("default"),
|
||||
traceFormat("xml"), traceClockSource("now"),
|
||||
supportedVersions(new ReferencedObject<Standalone<VectorRef<ClientVersionRef>>>()), runLoopProfilingEnabled(false) {
|
||||
}
|
||||
|
||||
static const Key CLIENT_LATENCY_INFO_PREFIX = LiteralStringRef("client_latency/");
|
||||
static const Key CLIENT_LATENCY_INFO_CTR_PREFIX = LiteralStringRef("client_latency_counter/");
|
||||
|
@ -1094,11 +1094,10 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<ClusterConnectionF
|
|||
IsInternal internal,
|
||||
int apiVersion,
|
||||
IsSwitchable switchable)
|
||||
: connectionFile(connectionFile), clientInfo(clientInfo), coordinator(coordinator),
|
||||
clientInfoMonitor(clientInfoMonitor), taskID(taskID), clientLocality(clientLocality),
|
||||
enableLocalityLoadBalance(enableLocalityLoadBalance), lockAware(lockAware), apiVersion(apiVersion),
|
||||
switchable(switchable), proxyProvisional(false), cc("TransactionMetrics"),
|
||||
transactionReadVersions("ReadVersions", cc), transactionReadVersionsThrottled("ReadVersionsThrottled", cc),
|
||||
: lockAware(lockAware), switchable(switchable), connectionFile(connectionFile), proxyProvisional(false),
|
||||
clientLocality(clientLocality), enableLocalityLoadBalance(enableLocalityLoadBalance), internal(internal),
|
||||
cc("TransactionMetrics"), transactionReadVersions("ReadVersions", cc),
|
||||
transactionReadVersionsThrottled("ReadVersionsThrottled", cc),
|
||||
transactionReadVersionsCompleted("ReadVersionsCompleted", cc),
|
||||
transactionReadVersionBatches("ReadVersionBatches", cc),
|
||||
transactionBatchReadVersions("BatchPriorityReadVersions", cc),
|
||||
|
@ -1123,11 +1122,12 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<ClusterConnectionF
|
|||
transactionStatusRequests("StatusRequests", cc), transactionsTooOld("TooOld", cc),
|
||||
transactionsFutureVersions("FutureVersions", cc), transactionsNotCommitted("NotCommitted", cc),
|
||||
transactionsMaybeCommitted("MaybeCommitted", cc), transactionsResourceConstrained("ResourceConstrained", cc),
|
||||
transactionsThrottled("Throttled", cc), transactionsProcessBehind("ProcessBehind", cc), outstandingWatches(0),
|
||||
latencies(1000), readLatencies(1000), commitLatencies(1000), GRVLatencies(1000), mutationsPerCommit(1000),
|
||||
bytesPerCommit(1000), mvCacheInsertLocation(0), healthMetricsLastUpdated(0), detailedHealthMetricsLastUpdated(0),
|
||||
internal(internal), transactionTracingEnabled(true), smoothMidShardSize(CLIENT_KNOBS->SHARD_STAT_SMOOTH_AMOUNT),
|
||||
transactionsExpensiveClearCostEstCount("ExpensiveClearCostEstCount", cc),
|
||||
transactionsProcessBehind("ProcessBehind", cc), transactionsThrottled("Throttled", cc),
|
||||
transactionsExpensiveClearCostEstCount("ExpensiveClearCostEstCount", cc), latencies(1000), readLatencies(1000),
|
||||
commitLatencies(1000), GRVLatencies(1000), mutationsPerCommit(1000), bytesPerCommit(1000), outstandingWatches(0),
|
||||
transactionTracingEnabled(true), taskID(taskID), clientInfo(clientInfo), clientInfoMonitor(clientInfoMonitor),
|
||||
coordinator(coordinator), apiVersion(apiVersion), mvCacheInsertLocation(0), healthMetricsLastUpdated(0),
|
||||
detailedHealthMetricsLastUpdated(0), smoothMidShardSize(CLIENT_KNOBS->SHARD_STAT_SMOOTH_AMOUNT),
|
||||
specialKeySpace(std::make_unique<SpecialKeySpace>(specialKeys.begin, specialKeys.end, /* test */ false)) {
|
||||
dbId = deterministicRandom()->randomUniqueID();
|
||||
connected = (clientInfo->get().commitProxies.size() && clientInfo->get().grvProxies.size())
|
||||
|
@ -1340,8 +1340,8 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<ClusterConnectionF
|
|||
}
|
||||
|
||||
DatabaseContext::DatabaseContext(const Error& err)
|
||||
: deferredError(err), cc("TransactionMetrics"), transactionReadVersions("ReadVersions", cc),
|
||||
transactionReadVersionsThrottled("ReadVersionsThrottled", cc),
|
||||
: deferredError(err), internal(IsInternal::False), cc("TransactionMetrics"),
|
||||
transactionReadVersions("ReadVersions", cc), transactionReadVersionsThrottled("ReadVersionsThrottled", cc),
|
||||
transactionReadVersionsCompleted("ReadVersionsCompleted", cc),
|
||||
transactionReadVersionBatches("ReadVersionBatches", cc),
|
||||
transactionBatchReadVersions("BatchPriorityReadVersions", cc),
|
||||
|
@ -1366,11 +1366,10 @@ DatabaseContext::DatabaseContext(const Error& err)
|
|||
transactionStatusRequests("StatusRequests", cc), transactionsTooOld("TooOld", cc),
|
||||
transactionsFutureVersions("FutureVersions", cc), transactionsNotCommitted("NotCommitted", cc),
|
||||
transactionsMaybeCommitted("MaybeCommitted", cc), transactionsResourceConstrained("ResourceConstrained", cc),
|
||||
transactionsThrottled("Throttled", cc), transactionsProcessBehind("ProcessBehind", cc), latencies(1000),
|
||||
readLatencies(1000), commitLatencies(1000), GRVLatencies(1000), mutationsPerCommit(1000), bytesPerCommit(1000),
|
||||
smoothMidShardSize(CLIENT_KNOBS->SHARD_STAT_SMOOTH_AMOUNT),
|
||||
transactionsExpensiveClearCostEstCount("ExpensiveClearCostEstCount", cc), internal(IsInternal::False),
|
||||
transactionTracingEnabled(true) {}
|
||||
transactionsProcessBehind("ProcessBehind", cc), transactionsThrottled("Throttled", cc),
|
||||
transactionsExpensiveClearCostEstCount("ExpensiveClearCostEstCount", cc), latencies(1000), readLatencies(1000),
|
||||
commitLatencies(1000), GRVLatencies(1000), mutationsPerCommit(1000), bytesPerCommit(1000),
|
||||
transactionTracingEnabled(true), smoothMidShardSize(CLIENT_KNOBS->SHARD_STAT_SMOOTH_AMOUNT) {}
|
||||
|
||||
// Static constructor used by server processes to create a DatabaseContext
|
||||
// For internal (fdbserver) use only
|
||||
|
@ -4093,9 +4092,9 @@ Transaction::Transaction()
|
|||
: info(TaskPriority::DefaultEndpoint, generateSpanID(true)), span(info.spanID, "Transaction"_loc) {}
|
||||
|
||||
Transaction::Transaction(Database const& cx)
|
||||
: cx(cx), info(cx->taskID, generateSpanID(cx->transactionTracingEnabled)), backoff(CLIENT_KNOBS->DEFAULT_BACKOFF),
|
||||
committedVersion(invalidVersion), versionstampPromise(Promise<Standalone<StringRef>>()), options(cx), numErrors(0),
|
||||
trLogInfo(createTrLogInfoProbabilistically(cx)), tr(info.spanID), span(info.spanID, "Transaction"_loc) {
|
||||
: info(cx->taskID, generateSpanID(cx->transactionTracingEnabled)), numErrors(0), options(cx),
|
||||
span(info.spanID, "Transaction"_loc), trLogInfo(createTrLogInfoProbabilistically(cx)), cx(cx),
|
||||
backoff(CLIENT_KNOBS->DEFAULT_BACKOFF), committedVersion(invalidVersion), tr(info.spanID) {
|
||||
if (DatabaseContext::debugUseTags) {
|
||||
debugAddTags(this);
|
||||
}
|
||||
|
|
|
@ -189,7 +189,7 @@ struct TransactionLogInfo : public ReferenceCounted<TransactionLogInfo>, NonCopy
|
|||
TransactionLogInfo() : logLocation(DONT_LOG), maxFieldLength(0) {}
|
||||
TransactionLogInfo(LoggingLocation location) : logLocation(location), maxFieldLength(0) {}
|
||||
TransactionLogInfo(std::string id, LoggingLocation location)
|
||||
: logLocation(location), identifier(id), maxFieldLength(0) {}
|
||||
: logLocation(location), maxFieldLength(0), identifier(id) {}
|
||||
|
||||
void setIdentifier(std::string id) { identifier = id; }
|
||||
void logTo(LoggingLocation loc) { logLocation = logLocation | loc; }
|
||||
|
@ -231,10 +231,10 @@ struct Watch : public ReferenceCounted<Watch>, NonCopyable {
|
|||
Promise<Void> onSetWatchTrigger;
|
||||
Future<Void> watchFuture;
|
||||
|
||||
Watch() : watchFuture(Never()), valuePresent(false), setPresent(false) {}
|
||||
Watch(Key key) : key(key), watchFuture(Never()), valuePresent(false), setPresent(false) {}
|
||||
Watch() : valuePresent(false), setPresent(false), watchFuture(Never()) {}
|
||||
Watch(Key key) : key(key), valuePresent(false), setPresent(false), watchFuture(Never()) {}
|
||||
Watch(Key key, Optional<Value> val)
|
||||
: key(key), value(val), watchFuture(Never()), valuePresent(true), setPresent(false) {}
|
||||
: key(key), value(val), valuePresent(true), setPresent(false), watchFuture(Never()) {}
|
||||
|
||||
void setWatch(Future<Void> watchFuture);
|
||||
};
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
class RYWIterator {
|
||||
public:
|
||||
RYWIterator(SnapshotCache* snapshotCache, WriteMap* writeMap)
|
||||
: cache(snapshotCache), writes(writeMap), begin_key_cmp(0), end_key_cmp(0), bypassUnreadable(false) {}
|
||||
: begin_key_cmp(0), end_key_cmp(0), cache(snapshotCache), writes(writeMap), bypassUnreadable(false) {}
|
||||
|
||||
enum SEGMENT_TYPE { UNKNOWN_RANGE, EMPTY_RANGE, KV };
|
||||
static const SEGMENT_TYPE typeMap[12];
|
||||
|
|
|
@ -1285,9 +1285,9 @@ public:
|
|||
};
|
||||
|
||||
ReadYourWritesTransaction::ReadYourWritesTransaction(Database const& cx)
|
||||
: ISingleThreadTransaction(cx->deferredError), cache(&arena), writes(&arena), tr(cx), retries(0), approximateSize(0),
|
||||
creationTime(now()), commitStarted(false), options(tr), versionStampFuture(tr.getVersionstamp()),
|
||||
specialKeySpaceWriteMap(std::make_pair(false, Optional<Value>()), specialKeys.end) {
|
||||
: ISingleThreadTransaction(cx->deferredError), tr(cx), cache(&arena), writes(&arena), retries(0), approximateSize(0),
|
||||
creationTime(now()), commitStarted(false), versionStampFuture(tr.getVersionstamp()),
|
||||
specialKeySpaceWriteMap(std::make_pair(false, Optional<Value>()), specialKeys.end), options(tr) {
|
||||
std::copy(
|
||||
cx.getTransactionDefaults().begin(), cx.getTransactionDefaults().end(), std::back_inserter(persistentOptions));
|
||||
applyPersistentOptions();
|
||||
|
@ -2284,10 +2284,11 @@ void ReadYourWritesTransaction::operator=(ReadYourWritesTransaction&& r) noexcep
|
|||
}
|
||||
|
||||
ReadYourWritesTransaction::ReadYourWritesTransaction(ReadYourWritesTransaction&& r) noexcept
|
||||
: ISingleThreadTransaction(std::move(r.deferredError)), cache(std::move(r.cache)), writes(std::move(r.writes)),
|
||||
arena(std::move(r.arena)), reading(std::move(r.reading)), retries(r.retries), approximateSize(r.approximateSize),
|
||||
creationTime(r.creationTime), timeoutActor(std::move(r.timeoutActor)), resetPromise(std::move(r.resetPromise)),
|
||||
commitStarted(r.commitStarted), options(r.options), transactionDebugInfo(r.transactionDebugInfo) {
|
||||
: ISingleThreadTransaction(std::move(r.deferredError)), arena(std::move(r.arena)), cache(std::move(r.cache)),
|
||||
writes(std::move(r.writes)), resetPromise(std::move(r.resetPromise)), reading(std::move(r.reading)),
|
||||
retries(r.retries), approximateSize(r.approximateSize), timeoutActor(std::move(r.timeoutActor)),
|
||||
creationTime(r.creationTime), commitStarted(r.commitStarted), transactionDebugInfo(r.transactionDebugInfo),
|
||||
options(r.options) {
|
||||
cache.arena = &arena;
|
||||
writes.arena = &arena;
|
||||
tr = std::move(r.tr);
|
||||
|
|
|
@ -311,7 +311,7 @@ public:
|
|||
entries.insert(Entry(allKeys.end, afterAllKeys, VectorRef<KeyValueRef>()), NoMetric(), true);
|
||||
}
|
||||
// Visual Studio refuses to generate these, apparently despite the standard
|
||||
SnapshotCache(SnapshotCache&& r) noexcept : entries(std::move(r.entries)), arena(r.arena) {}
|
||||
SnapshotCache(SnapshotCache&& r) noexcept : arena(r.arena), entries(std::move(r.entries)) {}
|
||||
SnapshotCache& operator=(SnapshotCache&& r) noexcept {
|
||||
entries = std::move(r.entries);
|
||||
arena = r.arena;
|
||||
|
|
|
@ -248,8 +248,9 @@ ACTOR Future<Void> 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);
|
||||
|
|
|
@ -873,13 +873,13 @@ TaskBucket::TaskBucket(const Subspace& subspace,
|
|||
AccessSystemKeys sysAccess,
|
||||
PriorityBatch priorityBatch,
|
||||
LockAware lockAware)
|
||||
: 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) {}
|
||||
: cc("TaskBucket"), dispatchSlotChecksStarted("DispatchSlotChecksStarted", cc), dispatchErrors("DispatchErrors", cc),
|
||||
dispatchDoTasks("DispatchDoTasks", cc), dispatchEmptyTasks("DispatchEmptyTasks", cc),
|
||||
dispatchSlotChecksComplete("DispatchSlotChecksComplete", cc), dbgid(deterministicRandom()->randomUniqueID()),
|
||||
prefix(subspace), active(prefix.get(LiteralStringRef("ac"))), pauseKey(prefix.pack(LiteralStringRef("pause"))),
|
||||
available(prefix.get(LiteralStringRef("av"))), available_prioritized(prefix.get(LiteralStringRef("avp"))),
|
||||
timeouts(prefix.get(LiteralStringRef("to"))), timeout(CLIENT_KNOBS->TASKBUCKET_TIMEOUT_VERSIONS),
|
||||
system_access(sysAccess), priority_batch(priorityBatch), lockAware(lockAware) {}
|
||||
|
||||
TaskBucket::~TaskBucket() {}
|
||||
|
||||
|
|
|
@ -58,11 +58,11 @@ struct PTree : public ReferenceCounted<PTree<T>>, FastAllocated<PTree<T>>, NonCo
|
|||
Reference<PTree> left(Version at) const { return child(false, at); }
|
||||
Reference<PTree> right(Version at) const { return child(true, at); }
|
||||
|
||||
PTree(const T& data, Version ver) : data(data), lastUpdateVersion(ver), updated(false) {
|
||||
PTree(const T& data, Version ver) : lastUpdateVersion(ver), updated(false), data(data) {
|
||||
priority = deterministicRandom()->randomUInt32();
|
||||
}
|
||||
PTree(uint32_t pri, T const& data, Reference<PTree> const& left, Reference<PTree> const& right, Version ver)
|
||||
: priority(pri), data(data), lastUpdateVersion(ver), updated(false) {
|
||||
: priority(pri), lastUpdateVersion(ver), updated(false), data(data) {
|
||||
pointer[0] = left;
|
||||
pointer[1] = right;
|
||||
}
|
||||
|
|
|
@ -168,7 +168,7 @@ private:
|
|||
typedef Reference<PTreeT> Tree;
|
||||
|
||||
public:
|
||||
explicit WriteMap(Arena* arena) : arena(arena), ver(-1), scratch_iterator(this), writeMapEmpty(true) {
|
||||
explicit WriteMap(Arena* arena) : arena(arena), writeMapEmpty(true), ver(-1), scratch_iterator(this) {
|
||||
PTreeImpl::insert(
|
||||
writes, ver, WriteMapEntry(allKeys.begin, OperationStack(), false, false, false, false, false));
|
||||
PTreeImpl::insert(writes, ver, WriteMapEntry(allKeys.end, OperationStack(), false, false, false, false, false));
|
||||
|
@ -177,8 +177,8 @@ public:
|
|||
}
|
||||
|
||||
WriteMap(WriteMap&& r) noexcept
|
||||
: writeMapEmpty(r.writeMapEmpty), writes(std::move(r.writes)), ver(r.ver),
|
||||
scratch_iterator(std::move(r.scratch_iterator)), arena(r.arena) {}
|
||||
: arena(r.arena), writeMapEmpty(r.writeMapEmpty), writes(std::move(r.writes)), ver(r.ver),
|
||||
scratch_iterator(std::move(r.scratch_iterator)) {}
|
||||
WriteMap& operator=(WriteMap&& r) noexcept {
|
||||
writeMapEmpty = r.writeMapEmpty;
|
||||
writes = std::move(r.writes);
|
||||
|
|
|
@ -393,7 +393,7 @@ public:
|
|||
|
||||
Command() : argv(nullptr) {}
|
||||
Command(const CSimpleIni& ini, std::string _section, ProcessID id, fdb_fd_set fds, int* maxfd)
|
||||
: section(_section), argv(nullptr), fork_retry_time(-1), quiet(false), delete_envvars(nullptr), fds(fds),
|
||||
: fds(fds), argv(nullptr), section(_section), fork_retry_time(-1), quiet(false), delete_envvars(nullptr),
|
||||
deconfigured(false), kill_on_configuration_change(true) {
|
||||
char _ssection[strlen(section.c_str()) + 22];
|
||||
snprintf(_ssection, strlen(section.c_str()) + 22, "%s", id.c_str());
|
||||
|
|
|
@ -298,7 +298,7 @@ private:
|
|||
const std::string& filename,
|
||||
int64_t length,
|
||||
Reference<EvictablePageCache> pageCache)
|
||||
: uncached(uncached), filename(filename), length(length), prevLength(length), pageCache(pageCache),
|
||||
: filename(filename), uncached(uncached), length(length), prevLength(length), pageCache(pageCache),
|
||||
currentTruncate(Void()), currentTruncateSize(0), rateControl(nullptr) {
|
||||
if (!g_network->isSimulated()) {
|
||||
countFileCacheWrites.init(LiteralStringRef("AsyncFile.CountFileCacheWrites"), filename);
|
||||
|
@ -610,8 +610,8 @@ struct AFCPage : public EvictablePage, public FastAllocated<AFCPage> {
|
|||
}
|
||||
|
||||
AFCPage(AsyncFileCached* owner, int64_t offset)
|
||||
: EvictablePage(owner->pageCache), owner(owner), pageOffset(offset), dirty(false), valid(false), truncated(false),
|
||||
notReading(Void()), notFlushing(Void()), zeroCopyRefCount(0), flushableIndex(-1), writeThroughCount(0) {
|
||||
: EvictablePage(owner->pageCache), owner(owner), pageOffset(offset), notReading(Void()), notFlushing(Void()),
|
||||
dirty(false), valid(false), truncated(false), writeThroughCount(0), flushableIndex(-1), zeroCopyRefCount(0) {
|
||||
pageCache->allocate(this);
|
||||
}
|
||||
|
||||
|
|
|
@ -277,7 +277,7 @@ private:
|
|||
mutable Int64MetricHandle countLogicalReads;
|
||||
|
||||
AsyncFileEIO(int fd, int flags, std::string const& filename)
|
||||
: fd(fd), flags(flags), filename(filename), err(new ErrorInfo) {
|
||||
: fd(fd), flags(flags), err(new ErrorInfo), filename(filename) {
|
||||
if (!g_network->isSimulated()) {
|
||||
countFileLogicalWrites.init(LiteralStringRef("AsyncFile.CountFileLogicalWrites"), filename);
|
||||
countFileLogicalReads.init(LiteralStringRef("AsyncFile.CountFileLogicalReads"), filename);
|
||||
|
|
|
@ -131,7 +131,7 @@ public:
|
|||
};
|
||||
|
||||
AsyncFileEncrypted::AsyncFileEncrypted(Reference<IAsyncFile> file, Mode mode)
|
||||
: file(file), mode(mode), currentBlock(0), readBuffers(FLOW_KNOBS->MAX_DECRYPTED_BLOCKS) {
|
||||
: file(file), mode(mode), readBuffers(FLOW_KNOBS->MAX_DECRYPTED_BLOCKS), currentBlock(0) {
|
||||
firstBlockIV = AsyncFileEncryptedImpl::getFirstBlockIV(file->getFilename());
|
||||
if (mode == Mode::APPEND_ONLY) {
|
||||
encryptor = std::make_unique<EncryptionStreamCipher>(StreamCipher::Key::getKey(), getIV(currentBlock));
|
||||
|
|
|
@ -568,8 +568,8 @@ private:
|
|||
|
||||
uint32_t opsIssued;
|
||||
Context()
|
||||
: iocx(0), evfd(-1), outstanding(0), opsIssued(0), ioStallBegin(0), fallocateSupported(true),
|
||||
fallocateZeroSupported(true), submittedRequestList(nullptr) {
|
||||
: iocx(0), evfd(-1), outstanding(0), ioStallBegin(0), fallocateSupported(true), fallocateZeroSupported(true),
|
||||
submittedRequestList(nullptr), opsIssued(0) {
|
||||
setIOTimeout(0);
|
||||
}
|
||||
|
||||
|
@ -619,7 +619,7 @@ private:
|
|||
static Context ctx;
|
||||
|
||||
explicit AsyncFileKAIO(int fd, int flags, std::string const& filename)
|
||||
: fd(fd), flags(flags), filename(filename), failed(false) {
|
||||
: failed(false), fd(fd), flags(flags), filename(filename) {
|
||||
ASSERT(!FLOW_KNOBS->DISABLE_POSIX_KERNEL_AIO);
|
||||
if (!g_network->isSimulated()) {
|
||||
countFileLogicalWrites.init(LiteralStringRef("AsyncFile.CountFileLogicalWrites"), filename);
|
||||
|
|
|
@ -190,9 +190,8 @@ private:
|
|||
Reference<DiskParameters> diskParameters,
|
||||
NetworkAddress openedAddress,
|
||||
bool aio)
|
||||
: filename(filename), initialFilename(initialFilename), file(file), diskParameters(diskParameters),
|
||||
openedAddress(openedAddress), pendingModifications(uint64_t(-1)), approximateSize(0), reponses(false),
|
||||
aio(aio) {
|
||||
: filename(filename), initialFilename(initialFilename), approximateSize(0), openedAddress(openedAddress),
|
||||
aio(aio), file(file), pendingModifications(uint64_t(-1)), diskParameters(diskParameters), reponses(false) {
|
||||
|
||||
// This is only designed to work in simulation
|
||||
ASSERT(g_network->isSimulated());
|
||||
|
|
|
@ -199,7 +199,7 @@ public:
|
|||
int maxConcurrentReads,
|
||||
int cacheSizeBlocks)
|
||||
: m_f(f), m_block_size(blockSize), m_read_ahead_blocks(readAheadBlocks),
|
||||
m_max_concurrent_reads(maxConcurrentReads), m_cache_block_limit(std::max<int>(1, cacheSizeBlocks)) {}
|
||||
m_cache_block_limit(std::max<int>(1, cacheSizeBlocks)), m_max_concurrent_reads(maxConcurrentReads) {}
|
||||
};
|
||||
|
||||
#include "flow/unactorcompiler.h"
|
||||
|
|
|
@ -1388,7 +1388,7 @@ TEST_CASE("/flow/DeterministicRandom/SignedOverflow") {
|
|||
struct Tracker {
|
||||
int copied;
|
||||
bool moved;
|
||||
Tracker(int copied = 0) : moved(false), copied(copied) {}
|
||||
Tracker(int copied = 0) : copied(copied), moved(false) {}
|
||||
Tracker(Tracker&& other) : Tracker(other.copied) {
|
||||
ASSERT(!other.moved);
|
||||
other.moved = true;
|
||||
|
|
|
@ -340,9 +340,8 @@ ACTOR Future<Void> 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<AsyncVar<bool>>(false);
|
||||
pingLogger = pingLatencyLogger(this);
|
||||
}
|
||||
|
@ -795,13 +794,14 @@ ACTOR Future<Void> connectionKeeper(Reference<Peer> 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<AsyncVar<Optional<ProtocolVersion>>>(new AsyncVar<Optional<ProtocolVersion>>())) {
|
||||
: transport(transport), destination(destination), compatible(true), outgoingConnectionIdle(true),
|
||||
lastConnectTime(0.0), reconnectionDelay(FLOW_KNOBS->INITIAL_RECONNECTION_TIME), peerReferences(-1),
|
||||
incompatibleProtocolVersionNewer(false), bytesReceived(0), bytesSent(0), lastDataPacketSentTime(now()),
|
||||
outstandingReplies(0), pingLatencies(destination.isPublic() ? FLOW_KNOBS->PING_SAMPLE_AMOUNT : 1),
|
||||
lastLoggedTime(0.0), lastLoggedBytesReceived(0), lastLoggedBytesSent(0), timeoutCount(0),
|
||||
protocolVersion(Reference<AsyncVar<Optional<ProtocolVersion>>>(new AsyncVar<Optional<ProtocolVersion>>())),
|
||||
connectOutgoingCount(0), connectIncomingCount(0), connectFailedCount(0),
|
||||
connectLatencies(destination.isPublic() ? FLOW_KNOBS->NETWORK_CONNECT_SAMPLE_AMOUNT : 1) {
|
||||
IFailureMonitor::failureMonitor().setStatus(destination, FailureStatus(false));
|
||||
}
|
||||
|
||||
|
|
|
@ -52,7 +52,7 @@ struct ModelHolder : NonCopyable, public ReferenceCounted<ModelHolder> {
|
|||
double delta;
|
||||
uint64_t token;
|
||||
|
||||
ModelHolder(QueueModel* model, uint64_t token) : model(model), token(token), released(false), startTime(now()) {
|
||||
ModelHolder(QueueModel* model, uint64_t token) : model(model), released(false), startTime(now()), token(token) {
|
||||
if (model) {
|
||||
delta = model->addRequest(token);
|
||||
}
|
||||
|
|
|
@ -30,11 +30,11 @@ using std::vector;
|
|||
|
||||
struct PerfMetric {
|
||||
constexpr static FileIdentifier file_identifier = 5980618;
|
||||
PerfMetric() : m_name(""), m_value(0), m_averaged(false), m_format_code("%.3g") {}
|
||||
PerfMetric() : m_name(""), m_format_code("%.3g"), m_value(0), m_averaged(false) {}
|
||||
PerfMetric(std::string name, double value, bool averaged)
|
||||
: m_name(name), m_value(value), m_averaged(averaged), m_format_code("%.3g") {}
|
||||
: m_name(name), m_format_code("%.3g"), m_value(value), m_averaged(averaged) {}
|
||||
PerfMetric(std::string name, double value, bool averaged, std::string format_code)
|
||||
: m_name(name), m_value(value), m_averaged(averaged), m_format_code(format_code) {}
|
||||
: m_name(name), m_format_code(format_code), m_value(value), m_averaged(averaged) {}
|
||||
|
||||
std::string name() const { return m_name; }
|
||||
double value() const { return m_value; }
|
||||
|
|
|
@ -75,7 +75,7 @@ struct QueueData {
|
|||
Optional<TSSEndpointData> tssData;
|
||||
|
||||
QueueData()
|
||||
: latency(0.001), penalty(1.0), smoothOutstanding(FLOW_KNOBS->QUEUE_MODEL_SMOOTHING_AMOUNT), failedUntil(0),
|
||||
: smoothOutstanding(FLOW_KNOBS->QUEUE_MODEL_SMOOTHING_AMOUNT), latency(0.001), penalty(1.0), failedUntil(0),
|
||||
futureVersionBackoff(FLOW_KNOBS->FUTURE_VERSION_INITIAL_BACKOFF), increaseBackoffTime(0) {}
|
||||
};
|
||||
|
||||
|
|
|
@ -29,12 +29,11 @@
|
|||
struct LocalitySet : public ReferenceCounted<LocalitySet> {
|
||||
public:
|
||||
LocalitySet(LocalitySet const& source)
|
||||
: _entryArray(source._entryArray), _mutableEntryArray(source._mutableEntryArray),
|
||||
: _keymap(source._keymap), _entryArray(source._entryArray), _mutableEntryArray(source._mutableEntryArray),
|
||||
_keyValueArray(source._keyValueArray), _keyIndexArray(source._keyIndexArray), _cacheArray(source._cacheArray),
|
||||
_keymap(source._keymap), _localitygroup(source._localitygroup), _cachehits(source._cachehits),
|
||||
_cachemisses(source._cachemisses) {}
|
||||
_localitygroup(source._localitygroup), _cachehits(source._cachehits), _cachemisses(source._cachemisses) {}
|
||||
LocalitySet(LocalitySet& localityGroup)
|
||||
: _localitygroup(&localityGroup), _keymap(new StringToIntMap()), _cachehits(0), _cachemisses(0) {}
|
||||
: _keymap(new StringToIntMap()), _localitygroup(&localityGroup), _cachehits(0), _cachemisses(0) {}
|
||||
virtual ~LocalitySet() {}
|
||||
|
||||
virtual void addref() { ReferenceCounted<LocalitySet>::addref(); }
|
||||
|
|
|
@ -22,8 +22,8 @@
|
|||
#include "flow/actorcompiler.h" // has to be last include
|
||||
|
||||
Counter::Counter(std::string const& name, CounterCollection& collection)
|
||||
: name(name), interval_start(0), last_event(0), interval_sq_time(0), interval_start_value(0), interval_delta(0),
|
||||
roughness_interval_start(0) {
|
||||
: name(name), interval_start(0), last_event(0), interval_sq_time(0), roughness_interval_start(0), interval_delta(0),
|
||||
interval_start_value(0) {
|
||||
metric.init(collection.name + "." + (char)toupper(name.at(0)) + name.substr(1), collection.id);
|
||||
collection.counters.push_back(this);
|
||||
}
|
||||
|
|
|
@ -227,7 +227,7 @@ private:
|
|||
class LatencySample {
|
||||
public:
|
||||
LatencySample(std::string name, UID id, double loggingInterval, int sampleSize)
|
||||
: name(name), id(id), sample(sampleSize), sampleStart(now()) {
|
||||
: name(name), id(id), sampleStart(now()), sample(sampleSize) {
|
||||
logger = recurring([this]() { logSample(); }, loggingInterval);
|
||||
}
|
||||
|
||||
|
|
|
@ -179,7 +179,7 @@ SimClogging g_clogging;
|
|||
|
||||
struct Sim2Conn final : IConnection, ReferenceCounted<Sim2Conn> {
|
||||
Sim2Conn(ISimulator::ProcessInfo* process)
|
||||
: process(process), dbgid(deterministicRandom()->randomUniqueID()), opened(false), closedByCaller(false),
|
||||
: opened(false), closedByCaller(false), process(process), dbgid(deterministicRandom()->randomUniqueID()),
|
||||
stopReceive(Never()) {
|
||||
pipes = sender(this) && receiver(this);
|
||||
}
|
||||
|
@ -563,8 +563,8 @@ private:
|
|||
const std::string& filename,
|
||||
const std::string& actualFilename,
|
||||
int flags)
|
||||
: h(h), diskParameters(diskParameters), delayOnWrite(delayOnWrite), filename(filename),
|
||||
actualFilename(actualFilename), dbgId(deterministicRandom()->randomUniqueID()), flags(flags) {}
|
||||
: h(h), diskParameters(diskParameters), filename(filename), actualFilename(actualFilename), flags(flags),
|
||||
dbgId(deterministicRandom()->randomUniqueID()), delayOnWrite(delayOnWrite) {}
|
||||
|
||||
static int flagConversion(int flags) {
|
||||
int outFlags = O_BINARY | O_CLOEXEC;
|
||||
|
@ -1993,7 +1993,7 @@ public:
|
|||
}
|
||||
|
||||
Sim2()
|
||||
: time(0.0), timerTime(0.0), taskCount(0), yielded(false), yield_limit(0), currentTaskID(TaskPriority::Zero) {
|
||||
: time(0.0), timerTime(0.0), currentTaskID(TaskPriority::Zero), taskCount(0), yielded(false), yield_limit(0) {
|
||||
// Not letting currentProcess be nullptr eliminates some annoying special cases
|
||||
currentProcess =
|
||||
new ProcessInfo("NoMachine",
|
||||
|
@ -2017,13 +2017,13 @@ public:
|
|||
ProcessInfo* machine;
|
||||
Promise<Void> action;
|
||||
Task(double time, TaskPriority taskID, uint64_t stable, ProcessInfo* machine, Promise<Void>&& action)
|
||||
: time(time), taskID(taskID), stable(stable), machine(machine), action(std::move(action)) {}
|
||||
: taskID(taskID), time(time), stable(stable), machine(machine), action(std::move(action)) {}
|
||||
Task(double time, TaskPriority taskID, uint64_t stable, ProcessInfo* machine, Future<Void>& future)
|
||||
: time(time), taskID(taskID), stable(stable), machine(machine) {
|
||||
: taskID(taskID), time(time), stable(stable), machine(machine) {
|
||||
future = action.getFuture();
|
||||
}
|
||||
Task(Task&& rhs) noexcept
|
||||
: time(rhs.time), taskID(rhs.taskID), stable(rhs.stable), machine(rhs.machine),
|
||||
: taskID(rhs.taskID), time(rhs.time), stable(rhs.stable), machine(rhs.machine),
|
||||
action(std::move(rhs.action)) {}
|
||||
void operator=(Task const& rhs) {
|
||||
taskID = rhs.taskID;
|
||||
|
|
|
@ -39,9 +39,9 @@ class ISimulator : public INetwork {
|
|||
public:
|
||||
ISimulator()
|
||||
: desiredCoordinators(1), physicalDatacenters(1), processesPerMachine(0), listenersPerProcess(1),
|
||||
isStopped(false), lastConnectionFailure(0), connectionFailuresDisableDuration(0), speedUpSimulation(false),
|
||||
allSwapsDisabled(false), backupAgents(BackupAgentType::WaitForType), drAgents(BackupAgentType::WaitForType),
|
||||
extraDB(nullptr), allowLogSetKills(true), usableRegions(1), tssMode(TSSMode::Disabled) {}
|
||||
extraDB(nullptr), usableRegions(1), allowLogSetKills(true), tssMode(TSSMode::Disabled), isStopped(false),
|
||||
lastConnectionFailure(0), connectionFailuresDisableDuration(0), speedUpSimulation(false),
|
||||
backupAgents(BackupAgentType::WaitForType), drAgents(BackupAgentType::WaitForType), allSwapsDisabled(false) {}
|
||||
|
||||
// Order matters!
|
||||
enum KillType {
|
||||
|
@ -99,10 +99,10 @@ public:
|
|||
INetworkConnections* net,
|
||||
const char* dataFolder,
|
||||
const char* coordinationFolder)
|
||||
: name(name), locality(locality), startingClass(startingClass), addresses(addresses),
|
||||
address(addresses.address), dataFolder(dataFolder), network(net), coordinationFolder(coordinationFolder),
|
||||
failed(false), excluded(false), rebooting(false), fault_injection_p1(0), fault_injection_p2(0),
|
||||
fault_injection_r(0), machine(0), cleared(false), failedDisk(false) {
|
||||
: name(name), coordinationFolder(coordinationFolder), dataFolder(dataFolder), machine(nullptr),
|
||||
addresses(addresses), address(addresses.address), locality(locality), startingClass(startingClass),
|
||||
failed(false), excluded(false), cleared(false), rebooting(false), network(net), fault_injection_r(0),
|
||||
fault_injection_p1(0), fault_injection_p2(0), failedDisk(false) {
|
||||
uid = deterministicRandom()->randomUniqueID();
|
||||
}
|
||||
|
||||
|
|
|
@ -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; });
|
||||
|
|
|
@ -128,15 +128,15 @@ public:
|
|||
std::map<NetworkAddress, std::pair<double, OpenDatabaseRequest>> clientStatus;
|
||||
|
||||
DBInfo()
|
||||
: masterRegistrationCount(0), recoveryStalled(false), forceRecovery(false), unfinishedRecoveries(0),
|
||||
logGenerations(0), cachePopulated(false), clientInfo(new AsyncVar<ClientDBInfo>()), dbInfoCount(0),
|
||||
serverInfo(new AsyncVar<ServerDBInfo>()), db(DatabaseContext::create(clientInfo,
|
||||
Future<Void>(),
|
||||
LocalityData(),
|
||||
EnableLocalityLoadBalance::True,
|
||||
TaskPriority::DefaultEndpoint,
|
||||
LockAware::True)) // SOMEDAY: Locality!
|
||||
{}
|
||||
: clientInfo(new AsyncVar<ClientDBInfo>()), serverInfo(new AsyncVar<ServerDBInfo>()),
|
||||
masterRegistrationCount(0), dbInfoCount(0), recoveryStalled(false), forceRecovery(false),
|
||||
db(DatabaseContext::create(clientInfo,
|
||||
Future<Void>(),
|
||||
LocalityData(),
|
||||
EnableLocalityLoadBalance::True,
|
||||
TaskPriority::DefaultEndpoint,
|
||||
LockAware::True)), // SOMEDAY: Locality!
|
||||
unfinishedRecoveries(0), logGenerations(0), cachePopulated(false) {}
|
||||
|
||||
void setDistributor(const DataDistributorInterface& interf) {
|
||||
auto newInfo = serverInfo->get();
|
||||
|
@ -1433,12 +1433,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),
|
||||
|
@ -3061,9 +3061,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()),
|
||||
|
|
|
@ -500,9 +500,7 @@ CommitBatchContext::CommitBatchContext(ProxyCommitData* const pProxyCommitData_,
|
|||
|
||||
localBatchNumber(++pProxyCommitData->localCommitBatchesStarted), toCommit(pProxyCommitData->logSystem),
|
||||
|
||||
committed(trs.size()),
|
||||
|
||||
span("MP:commitBatch"_loc) {
|
||||
span("MP:commitBatch"_loc), committed(trs.size()) {
|
||||
|
||||
evaluateBatchSize();
|
||||
|
||||
|
|
|
@ -203,7 +203,7 @@ class ConfigBroadcasterImpl {
|
|||
}
|
||||
|
||||
ConfigBroadcasterImpl()
|
||||
: id(deterministicRandom()->randomUniqueID()), lastCompactedVersion(0), mostRecentVersion(0),
|
||||
: lastCompactedVersion(0), mostRecentVersion(0), id(deterministicRandom()->randomUniqueID()),
|
||||
cc("ConfigBroadcaster"), compactRequest("CompactRequest", cc),
|
||||
successfulChangeRequest("SuccessfulChangeRequest", cc), failedChangeRequest("FailedChangeRequest", cc),
|
||||
snapshotRequest("SnapshotRequest", cc) {
|
||||
|
|
|
@ -241,8 +241,8 @@ class BroadcasterToLocalConfigEnvironment {
|
|||
|
||||
public:
|
||||
BroadcasterToLocalConfigEnvironment(std::string const& dataDir, std::string const& configPath)
|
||||
: broadcaster(ConfigFollowerInterface{}), cbfi(makeReference<AsyncVar<ConfigBroadcastFollowerInterface>>()),
|
||||
readFrom(dataDir, configPath, {}) {}
|
||||
: readFrom(dataDir, configPath, {}), cbfi(makeReference<AsyncVar<ConfigBroadcastFollowerInterface>>()),
|
||||
broadcaster(ConfigFollowerInterface{}) {}
|
||||
|
||||
Future<Void> 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<AsyncVar<ConfigBroadcastFollowerInterface>>()) {}
|
||||
: writeTo(dataDir), readFrom(dataDir, configPath, {}),
|
||||
cbfi(makeReference<AsyncVar<ConfigBroadcastFollowerInterface>>()), broadcaster(writeTo.getFollowerInterface()) {
|
||||
}
|
||||
|
||||
Future<Void> setup() { return setup(this); }
|
||||
|
||||
|
|
|
@ -97,7 +97,7 @@ struct OldTLogCoreData {
|
|||
std::set<int8_t> pseudoLocalities;
|
||||
LogEpoch epoch;
|
||||
|
||||
OldTLogCoreData() : epochBegin(0), epochEnd(0), logRouterTags(0), txsTags(0), epoch(0) {}
|
||||
OldTLogCoreData() : logRouterTags(0), txsTags(0), epochBegin(0), epochEnd(0), epoch(0) {}
|
||||
explicit OldTLogCoreData(const OldLogData&);
|
||||
|
||||
bool operator==(const OldTLogCoreData& rhs) const {
|
||||
|
|
|
@ -87,10 +87,10 @@ struct TCServerInfo : public ReferenceCounted<TCServerInfo> {
|
|||
bool inDesiredDC,
|
||||
Reference<LocalitySet> storageServerSet,
|
||||
Version addedVersion = 0)
|
||||
: id(ssi.id()), collection(collection), lastKnownInterface(ssi), lastKnownClass(processClass),
|
||||
dataInFlightToServer(0), onInterfaceChanged(interfaceChanged.getFuture()), onRemoved(removed.getFuture()),
|
||||
inDesiredDC(inDesiredDC), storeType(KeyValueStoreType::END), onTSSPairRemoved(Never()),
|
||||
addedVersion(addedVersion) {
|
||||
: id(ssi.id()), addedVersion(addedVersion), collection(collection), lastKnownInterface(ssi),
|
||||
lastKnownClass(processClass), dataInFlightToServer(0), onInterfaceChanged(interfaceChanged.getFuture()),
|
||||
onRemoved(removed.getFuture()), onTSSPairRemoved(Never()), inDesiredDC(inDesiredDC),
|
||||
storeType(KeyValueStoreType::END) {
|
||||
|
||||
if (!ssi.isTss()) {
|
||||
localityEntry = ((LocalityMap<UID>*)storageServerSet.getPtr())->add(ssi.locality, &id);
|
||||
|
@ -187,7 +187,7 @@ public:
|
|||
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), wrongConfiguration(false), priority(SERVER_KNOBS->PRIORITY_TEAM_HEALTHY),
|
||||
id(deterministicRandom()->randomUniqueID()) {
|
||||
if (servers.empty()) {
|
||||
TraceEvent(SevInfo, "ConstructTCTeamFromEmptyServers").log();
|
||||
|
@ -377,8 +377,8 @@ struct ServerStatus {
|
|||
ServerStatus()
|
||||
: isWiggling(false), isFailed(true), isUndesired(false), isWrongConfiguration(false), initialized(false) {}
|
||||
ServerStatus(bool isFailed, bool isUndesired, bool isWiggling, LocalityData const& locality)
|
||||
: isFailed(isFailed), isUndesired(isUndesired), locality(locality), isWrongConfiguration(false),
|
||||
initialized(true), isWiggling(isWiggling) {}
|
||||
: isWiggling(isWiggling), isFailed(isFailed), isUndesired(isUndesired), isWrongConfiguration(false),
|
||||
initialized(true), locality(locality) {}
|
||||
bool isUnhealthy() const { return isFailed || isUndesired; }
|
||||
const char* toString() const {
|
||||
return isFailed ? "Failed" : isUndesired ? "Undesired" : isWiggling ? "Wiggling" : "Healthy";
|
||||
|
@ -737,22 +737,22 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
|
|||
PromiseStream<GetMetricsRequest> getShardMetrics,
|
||||
Promise<UID> removeFailedServer,
|
||||
PromiseStream<Promise<int>> getUnhealthyRelocationCount)
|
||||
: cx(cx), distributorId(distributorId), lock(lock), output(output),
|
||||
shardsAffectedByTeamFailure(shardsAffectedByTeamFailure), doBuildTeams(true), lastBuildTeamsFailed(false),
|
||||
teamBuilder(Void()), badTeamRemover(Void()), checkInvalidLocalities(Void()), wrongStoreTypeRemover(Void()),
|
||||
configuration(configuration), readyToStart(readyToStart), clearHealthyZoneFuture(true),
|
||||
checkTeamDelay(delay(SERVER_KNOBS->CHECK_TEAM_DELAY, TaskPriority::DataDistribution)),
|
||||
: cx(cx), distributorId(distributorId), configuration(configuration), doBuildTeams(true),
|
||||
lastBuildTeamsFailed(false), teamBuilder(Void()), lock(lock), output(output), unhealthyServers(0),
|
||||
shardsAffectedByTeamFailure(shardsAffectedByTeamFailure),
|
||||
initialFailureReactionDelay(
|
||||
delayed(readyToStart, SERVER_KNOBS->INITIAL_FAILURE_REACTION_DELAY, TaskPriority::DataDistribution)),
|
||||
healthyTeamCount(0), storageServerSet(new LocalityMap<UID>()),
|
||||
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), isTssRecruiting(false),
|
||||
medianAvailableSpace(SERVER_KNOBS->MIN_AVAILABLE_SPACE_RATIO), lastMedianAvailableSpaceUpdate(0),
|
||||
processingUnhealthy(processingUnhealthy), lowestUtilizationTeam(0), highestUtilizationTeam(0),
|
||||
getShardMetrics(getShardMetrics), removeFailedServer(removeFailedServer),
|
||||
getUnhealthyRelocationCount(getUnhealthyRelocationCount) {
|
||||
recruitingStream(0), restartRecruiting(SERVER_KNOBS->DEBOUNCE_RECRUITING_DELAY), healthyTeamCount(0),
|
||||
zeroHealthyTeams(zeroHealthyTeams), optimalTeamCount(0), zeroOptimalTeams(true), isTssRecruiting(false),
|
||||
includedDCs(includedDCs), otherTrackedDCs(otherTrackedDCs), primary(primary),
|
||||
processingUnhealthy(processingUnhealthy), readyToStart(readyToStart),
|
||||
checkTeamDelay(delay(SERVER_KNOBS->CHECK_TEAM_DELAY, TaskPriority::DataDistribution)), badTeamRemover(Void()),
|
||||
checkInvalidLocalities(Void()), wrongStoreTypeRemover(Void()), storageServerSet(new LocalityMap<UID>()),
|
||||
clearHealthyZoneFuture(true), medianAvailableSpace(SERVER_KNOBS->MIN_AVAILABLE_SPACE_RATIO),
|
||||
lastMedianAvailableSpaceUpdate(0), lowestUtilizationTeam(0), highestUtilizationTeam(0),
|
||||
getShardMetrics(getShardMetrics), getUnhealthyRelocationCount(getUnhealthyRelocationCount),
|
||||
removeFailedServer(removeFailedServer) {
|
||||
if (!primary || configuration.usableRegions == 1) {
|
||||
TraceEvent("DDTrackerStarting", distributorId).detail("State", "Inactive").trackLatest("DDTrackerStarting");
|
||||
}
|
||||
|
@ -4987,7 +4987,7 @@ struct TSSPairState : ReferenceCounted<TSSPairState>, NonCopyable {
|
|||
TSSPairState() : active(false) {}
|
||||
|
||||
TSSPairState(const LocalityData& locality)
|
||||
: active(true), dcId(locality.dcId()), dataHallId(locality.dataHallId()) {}
|
||||
: dcId(locality.dcId()), dataHallId(locality.dataHallId()), active(true) {}
|
||||
|
||||
bool inDataZone(const LocalityData& locality) {
|
||||
return locality.dcId() == dcId && locality.dataHallId() == dataHallId;
|
||||
|
|
|
@ -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<RelocateShard> input,
|
||||
PromiseStream<GetMetricsRequest> 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<bool>(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<bool>(false)), unhealthyRelocations(0) {}
|
||||
|
||||
void validate() {
|
||||
if (EXPENSIVE_VALIDATION) {
|
||||
|
|
|
@ -123,10 +123,10 @@ struct DataDistributionTracker {
|
|||
Reference<AsyncVar<bool>> anyZeroHealthyTeams,
|
||||
KeyRangeMap<ShardTrackedData>& shards,
|
||||
bool& trackerCancelled)
|
||||
: cx(cx), distributorId(distributorId), dbSizeEstimate(new AsyncVar<int64_t>()), systemSizeEstimate(0),
|
||||
maxShardSize(new AsyncVar<Optional<int64_t>>()), 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<int64_t>()), maxShardSize(new AsyncVar<Optional<int64_t>>()), output(output),
|
||||
shardsAffectedByTeamFailure(shardsAffectedByTeamFailure), readyToStart(readyToStart),
|
||||
anyZeroHealthyTeams(anyZeroHealthyTeams), trackerCancelled(trackerCancelled) {}
|
||||
|
||||
~DataDistributionTracker() {
|
||||
trackerCancelled = true;
|
||||
|
|
|
@ -239,19 +239,19 @@ public:
|
|||
|
||||
// construct root node
|
||||
DecodedNode(Node* raw, const T* prev, const T* next, Arena& arena, bool large)
|
||||
: raw(raw), parent(nullptr), otherAncestor(nullptr), leftChild(nullptr), rightChild(nullptr), prev(prev),
|
||||
next(next), item(raw->delta(large).apply(raw->delta(large).getPrefixSource() ? *prev : *next, arena)),
|
||||
large(large) {
|
||||
: large(large), raw(raw), parent(nullptr), otherAncestor(nullptr), leftChild(nullptr), rightChild(nullptr),
|
||||
prev(prev), next(next),
|
||||
item(raw->delta(large).apply(raw->delta(large).getPrefixSource() ? *prev : *next, arena)) {
|
||||
// printf("DecodedNode1 raw=%p delta=%s\n", raw, raw->delta(large).toString().c_str());
|
||||
}
|
||||
|
||||
// Construct non-root node
|
||||
// wentLeft indicates that we've gone left to get to the raw node.
|
||||
DecodedNode(Node* raw, DecodedNode* parent, bool wentLeft, Arena& arena)
|
||||
: parent(parent), large(parent->large),
|
||||
otherAncestor(wentLeft ? parent->getPrevAncestor() : parent->getNextAncestor()),
|
||||
prev(wentLeft ? parent->prev : &parent->item), next(wentLeft ? &parent->item : parent->next),
|
||||
leftChild(nullptr), rightChild(nullptr), raw(raw),
|
||||
: large(parent->large), raw(raw), parent(parent),
|
||||
otherAncestor(wentLeft ? parent->getPrevAncestor() : parent->getNextAncestor()), leftChild(nullptr),
|
||||
rightChild(nullptr), prev(wentLeft ? parent->prev : &parent->item),
|
||||
next(wentLeft ? &parent->item : parent->next),
|
||||
item(raw->delta(large).apply(raw->delta(large).getPrefixSource() ? *prev : *next, arena)) {
|
||||
// printf("DecodedNode2 raw=%p delta=%s\n", raw, raw->delta(large).toString().c_str());
|
||||
}
|
||||
|
@ -1134,12 +1134,12 @@ public:
|
|||
struct Cursor {
|
||||
Cursor() : cache(nullptr), nodeIndex(-1) {}
|
||||
|
||||
Cursor(DecodeCache* cache, DeltaTree2* tree) : cache(cache), tree(tree), nodeIndex(-1) {}
|
||||
Cursor(DecodeCache* cache, DeltaTree2* tree) : tree(tree), cache(cache), nodeIndex(-1) {}
|
||||
|
||||
Cursor(DecodeCache* cache, DeltaTree2* tree, int nodeIndex) : cache(cache), tree(tree), nodeIndex(nodeIndex) {}
|
||||
Cursor(DecodeCache* cache, DeltaTree2* tree, int nodeIndex) : tree(tree), cache(cache), nodeIndex(nodeIndex) {}
|
||||
|
||||
// Copy constructor does not copy item because normally a copied cursor will be immediately moved.
|
||||
Cursor(const Cursor& c) : cache(c.cache), tree(c.tree), nodeIndex(c.nodeIndex) {}
|
||||
Cursor(const Cursor& c) : tree(c.tree), cache(c.cache), nodeIndex(c.nodeIndex) {}
|
||||
|
||||
Cursor next() const {
|
||||
Cursor c = *this;
|
||||
|
|
|
@ -168,11 +168,11 @@ private:
|
|||
class RawDiskQueue_TwoFiles : public Tracked<RawDiskQueue_TwoFiles> {
|
||||
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);
|
||||
|
|
|
@ -81,8 +81,8 @@ struct GrvProxyStats {
|
|||
|
||||
// Current stats maintained for a given grv proxy server
|
||||
explicit GrvProxyStats(UID id)
|
||||
: cc("GrvProxyStats", id.toString()), recentRequests(0), lastBucketBegin(now()),
|
||||
bucketInterval(FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE / FLOW_KNOBS->BASIC_LOAD_BALANCE_BUCKETS),
|
||||
: cc("GrvProxyStats", id.toString()),
|
||||
|
||||
txnRequestIn("TxnRequestIn", cc), txnRequestOut("TxnRequestOut", cc), txnRequestErrors("TxnRequestErrors", cc),
|
||||
txnStartIn("TxnStartIn", cc), txnStartOut("TxnStartOut", cc), txnStartBatch("TxnStartBatch", cc),
|
||||
txnSystemPriorityStartIn("TxnSystemPriorityStartIn", cc),
|
||||
|
@ -102,6 +102,7 @@ struct GrvProxyStats {
|
|||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
grvLatencyBands("GRVLatencyBands", id, SERVER_KNOBS->STORAGE_LOGGING_DELAY),
|
||||
grvLatencySample("GRVLatencyMetrics",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
|
@ -110,7 +111,8 @@ struct GrvProxyStats {
|
|||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
grvLatencyBands("GRVLatencyBands", id, SERVER_KNOBS->STORAGE_LOGGING_DELAY) {
|
||||
recentRequests(0), lastBucketBegin(now()),
|
||||
bucketInterval(FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE / FLOW_KNOBS->BASIC_LOAD_BALANCE_BUCKETS) {
|
||||
// The rate at which the limit(budget) is allowed to grow.
|
||||
specialCounter(cc, "SystemGRVQueueSize", [this]() { return this->systemGRVQueueSize; });
|
||||
specialCounter(cc, "DefaultGRVQueueSize", [this]() { return this->defaultGRVQueueSize; });
|
||||
|
|
|
@ -861,10 +861,10 @@ KeyValueStoreMemory<Container>::KeyValueStoreMemory(IDiskQueue* log,
|
|||
bool disableSnapshot,
|
||||
bool replaceContent,
|
||||
bool exactRecovery)
|
||||
: log(log), id(id), type(storeType), previousSnapshotEnd(-1), currentSnapshotEnd(-1), resetSnapshot(false),
|
||||
memoryLimit(memoryLimit), committedWriteBytes(0), overheadWriteBytes(0), committedDataSize(0), transactionSize(0),
|
||||
transactionIsLarge(false), disableSnapshot(disableSnapshot), replaceContent(replaceContent), snapshotCount(0),
|
||||
firstCommitWithSnapshot(true) {
|
||||
: type(storeType), id(id), log(log), committedWriteBytes(0), overheadWriteBytes(0), currentSnapshotEnd(-1),
|
||||
previousSnapshotEnd(-1), committedDataSize(0), transactionSize(0), transactionIsLarge(false), resetSnapshot(false),
|
||||
disableSnapshot(disableSnapshot), replaceContent(replaceContent), firstCommitWithSnapshot(true), snapshotCount(0),
|
||||
memoryLimit(memoryLimit) {
|
||||
// create reserved buffer for radixtree store type
|
||||
this->reserved_buffer =
|
||||
(storeType == KeyValueStoreType::MEMORY) ? nullptr : new uint8_t[CLIENT_KNOBS->SYSTEM_KEY_SIZE_LIMIT];
|
||||
|
|
|
@ -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<Reference<ReadCursor>>* 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).log();
|
||||
|
@ -2110,7 +2110,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);
|
||||
|
|
|
@ -326,10 +326,10 @@ public:
|
|||
std::string const& configPath,
|
||||
std::map<std::string, std::string> const& manualKnobOverrides,
|
||||
IsTest isTest)
|
||||
: id(deterministicRandom()->randomUniqueID()), kvStore(dataFolder, id, "localconf-"), cc("LocalConfiguration"),
|
||||
: id(deterministicRandom()->randomUniqueID()), kvStore(dataFolder, id, "localconf-"),
|
||||
configKnobOverrides(configPath), manualKnobOverrides(manualKnobOverrides), cc("LocalConfiguration"),
|
||||
broadcasterChanges("BroadcasterChanges", cc), snapshots("Snapshots", cc),
|
||||
changeRequestsFetched("ChangeRequestsFetched", cc), mutations("Mutations", cc), configKnobOverrides(configPath),
|
||||
manualKnobOverrides(manualKnobOverrides) {
|
||||
changeRequestsFetched("ChangeRequestsFetched", cc), mutations("Mutations", cc) {
|
||||
if (isTest) {
|
||||
testKnobCollection =
|
||||
IKnobCollection::create(IKnobCollection::Type::TEST,
|
||||
|
|
|
@ -43,11 +43,11 @@ struct LogRouterData {
|
|||
Tag tag;
|
||||
|
||||
TagData(Tag tag, Version popped, Version durableKnownCommittedVersion)
|
||||
: tag(tag), popped(popped), durableKnownCommittedVersion(durableKnownCommittedVersion) {}
|
||||
: popped(popped), durableKnownCommittedVersion(durableKnownCommittedVersion), tag(tag) {}
|
||||
|
||||
TagData(TagData&& r) noexcept
|
||||
: version_messages(std::move(r.version_messages)), tag(r.tag), popped(r.popped),
|
||||
durableKnownCommittedVersion(r.durableKnownCommittedVersion) {}
|
||||
: version_messages(std::move(r.version_messages)), popped(r.popped),
|
||||
durableKnownCommittedVersion(r.durableKnownCommittedVersion), tag(r.tag) {}
|
||||
void operator=(TagData&& r) noexcept {
|
||||
version_messages = std::move(r.version_messages);
|
||||
tag = r.tag;
|
||||
|
@ -136,14 +136,14 @@ struct LogRouterData {
|
|||
}
|
||||
|
||||
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),
|
||||
: dbgid(dbgid), logSystem(new AsyncVar<Reference<ILogSystem>>()), version(req.startVersion - 1), minPopped(0),
|
||||
startVersion(req.startVersion), minKnownCommittedVersion(0), poppedVersion(0), routerTag(req.routerTag),
|
||||
allowPops(false), foundEpochEnd(false), generation(req.recoveryCount),
|
||||
peekLatencyDist(Histogram::getHistogram(LiteralStringRef("LogRouter"),
|
||||
LiteralStringRef("PeekTLogLatency"),
|
||||
Histogram::Unit::microseconds)) {
|
||||
Histogram::Unit::microseconds)),
|
||||
cc("LogRouter", dbgid.toString()), getMoreCount("GetMoreCount", cc),
|
||||
getMoreBlockedCount("GetMoreBlockedCount", cc) {
|
||||
// setup just enough of a logSet to be able to call getPushLocations
|
||||
logSet.logServers.resize(req.tLogLocalities.size());
|
||||
logSet.tLogPolicy = req.tLogPolicy;
|
||||
|
|
|
@ -47,7 +47,7 @@ struct ConnectionResetInfo : public ReferenceCounted<ConnectionResetInfo> {
|
|||
int slowReplies;
|
||||
int fastReplies;
|
||||
|
||||
ConnectionResetInfo() : lastReset(now()), slowReplies(0), fastReplies(0), resetCheck(Void()) {}
|
||||
ConnectionResetInfo() : lastReset(now()), resetCheck(Void()), slowReplies(0), fastReplies(0) {}
|
||||
};
|
||||
|
||||
// The set of tLog servers, logRouters and backupWorkers for a log tag
|
||||
|
|
|
@ -60,9 +60,9 @@ public:
|
|||
Reference<AsyncVar<PeekTxsInfo>> peekLocality,
|
||||
Version txsPoppedVersion,
|
||||
bool recover)
|
||||
: logSystem(logSystem), peekLocality(peekLocality), enableRecovery(recover), recoveryLoc(txsPoppedVersion),
|
||||
recoveryQueueLoc(txsPoppedVersion), poppedUpTo(0), nextCommit(1), recoveryQueueDataSize(0), peekTypeSwitches(0),
|
||||
hasDiscardedData(false), totalRecoveredBytes(0) {
|
||||
: peekLocality(peekLocality), peekTypeSwitches(0), enableRecovery(recover), logSystem(logSystem),
|
||||
recoveryLoc(txsPoppedVersion), recoveryQueueLoc(txsPoppedVersion), recoveryQueueDataSize(0), poppedUpTo(0),
|
||||
nextCommit(1), hasDiscardedData(false), totalRecoveredBytes(0) {
|
||||
if (enableRecovery) {
|
||||
localityChanged = peekLocality ? peekLocality->onChange() : Never();
|
||||
cursor = logSystem->peekTxs(UID(),
|
||||
|
|
|
@ -31,11 +31,10 @@ ILogSystem::ServerPeekCursor::ServerPeekCursor(Reference<AsyncVar<OptionalInterf
|
|||
Version end,
|
||||
bool returnIfBlocked,
|
||||
bool parallelGetMore)
|
||||
: interf(interf), tag(tag), messageVersion(begin), end(end), hasMsg(false),
|
||||
rd(results.arena, results.messages, Unversioned()), randomID(deterministicRandom()->randomUniqueID()),
|
||||
poppedVersion(0), returnIfBlocked(returnIfBlocked), sequence(0), onlySpilled(false),
|
||||
parallelGetMore(parallelGetMore), lastReset(0), slowReplies(0), fastReplies(0), unknownReplies(0),
|
||||
resetCheck(Void()) {
|
||||
: interf(interf), tag(tag), rd(results.arena, results.messages, Unversioned()), messageVersion(begin), end(end),
|
||||
poppedVersion(0), hasMsg(false), randomID(deterministicRandom()->randomUniqueID()),
|
||||
returnIfBlocked(returnIfBlocked), onlySpilled(false), parallelGetMore(parallelGetMore), sequence(0), lastReset(0),
|
||||
resetCheck(Void()), slowReplies(0), fastReplies(0), unknownReplies(0) {
|
||||
this->results.maxKnownVersion = 0;
|
||||
this->results.minKnownCommittedVersion = 0;
|
||||
//TraceEvent("SPC_Starting", randomID).detail("Tag", tag.toString()).detail("Begin", begin).detail("End", end).backtrace();
|
||||
|
@ -48,10 +47,11 @@ ILogSystem::ServerPeekCursor::ServerPeekCursor(TLogPeekReply const& results,
|
|||
bool hasMsg,
|
||||
Version poppedVersion,
|
||||
Tag tag)
|
||||
: results(results), tag(tag), rd(results.arena, results.messages, Unversioned()), messageVersion(messageVersion),
|
||||
end(end), messageAndTags(message), hasMsg(hasMsg), randomID(deterministicRandom()->randomUniqueID()),
|
||||
poppedVersion(poppedVersion), returnIfBlocked(false), sequence(0), onlySpilled(false), parallelGetMore(false),
|
||||
lastReset(0), slowReplies(0), fastReplies(0), unknownReplies(0), resetCheck(Void()) {
|
||||
: tag(tag), results(results), rd(results.arena, results.messages, Unversioned()), messageVersion(messageVersion),
|
||||
end(end), poppedVersion(poppedVersion), messageAndTags(message), hasMsg(hasMsg),
|
||||
randomID(deterministicRandom()->randomUniqueID()), returnIfBlocked(false), onlySpilled(false),
|
||||
parallelGetMore(false), sequence(0), lastReset(0), resetCheck(Void()), slowReplies(0), fastReplies(0),
|
||||
unknownReplies(0) {
|
||||
//TraceEvent("SPC_Clone", randomID);
|
||||
this->results.maxKnownVersion = 0;
|
||||
this->results.minKnownCommittedVersion = 0;
|
||||
|
@ -409,8 +409,8 @@ Version ILogSystem::ServerPeekCursor::popped() const {
|
|||
|
||||
ILogSystem::MergedPeekCursor::MergedPeekCursor(vector<Reference<ILogSystem::IPeekCursor>> const& serverCursors,
|
||||
Version begin)
|
||||
: serverCursors(serverCursors), bestServer(-1), readQuorum(serverCursors.size()), tag(invalidTag), currentCursor(0),
|
||||
hasNextMessage(false), messageVersion(begin), randomID(deterministicRandom()->randomUniqueID()),
|
||||
: serverCursors(serverCursors), tag(invalidTag), bestServer(-1), currentCursor(0), readQuorum(serverCursors.size()),
|
||||
messageVersion(begin), hasNextMessage(false), randomID(deterministicRandom()->randomUniqueID()),
|
||||
tLogReplicationFactor(0) {
|
||||
sortedVersions.resize(serverCursors.size());
|
||||
}
|
||||
|
@ -426,8 +426,8 @@ ILogSystem::MergedPeekCursor::MergedPeekCursor(
|
|||
std::vector<LocalityData> const& tLogLocalities,
|
||||
Reference<IReplicationPolicy> const tLogPolicy,
|
||||
int tLogReplicationFactor)
|
||||
: bestServer(bestServer), readQuorum(readQuorum), tag(tag), currentCursor(0), hasNextMessage(false),
|
||||
messageVersion(begin), randomID(deterministicRandom()->randomUniqueID()),
|
||||
: tag(tag), bestServer(bestServer), currentCursor(0), readQuorum(readQuorum), messageVersion(begin),
|
||||
hasNextMessage(false), randomID(deterministicRandom()->randomUniqueID()),
|
||||
tLogReplicationFactor(tLogReplicationFactor) {
|
||||
if (tLogPolicy) {
|
||||
logSet = makeReference<LogSet>();
|
||||
|
@ -453,8 +453,8 @@ ILogSystem::MergedPeekCursor::MergedPeekCursor(vector<Reference<ILogSystem::IPee
|
|||
Optional<LogMessageVersion> nextVersion,
|
||||
Reference<LogSet> logSet,
|
||||
int tLogReplicationFactor)
|
||||
: serverCursors(serverCursors), bestServer(bestServer), readQuorum(readQuorum), currentCursor(0),
|
||||
hasNextMessage(false), messageVersion(messageVersion), nextVersion(nextVersion), logSet(logSet),
|
||||
: logSet(logSet), serverCursors(serverCursors), bestServer(bestServer), currentCursor(0), readQuorum(readQuorum),
|
||||
nextVersion(nextVersion), messageVersion(messageVersion), hasNextMessage(false),
|
||||
randomID(deterministicRandom()->randomUniqueID()), tLogReplicationFactor(tLogReplicationFactor) {
|
||||
sortedVersions.resize(serverCursors.size());
|
||||
calcHasMessage();
|
||||
|
@ -698,8 +698,8 @@ ILogSystem::SetPeekCursor::SetPeekCursor(std::vector<Reference<LogSet>> const& l
|
|||
Version begin,
|
||||
Version end,
|
||||
bool parallelGetMore)
|
||||
: logSets(logSets), bestSet(bestSet), bestServer(bestServer), tag(tag), currentCursor(0), currentSet(bestSet),
|
||||
hasNextMessage(false), messageVersion(begin), useBestSet(true), randomID(deterministicRandom()->randomUniqueID()) {
|
||||
: logSets(logSets), tag(tag), bestSet(bestSet), bestServer(bestServer), currentSet(bestSet), currentCursor(0),
|
||||
messageVersion(begin), hasNextMessage(false), useBestSet(true), randomID(deterministicRandom()->randomUniqueID()) {
|
||||
serverCursors.resize(logSets.size());
|
||||
int maxServers = 0;
|
||||
for (int i = 0; i < logSets.size(); i++) {
|
||||
|
@ -720,8 +720,8 @@ ILogSystem::SetPeekCursor::SetPeekCursor(std::vector<Reference<LogSet>> const& l
|
|||
int bestServer,
|
||||
Optional<LogMessageVersion> nextVersion,
|
||||
bool useBestSet)
|
||||
: logSets(logSets), serverCursors(serverCursors), messageVersion(messageVersion), bestSet(bestSet),
|
||||
bestServer(bestServer), nextVersion(nextVersion), currentSet(bestSet), currentCursor(0), hasNextMessage(false),
|
||||
: logSets(logSets), serverCursors(serverCursors), bestSet(bestSet), bestServer(bestServer), currentSet(bestSet),
|
||||
currentCursor(0), nextVersion(nextVersion), messageVersion(messageVersion), hasNextMessage(false),
|
||||
useBestSet(useBestSet), randomID(deterministicRandom()->randomUniqueID()) {
|
||||
int maxServers = 0;
|
||||
for (int i = 0; i < logSets.size(); i++) {
|
||||
|
@ -1155,10 +1155,9 @@ ILogSystem::BufferedCursor::BufferedCursor(std::vector<Reference<IPeekCursor>> c
|
|||
bool withTags,
|
||||
bool collectTags,
|
||||
bool canDiscardPopped)
|
||||
: cursors(cursors), messageVersion(begin), end(end), withTags(withTags), collectTags(collectTags),
|
||||
hasNextMessage(false), messageIndex(0), poppedVersion(0), initialPoppedVersion(0),
|
||||
canDiscardPopped(canDiscardPopped), knownUnique(false), minKnownCommittedVersion(0),
|
||||
randomID(deterministicRandom()->randomUniqueID()) {
|
||||
: cursors(cursors), messageIndex(0), messageVersion(begin), end(end), hasNextMessage(false), withTags(withTags),
|
||||
knownUnique(false), minKnownCommittedVersion(0), poppedVersion(0), initialPoppedVersion(0),
|
||||
canDiscardPopped(canDiscardPopped), randomID(deterministicRandom()->randomUniqueID()), collectTags(collectTags) {
|
||||
targetQueueSize = SERVER_KNOBS->DESIRED_OUTSTANDING_MESSAGES / cursors.size();
|
||||
messages.reserve(SERVER_KNOBS->DESIRED_OUTSTANDING_MESSAGES);
|
||||
cursorMessages.resize(cursors.size());
|
||||
|
@ -1170,9 +1169,9 @@ ILogSystem::BufferedCursor::BufferedCursor(
|
|||
Version begin,
|
||||
Version end,
|
||||
bool parallelGetMore)
|
||||
: messageVersion(begin), end(end), withTags(true), collectTags(false), hasNextMessage(false), messageIndex(0),
|
||||
poppedVersion(0), initialPoppedVersion(0), canDiscardPopped(false), knownUnique(true), minKnownCommittedVersion(0),
|
||||
randomID(deterministicRandom()->randomUniqueID()) {
|
||||
: messageIndex(0), messageVersion(begin), end(end), hasNextMessage(false), withTags(true), knownUnique(true),
|
||||
minKnownCommittedVersion(0), poppedVersion(0), initialPoppedVersion(0), canDiscardPopped(false),
|
||||
randomID(deterministicRandom()->randomUniqueID()), collectTags(false) {
|
||||
targetQueueSize = SERVER_KNOBS->DESIRED_OUTSTANDING_MESSAGES / logServers.size();
|
||||
messages.reserve(SERVER_KNOBS->DESIRED_OUTSTANDING_MESSAGES);
|
||||
cursorMessages.resize(logServers.size());
|
||||
|
|
|
@ -154,7 +154,7 @@ struct GetCommitVersionReply {
|
|||
|
||||
GetCommitVersionReply() : resolverChangesVersion(0), version(0), prevVersion(0), requestNum(0) {}
|
||||
explicit GetCommitVersionReply(Version version, Version prevVersion, uint64_t requestNum)
|
||||
: version(version), prevVersion(prevVersion), resolverChangesVersion(0), requestNum(requestNum) {}
|
||||
: resolverChangesVersion(0), version(version), prevVersion(prevVersion), requestNum(requestNum) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
|
|
|
@ -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<StringRef> typePattern;
|
||||
Standalone<StringRef> namePattern;
|
||||
|
|
|
@ -90,7 +90,7 @@ struct TLogQueueEntryRef {
|
|||
|
||||
TLogQueueEntryRef() : version(0), knownCommittedVersion(0) {}
|
||||
TLogQueueEntryRef(Arena& a, TLogQueueEntryRef const& from)
|
||||
: version(from.version), knownCommittedVersion(from.knownCommittedVersion), id(from.id),
|
||||
: id(from.id), version(from.version), knownCommittedVersion(from.knownCommittedVersion),
|
||||
messages(a, from.messages), tags(a, from.tags) {}
|
||||
|
||||
template <class Ar>
|
||||
|
@ -322,10 +322,10 @@ struct TLogData : NonCopyable {
|
|||
IKeyValueStore* persistentData,
|
||||
IDiskQueue* persistentQueue,
|
||||
Reference<AsyncVar<ServerDBInfo> const> const& dbInfo)
|
||||
: dbgid(dbgid), workerID(workerID), instanceID(deterministicRandom()->randomUniqueID().first()),
|
||||
persistentData(persistentData), rawPersistentQueue(persistentQueue),
|
||||
persistentQueue(new TLogQueue(persistentQueue, dbgid)), dbInfo(dbInfo), queueCommitBegin(0), queueCommitEnd(0),
|
||||
prevVersion(0), diskQueueCommitBytes(0), largeDiskQueueCommitBytes(false), bytesInput(0), bytesDurable(0),
|
||||
: dbgid(dbgid), workerID(workerID), persistentData(persistentData), rawPersistentQueue(persistentQueue),
|
||||
persistentQueue(new TLogQueue(persistentQueue, dbgid)), diskQueueCommitBytes(0),
|
||||
largeDiskQueueCommitBytes(false), dbInfo(dbInfo), queueCommitEnd(0), queueCommitBegin(0),
|
||||
instanceID(deterministicRandom()->randomUniqueID().first()), bytesInput(0), bytesDurable(0), prevVersion(0),
|
||||
updatePersist(Void()), terminated(false) {}
|
||||
};
|
||||
|
||||
|
@ -339,7 +339,7 @@ struct LogData : NonCopyable, public ReferenceCounted<LogData> {
|
|||
bool update_version_sizes;
|
||||
|
||||
TagData(Version popped, bool nothing_persistent, bool popped_recently, OldTag tag)
|
||||
: nothing_persistent(nothing_persistent), popped(popped), popped_recently(popped_recently),
|
||||
: nothing_persistent(nothing_persistent), popped_recently(popped_recently), popped(popped),
|
||||
update_version_sizes(tag != txsTagOld) {}
|
||||
|
||||
TagData(TagData&& r) noexcept
|
||||
|
@ -440,11 +440,10 @@ struct LogData : NonCopyable, public ReferenceCounted<LogData> {
|
|||
Future<Void> recovery;
|
||||
|
||||
explicit LogData(TLogData* tLogData, TLogInterface interf)
|
||||
: tLogData(tLogData), knownCommittedVersion(0), tli(interf), logId(interf.id()),
|
||||
: stopped(false), initialized(false), recoveryCount(), queueCommittingVersion(0), knownCommittedVersion(0),
|
||||
cc("TLog", interf.id().toString()), bytesInput("BytesInput", cc), bytesDurable("BytesDurable", cc),
|
||||
// These are initialized differently on init() or recovery
|
||||
recoveryCount(), stopped(false), initialized(false), queueCommittingVersion(0),
|
||||
newPersistentDataVersion(invalidVersion), recovery(Void()) {
|
||||
logId(interf.id()), newPersistentDataVersion(invalidVersion), tli(interf), tLogData(tLogData),
|
||||
recovery(Void()) {
|
||||
startRole(Role::TRANSACTION_LOG,
|
||||
interf.id(),
|
||||
tLogData->workerID,
|
||||
|
|
|
@ -57,7 +57,7 @@ struct TLogQueueEntryRef {
|
|||
|
||||
TLogQueueEntryRef() : version(0), knownCommittedVersion(0) {}
|
||||
TLogQueueEntryRef(Arena& a, TLogQueueEntryRef const& from)
|
||||
: version(from.version), knownCommittedVersion(from.knownCommittedVersion), id(from.id),
|
||||
: id(from.id), version(from.version), knownCommittedVersion(from.knownCommittedVersion),
|
||||
messages(a, from.messages) {}
|
||||
|
||||
template <class Ar>
|
||||
|
@ -304,13 +304,13 @@ struct TLogData : NonCopyable {
|
|||
Reference<AsyncVar<ServerDBInfo> const> dbInfo,
|
||||
Reference<AsyncVar<bool>> degraded,
|
||||
std::string folder)
|
||||
: dbgid(dbgid), workerID(workerID), instanceID(deterministicRandom()->randomUniqueID().first()),
|
||||
persistentData(persistentData), rawPersistentQueue(persistentQueue),
|
||||
persistentQueue(new TLogQueue(persistentQueue, dbgid)), dbInfo(dbInfo), degraded(degraded), queueCommitBegin(0),
|
||||
queueCommitEnd(0), diskQueueCommitBytes(0), largeDiskQueueCommitBytes(false), bytesInput(0), bytesDurable(0),
|
||||
: dbgid(dbgid), workerID(workerID), persistentData(persistentData), rawPersistentQueue(persistentQueue),
|
||||
persistentQueue(new TLogQueue(persistentQueue, dbgid)), diskQueueCommitBytes(0),
|
||||
largeDiskQueueCommitBytes(false), dbInfo(dbInfo), queueCommitEnd(0), queueCommitBegin(0),
|
||||
instanceID(deterministicRandom()->randomUniqueID().first()), bytesInput(0), bytesDurable(0),
|
||||
targetVolatileBytes(SERVER_KNOBS->TLOG_SPILL_THRESHOLD), overheadBytesInput(0), overheadBytesDurable(0),
|
||||
concurrentLogRouterReads(SERVER_KNOBS->CONCURRENT_LOG_ROUTER_READS), ignorePopRequest(false),
|
||||
ignorePopDeadline(), ignorePopUid(), dataFolder(folder), toBePopped() {
|
||||
dataFolder(folder), degraded(degraded) {
|
||||
cx = openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::True);
|
||||
}
|
||||
};
|
||||
|
@ -326,12 +326,12 @@ struct LogData : NonCopyable, public ReferenceCounted<LogData> {
|
|||
Tag tag;
|
||||
|
||||
TagData(Tag tag, Version popped, bool nothingPersistent, bool poppedRecently, bool unpoppedRecovered)
|
||||
: tag(tag), nothingPersistent(nothingPersistent), popped(popped), poppedRecently(poppedRecently),
|
||||
unpoppedRecovered(unpoppedRecovered) {}
|
||||
: nothingPersistent(nothingPersistent), poppedRecently(poppedRecently), popped(popped),
|
||||
unpoppedRecovered(unpoppedRecovered), tag(tag) {}
|
||||
|
||||
TagData(TagData&& r) noexcept
|
||||
: versionMessages(std::move(r.versionMessages)), nothingPersistent(r.nothingPersistent),
|
||||
poppedRecently(r.poppedRecently), popped(r.popped), tag(r.tag), unpoppedRecovered(r.unpoppedRecovered) {}
|
||||
poppedRecently(r.poppedRecently), popped(r.popped), unpoppedRecovered(r.unpoppedRecovered), tag(r.tag) {}
|
||||
void operator=(TagData&& r) noexcept {
|
||||
versionMessages = std::move(r.versionMessages);
|
||||
nothingPersistent = r.nothingPersistent;
|
||||
|
@ -524,15 +524,14 @@ struct LogData : NonCopyable, public ReferenceCounted<LogData> {
|
|||
UID recruitmentID,
|
||||
std::vector<Tag> tags,
|
||||
std::string context)
|
||||
: tLogData(tLogData), knownCommittedVersion(0), logId(interf.id()), cc("TLog", interf.id().toString()),
|
||||
bytesInput("BytesInput", cc), bytesDurable("BytesDurable", cc), remoteTag(remoteTag), isPrimary(isPrimary),
|
||||
logRouterTags(logRouterTags), txsTags(txsTags), recruitmentID(recruitmentID),
|
||||
logSystem(new AsyncVar<Reference<ILogSystem>>()), logRouterPoppedVersion(0), durableKnownCommittedVersion(0),
|
||||
minKnownCommittedVersion(0), allTags(tags.begin(), tags.end()), terminated(tLogData->terminated.getFuture()),
|
||||
// These are initialized differently on init() or recovery
|
||||
recoveryCount(), stopped(false), initialized(false), queueCommittingVersion(0),
|
||||
newPersistentDataVersion(invalidVersion), unrecoveredBefore(1), recoveredAt(1), unpoppedRecoveredTags(0),
|
||||
logRouterPopToVersion(0), locality(tagLocalityInvalid), execOpCommitInProgress(false) {
|
||||
: stopped(false), initialized(false), queueCommittingVersion(0), knownCommittedVersion(0),
|
||||
durableKnownCommittedVersion(0), minKnownCommittedVersion(0), unpoppedRecoveredTags(0),
|
||||
cc("TLog", interf.id().toString()), bytesInput("BytesInput", cc), bytesDurable("BytesDurable", cc),
|
||||
logId(interf.id()), newPersistentDataVersion(invalidVersion), tLogData(tLogData), unrecoveredBefore(1),
|
||||
recoveredAt(1), logSystem(new AsyncVar<Reference<ILogSystem>>()), remoteTag(remoteTag), isPrimary(isPrimary),
|
||||
logRouterTags(logRouterTags), logRouterPoppedVersion(0), logRouterPopToVersion(0), locality(tagLocalityInvalid),
|
||||
recruitmentID(recruitmentID), allTags(tags.begin(), tags.end()), terminated(tLogData->terminated.getFuture()),
|
||||
execOpCommitInProgress(false), txsTags(txsTags) {
|
||||
startRole(Role::TRANSACTION_LOG,
|
||||
interf.id(),
|
||||
tLogData->workerID,
|
||||
|
|
|
@ -58,7 +58,7 @@ struct TLogQueueEntryRef {
|
|||
|
||||
TLogQueueEntryRef() : version(0), knownCommittedVersion(0) {}
|
||||
TLogQueueEntryRef(Arena& a, TLogQueueEntryRef const& from)
|
||||
: version(from.version), knownCommittedVersion(from.knownCommittedVersion), id(from.id),
|
||||
: id(from.id), version(from.version), knownCommittedVersion(from.knownCommittedVersion),
|
||||
messages(a, from.messages) {}
|
||||
|
||||
template <class Ar>
|
||||
|
@ -367,14 +367,14 @@ struct TLogData : NonCopyable {
|
|||
Reference<AsyncVar<ServerDBInfo> const> dbInfo,
|
||||
Reference<AsyncVar<bool>> degraded,
|
||||
std::string folder)
|
||||
: dbgid(dbgid), workerID(workerID), instanceID(deterministicRandom()->randomUniqueID().first()),
|
||||
persistentData(persistentData), rawPersistentQueue(persistentQueue),
|
||||
persistentQueue(new TLogQueue(persistentQueue, dbgid)), dbInfo(dbInfo), degraded(degraded), queueCommitBegin(0),
|
||||
queueCommitEnd(0), diskQueueCommitBytes(0), largeDiskQueueCommitBytes(false), bytesInput(0), bytesDurable(0),
|
||||
: dbgid(dbgid), workerID(workerID), persistentData(persistentData), rawPersistentQueue(persistentQueue),
|
||||
persistentQueue(new TLogQueue(persistentQueue, dbgid)), diskQueueCommitBytes(0),
|
||||
largeDiskQueueCommitBytes(false), dbInfo(dbInfo), queueCommitEnd(0), queueCommitBegin(0),
|
||||
instanceID(deterministicRandom()->randomUniqueID().first()), bytesInput(0), bytesDurable(0),
|
||||
targetVolatileBytes(SERVER_KNOBS->TLOG_SPILL_THRESHOLD), overheadBytesInput(0), overheadBytesDurable(0),
|
||||
peekMemoryLimiter(SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_PEEK_MEMORY_BYTES),
|
||||
concurrentLogRouterReads(SERVER_KNOBS->CONCURRENT_LOG_ROUTER_READS), ignorePopRequest(false),
|
||||
ignorePopDeadline(), ignorePopUid(), dataFolder(folder), toBePopped() {
|
||||
dataFolder(folder), degraded(degraded) {
|
||||
cx = openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::True);
|
||||
}
|
||||
};
|
||||
|
@ -398,15 +398,15 @@ struct LogData : NonCopyable, public ReferenceCounted<LogData> {
|
|||
bool nothingPersistent,
|
||||
bool poppedRecently,
|
||||
bool unpoppedRecovered)
|
||||
: tag(tag), nothingPersistent(nothingPersistent), poppedRecently(poppedRecently), popped(popped),
|
||||
persistentPopped(0), versionForPoppedLocation(0), poppedLocation(poppedLocation),
|
||||
unpoppedRecovered(unpoppedRecovered) {}
|
||||
: nothingPersistent(nothingPersistent), poppedRecently(poppedRecently), popped(popped), persistentPopped(0),
|
||||
versionForPoppedLocation(0), poppedLocation(poppedLocation), unpoppedRecovered(unpoppedRecovered),
|
||||
tag(tag) {}
|
||||
|
||||
TagData(TagData&& r) noexcept
|
||||
: versionMessages(std::move(r.versionMessages)), nothingPersistent(r.nothingPersistent),
|
||||
poppedRecently(r.poppedRecently), popped(r.popped), persistentPopped(r.persistentPopped),
|
||||
versionForPoppedLocation(r.versionForPoppedLocation), poppedLocation(r.poppedLocation), tag(r.tag),
|
||||
unpoppedRecovered(r.unpoppedRecovered) {}
|
||||
versionForPoppedLocation(r.versionForPoppedLocation), poppedLocation(r.poppedLocation),
|
||||
unpoppedRecovered(r.unpoppedRecovered), tag(r.tag) {}
|
||||
void operator=(TagData&& r) noexcept {
|
||||
versionMessages = std::move(r.versionMessages);
|
||||
nothingPersistent = r.nothingPersistent;
|
||||
|
@ -607,16 +607,15 @@ struct LogData : NonCopyable, public ReferenceCounted<LogData> {
|
|||
ProtocolVersion protocolVersion,
|
||||
std::vector<Tag> tags,
|
||||
std::string context)
|
||||
: tLogData(tLogData), knownCommittedVersion(0), logId(interf.id()), cc("TLog", interf.id().toString()),
|
||||
bytesInput("BytesInput", cc), bytesDurable("BytesDurable", cc), remoteTag(remoteTag), isPrimary(isPrimary),
|
||||
logRouterTags(logRouterTags), txsTags(txsTags), recruitmentID(recruitmentID), protocolVersion(protocolVersion),
|
||||
logSystem(new AsyncVar<Reference<ILogSystem>>()), logRouterPoppedVersion(0), durableKnownCommittedVersion(0),
|
||||
minKnownCommittedVersion(0), queuePoppedVersion(0), allTags(tags.begin(), tags.end()),
|
||||
terminated(tLogData->terminated.getFuture()), minPoppedTagVersion(0), minPoppedTag(invalidTag),
|
||||
// These are initialized differently on init() or recovery
|
||||
recoveryCount(), stopped(false), initialized(false), queueCommittingVersion(0),
|
||||
newPersistentDataVersion(invalidVersion), unrecoveredBefore(1), recoveredAt(1), unpoppedRecoveredTags(0),
|
||||
logRouterPopToVersion(0), locality(tagLocalityInvalid), execOpCommitInProgress(false) {
|
||||
: stopped(false), initialized(false), recoveryCount(), queueCommittingVersion(0), knownCommittedVersion(0),
|
||||
durableKnownCommittedVersion(0), minKnownCommittedVersion(0), queuePoppedVersion(0), minPoppedTagVersion(0),
|
||||
minPoppedTag(invalidTag), unpoppedRecoveredTags(0), cc("TLog", interf.id().toString()),
|
||||
bytesInput("BytesInput", cc), bytesDurable("BytesDurable", cc), logId(interf.id()),
|
||||
protocolVersion(protocolVersion), newPersistentDataVersion(invalidVersion), tLogData(tLogData),
|
||||
unrecoveredBefore(1), recoveredAt(1), logSystem(new AsyncVar<Reference<ILogSystem>>()), remoteTag(remoteTag),
|
||||
isPrimary(isPrimary), logRouterTags(logRouterTags), logRouterPoppedVersion(0), logRouterPopToVersion(0),
|
||||
locality(tagLocalityInvalid), recruitmentID(recruitmentID), allTags(tags.begin(), tags.end()),
|
||||
terminated(tLogData->terminated.getFuture()), execOpCommitInProgress(false), txsTags(txsTags) {
|
||||
startRole(Role::TRANSACTION_LOG,
|
||||
interf.id(),
|
||||
tLogData->workerID,
|
||||
|
|
|
@ -34,7 +34,7 @@ void OnDemandStore::open() {
|
|||
}
|
||||
|
||||
OnDemandStore::OnDemandStore(std::string const& folder, UID myID, std::string const& prefix)
|
||||
: folder(folder), prefix(prefix), store(nullptr), myID(myID) {}
|
||||
: folder(folder), myID(myID), store(nullptr), prefix(prefix) {}
|
||||
|
||||
OnDemandStore::~OnDemandStore() {
|
||||
if (store) {
|
||||
|
|
|
@ -90,16 +90,16 @@ struct ProxyStats {
|
|||
Version* pVersion,
|
||||
NotifiedVersion* pCommittedVersion,
|
||||
int64_t* commitBatchesMemBytesCountPtr)
|
||||
: cc("ProxyStats", id.toString()), maxComputeNS(0), minComputeNS(1e12), txnCommitIn("TxnCommitIn", cc),
|
||||
: cc("ProxyStats", id.toString()), txnCommitIn("TxnCommitIn", cc),
|
||||
txnCommitVersionAssigned("TxnCommitVersionAssigned", cc), txnCommitResolving("TxnCommitResolving", cc),
|
||||
txnCommitResolved("TxnCommitResolved", cc), txnCommitOut("TxnCommitOut", cc),
|
||||
txnCommitOutSuccess("TxnCommitOutSuccess", cc), txnCommitErrors("TxnCommitErrors", cc),
|
||||
txnConflicts("TxnConflicts", cc), commitBatchIn("CommitBatchIn", cc),
|
||||
txnRejectedForQueuedTooLong("TxnRejectedForQueuedTooLong", cc), commitBatchOut("CommitBatchOut", cc),
|
||||
mutationBytes("MutationBytes", cc), mutations("Mutations", cc), conflictRanges("ConflictRanges", cc),
|
||||
txnConflicts("TxnConflicts", cc), txnRejectedForQueuedTooLong("TxnRejectedForQueuedTooLong", cc),
|
||||
commitBatchIn("CommitBatchIn", cc), commitBatchOut("CommitBatchOut", cc), mutationBytes("MutationBytes", cc),
|
||||
mutations("Mutations", cc), conflictRanges("ConflictRanges", cc),
|
||||
keyServerLocationIn("KeyServerLocationIn", cc), keyServerLocationOut("KeyServerLocationOut", cc),
|
||||
keyServerLocationErrors("KeyServerLocationErrors", cc), lastCommitVersionAssigned(0),
|
||||
txnExpensiveClearCostEstCount("ExpensiveClearCostEstCount", cc),
|
||||
keyServerLocationErrors("KeyServerLocationErrors", cc),
|
||||
txnExpensiveClearCostEstCount("ExpensiveClearCostEstCount", cc), lastCommitVersionAssigned(0),
|
||||
commitLatencySample("CommitLatencyMetrics",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
|
@ -112,7 +112,8 @@ struct ProxyStats {
|
|||
commitBatchingWindowSize("CommitBatchingWindowSize",
|
||||
id,
|
||||
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE) {
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
maxComputeNS(0), minComputeNS(1e12) {
|
||||
specialCounter(cc, "LastAssignedCommitVersion", [this]() { return this->lastCommitVersionAssigned; });
|
||||
specialCounter(cc, "Version", [pVersion]() { return *pVersion; });
|
||||
specialCounter(cc, "CommittedVersion", [pCommittedVersion]() { return pCommittedVersion->get(); });
|
||||
|
@ -241,14 +242,14 @@ struct ProxyCommitData {
|
|||
RequestStream<CommitTransactionRequest> commit,
|
||||
Reference<AsyncVar<ServerDBInfo> const> db,
|
||||
bool firstProxy)
|
||||
: dbgid(dbgid), stats(dbgid, &version, &committedVersion, &commitBatchesMemBytesCount), master(master),
|
||||
logAdapter(nullptr), txnStateStore(nullptr), popRemoteTxs(false), committedVersion(recoveryTransactionVersion),
|
||||
version(0), minKnownCommittedVersion(0), lastVersionTime(0), commitVersionRequestNumber(1),
|
||||
mostRecentProcessedRequestNumber(0), getConsistentReadVersion(getConsistentReadVersion), commit(commit),
|
||||
lastCoalesceTime(0), localCommitBatchesStarted(0), locked(false),
|
||||
commitBatchInterval(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_MIN), firstProxy(firstProxy),
|
||||
: dbgid(dbgid), commitBatchesMemBytesCount(0),
|
||||
stats(dbgid, &version, &committedVersion, &commitBatchesMemBytesCount), master(master), logAdapter(nullptr),
|
||||
txnStateStore(nullptr), committedVersion(recoveryTransactionVersion), minKnownCommittedVersion(0), version(0),
|
||||
lastVersionTime(0), commitVersionRequestNumber(1), mostRecentProcessedRequestNumber(0), firstProxy(firstProxy),
|
||||
lastCoalesceTime(0), locked(false), commitBatchInterval(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_MIN),
|
||||
localCommitBatchesStarted(0), getConsistentReadVersion(getConsistentReadVersion), commit(commit),
|
||||
cx(openDBOnServer(db, TaskPriority::DefaultEndpoint, LockAware::True)), db(db),
|
||||
singleKeyMutationEvent(LiteralStringRef("SingleKeyMutation")), commitBatchesMemBytesCount(0), lastTxsPop(0),
|
||||
singleKeyMutationEvent(LiteralStringRef("SingleKeyMutation")), lastTxsPop(0), popRemoteTxs(false),
|
||||
lastStartCommit(0), lastCommitLatency(SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION), lastCommitTime(0),
|
||||
lastMasterReset(now()), lastResolverReset(now()) {
|
||||
commitComputePerOperation.resize(SERVER_KNOBS->PROXY_COMPUTE_BUCKETS, 0.0);
|
||||
|
|
|
@ -515,8 +515,7 @@ struct RatekeeperLimits {
|
|||
int64_t logSpringBytes,
|
||||
double maxVersionDifference,
|
||||
int64_t durabilityLagTargetVersions)
|
||||
: priority(priority), tpsLimit(std::numeric_limits<double>::infinity()),
|
||||
tpsLimitMetric(StringRef("Ratekeeper.TPSLimit" + context)),
|
||||
: tpsLimit(std::numeric_limits<double>::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<double>::infinity()), lastDurabilityLag(0), context(context) {}
|
||||
lastDurabilityLag(0), durabilityLagLimit(std::numeric_limits<double>::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,
|
||||
|
|
|
@ -55,7 +55,7 @@ struct StagingKey {
|
|||
LogMessageVersion version; // largest version of set or clear for the key
|
||||
std::map<LogMessageVersion, Standalone<MutationRef>> pendingMutations; // mutations not set or clear type
|
||||
|
||||
explicit StagingKey(Key key) : key(key), version(0), type(MutationRef::MAX_ATOMIC_OP) {}
|
||||
explicit StagingKey(Key key) : key(key), type(MutationRef::MAX_ATOMIC_OP), version(0) {}
|
||||
|
||||
// Add mutation m at newVersion to stagingKey
|
||||
// Assume: SetVersionstampedKey and SetVersionstampedValue have been converted to set
|
||||
|
@ -269,8 +269,8 @@ struct ApplierBatchData : public ReferenceCounted<ApplierBatchData> {
|
|||
|
||||
Counters(ApplierBatchData* self, UID applierInterfID, int batchIndex)
|
||||
: cc("ApplierBatch", applierInterfID.toString() + ":" + std::to_string(batchIndex)),
|
||||
receivedBytes("ReceivedBytes", cc), receivedMutations("ReceivedMutations", cc),
|
||||
receivedAtomicOps("ReceivedAtomicOps", cc), receivedWeightedBytes("ReceivedWeightedMutations", cc),
|
||||
receivedBytes("ReceivedBytes", cc), receivedWeightedBytes("ReceivedWeightedMutations", cc),
|
||||
receivedMutations("ReceivedMutations", cc), receivedAtomicOps("ReceivedAtomicOps", cc),
|
||||
appliedBytes("AppliedBytes", cc), appliedWeightedBytes("AppliedWeightedBytes", cc),
|
||||
appliedMutations("AppliedMutations", cc), appliedAtomicOps("AppliedAtomicOps", cc),
|
||||
appliedTxns("AppliedTxns", cc), appliedTxnRetries("AppliedTxnRetries", cc), fetchKeys("FetchKeys", cc),
|
||||
|
@ -282,10 +282,9 @@ struct ApplierBatchData : public ReferenceCounted<ApplierBatchData> {
|
|||
void delref() { return ReferenceCounted<ApplierBatchData>::delref(); }
|
||||
|
||||
explicit ApplierBatchData(UID nodeID, int batchIndex)
|
||||
: counters(this, nodeID, batchIndex),
|
||||
: vbState(ApplierVersionBatchState::NOT_INIT), receiveMutationReqs(0), receivedBytes(0), appliedBytes(0),
|
||||
targetWriteRateMB(SERVER_KNOBS->FASTRESTORE_WRITE_BW_MB / SERVER_KNOBS->FASTRESTORE_NUM_APPLIERS),
|
||||
totalBytesToWrite(-1), applyingDataBytes(0), vbState(ApplierVersionBatchState::NOT_INIT),
|
||||
receiveMutationReqs(0), receivedBytes(0), appliedBytes(0) {
|
||||
totalBytesToWrite(-1), applyingDataBytes(0), counters(this, nodeID, batchIndex) {
|
||||
pollMetrics = traceCounters(format("FastRestoreApplierMetrics%d", batchIndex),
|
||||
nodeID,
|
||||
SERVER_KNOBS->FASTRESTORE_ROLE_LOGGING_DELAY,
|
||||
|
|
|
@ -92,7 +92,7 @@ struct LoaderBatchData : public ReferenceCounted<LoaderBatchData> {
|
|||
} counters;
|
||||
|
||||
explicit LoaderBatchData(UID nodeID, int batchIndex)
|
||||
: counters(this, nodeID, batchIndex), vbState(LoaderVersionBatchState::NOT_INIT), loadFileReqs(0) {
|
||||
: vbState(LoaderVersionBatchState::NOT_INIT), loadFileReqs(0), counters(this, nodeID, batchIndex) {
|
||||
pollMetrics = traceCounters(format("FastRestoreLoaderMetrics%d", batchIndex),
|
||||
nodeID,
|
||||
SERVER_KNOBS->FASTRESTORE_ROLE_LOGGING_DELAY,
|
||||
|
|
|
@ -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;
|
||||
|
@ -786,7 +786,7 @@ private:
|
|||
};
|
||||
|
||||
struct ConflictSet {
|
||||
ConflictSet() : oldestVersion(0), removalKey(makeString(0)) {}
|
||||
ConflictSet() : removalKey(makeString(0)), oldestVersion(0) {}
|
||||
~ConflictSet() {}
|
||||
|
||||
SkipList versionHistory;
|
||||
|
|
|
@ -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<Void> 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);
|
||||
|
|
|
@ -52,13 +52,13 @@ struct TLogInterface {
|
|||
|
||||
TLogInterface() {}
|
||||
explicit TLogInterface(const LocalityData& locality)
|
||||
: uniqueID(deterministicRandom()->randomUniqueID()), filteredLocality(locality) {
|
||||
: filteredLocality(locality), uniqueID(deterministicRandom()->randomUniqueID()) {
|
||||
sharedTLogID = uniqueID;
|
||||
}
|
||||
TLogInterface(UID sharedTLogID, const LocalityData& locality)
|
||||
: uniqueID(deterministicRandom()->randomUniqueID()), sharedTLogID(sharedTLogID), filteredLocality(locality) {}
|
||||
: filteredLocality(locality), uniqueID(deterministicRandom()->randomUniqueID()), sharedTLogID(sharedTLogID) {}
|
||||
TLogInterface(UID uniqueID, UID sharedTLogID, const LocalityData& locality)
|
||||
: uniqueID(uniqueID), sharedTLogID(sharedTLogID), filteredLocality(locality) {}
|
||||
: filteredLocality(locality), uniqueID(uniqueID), sharedTLogID(sharedTLogID) {}
|
||||
UID id() const { return uniqueID; }
|
||||
UID getSharedTLogID() const { return sharedTLogID; }
|
||||
std::string toString() const { return id().shortString(); }
|
||||
|
@ -152,7 +152,7 @@ struct VerUpdateRef {
|
|||
VectorRef<MutationRef> mutations;
|
||||
bool isPrivateData;
|
||||
|
||||
VerUpdateRef() : isPrivateData(false), version(invalidVersion) {}
|
||||
VerUpdateRef() : version(invalidVersion), isPrivateData(false) {}
|
||||
VerUpdateRef(Arena& to, const VerUpdateRef& from)
|
||||
: version(from.version), mutations(to, from.mutations), isPrivateData(from.isPrivateData) {}
|
||||
int expectedSize() const { return mutations.expectedSize(); }
|
||||
|
@ -200,7 +200,7 @@ struct TLogPeekRequest {
|
|||
bool returnIfBlocked,
|
||||
bool onlySpilled,
|
||||
Optional<std::pair<UID, int>> sequence = Optional<std::pair<UID, int>>())
|
||||
: begin(begin), tag(tag), returnIfBlocked(returnIfBlocked), sequence(sequence), onlySpilled(onlySpilled) {}
|
||||
: begin(begin), tag(tag), returnIfBlocked(returnIfBlocked), onlySpilled(onlySpilled), sequence(sequence) {}
|
||||
TLogPeekRequest() {}
|
||||
|
||||
template <class Ar>
|
||||
|
|
|
@ -60,7 +60,7 @@ struct TLogQueueEntryRef {
|
|||
|
||||
TLogQueueEntryRef() : version(0), knownCommittedVersion(0) {}
|
||||
TLogQueueEntryRef(Arena& a, TLogQueueEntryRef const& from)
|
||||
: version(from.version), knownCommittedVersion(from.knownCommittedVersion), id(from.id),
|
||||
: id(from.id), version(from.version), knownCommittedVersion(from.knownCommittedVersion),
|
||||
messages(a, from.messages) {}
|
||||
|
||||
// To change this serialization, ProtocolVersion::TLogQueueEntryRef must be updated, and downgrades need to be
|
||||
|
@ -375,14 +375,14 @@ struct TLogData : NonCopyable {
|
|||
Reference<AsyncVar<ServerDBInfo> const> dbInfo,
|
||||
Reference<AsyncVar<bool>> degraded,
|
||||
std::string folder)
|
||||
: dbgid(dbgid), workerID(workerID), instanceID(deterministicRandom()->randomUniqueID().first()),
|
||||
persistentData(persistentData), rawPersistentQueue(persistentQueue),
|
||||
persistentQueue(new TLogQueue(persistentQueue, dbgid)), dbInfo(dbInfo), degraded(degraded), queueCommitBegin(0),
|
||||
queueCommitEnd(0), diskQueueCommitBytes(0), largeDiskQueueCommitBytes(false), bytesInput(0), bytesDurable(0),
|
||||
: dbgid(dbgid), workerID(workerID), persistentData(persistentData), rawPersistentQueue(persistentQueue),
|
||||
persistentQueue(new TLogQueue(persistentQueue, dbgid)), diskQueueCommitBytes(0),
|
||||
largeDiskQueueCommitBytes(false), dbInfo(dbInfo), queueCommitEnd(0), queueCommitBegin(0),
|
||||
instanceID(deterministicRandom()->randomUniqueID().first()), bytesInput(0), bytesDurable(0),
|
||||
targetVolatileBytes(SERVER_KNOBS->TLOG_SPILL_THRESHOLD), overheadBytesInput(0), overheadBytesDurable(0),
|
||||
peekMemoryLimiter(SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_PEEK_MEMORY_BYTES),
|
||||
concurrentLogRouterReads(SERVER_KNOBS->CONCURRENT_LOG_ROUTER_READS), ignorePopRequest(false),
|
||||
ignorePopDeadline(), ignorePopUid(), dataFolder(folder), toBePopped(),
|
||||
dataFolder(folder), degraded(degraded),
|
||||
commitLatencyDist(Histogram::getHistogram(LiteralStringRef("tLog"),
|
||||
LiteralStringRef("commit"),
|
||||
Histogram::Unit::microseconds)) {
|
||||
|
@ -409,15 +409,15 @@ struct LogData : NonCopyable, public ReferenceCounted<LogData> {
|
|||
bool nothingPersistent,
|
||||
bool poppedRecently,
|
||||
bool unpoppedRecovered)
|
||||
: tag(tag), nothingPersistent(nothingPersistent), poppedRecently(poppedRecently), popped(popped),
|
||||
persistentPopped(0), versionForPoppedLocation(0), poppedLocation(poppedLocation),
|
||||
unpoppedRecovered(unpoppedRecovered) {}
|
||||
: nothingPersistent(nothingPersistent), poppedRecently(poppedRecently), popped(popped), persistentPopped(0),
|
||||
versionForPoppedLocation(0), poppedLocation(poppedLocation), unpoppedRecovered(unpoppedRecovered),
|
||||
tag(tag) {}
|
||||
|
||||
TagData(TagData&& r) noexcept
|
||||
: versionMessages(std::move(r.versionMessages)), nothingPersistent(r.nothingPersistent),
|
||||
poppedRecently(r.poppedRecently), popped(r.popped), persistentPopped(r.persistentPopped),
|
||||
versionForPoppedLocation(r.versionForPoppedLocation), poppedLocation(r.poppedLocation), tag(r.tag),
|
||||
unpoppedRecovered(r.unpoppedRecovered) {}
|
||||
versionForPoppedLocation(r.versionForPoppedLocation), poppedLocation(r.poppedLocation),
|
||||
unpoppedRecovered(r.unpoppedRecovered), tag(r.tag) {}
|
||||
void operator=(TagData&& r) noexcept {
|
||||
versionMessages = std::move(r.versionMessages);
|
||||
nothingPersistent = r.nothingPersistent;
|
||||
|
@ -626,17 +626,16 @@ struct LogData : NonCopyable, public ReferenceCounted<LogData> {
|
|||
TLogSpillType logSpillType,
|
||||
std::vector<Tag> tags,
|
||||
std::string context)
|
||||
: tLogData(tLogData), knownCommittedVersion(0), logId(interf.id()), cc("TLog", interf.id().toString()),
|
||||
bytesInput("BytesInput", cc), bytesDurable("BytesDurable", cc), remoteTag(remoteTag), isPrimary(isPrimary),
|
||||
logRouterTags(logRouterTags), txsTags(txsTags), recruitmentID(recruitmentID), protocolVersion(protocolVersion),
|
||||
logSpillType(logSpillType), logSystem(new AsyncVar<Reference<ILogSystem>>()), logRouterPoppedVersion(0),
|
||||
durableKnownCommittedVersion(0), minKnownCommittedVersion(0), queuePoppedVersion(0),
|
||||
allTags(tags.begin(), tags.end()), terminated(tLogData->terminated.getFuture()), minPoppedTagVersion(0),
|
||||
minPoppedTag(invalidTag),
|
||||
// These are initialized differently on init() or recovery
|
||||
recoveryCount(), stopped(false), initialized(false), queueCommittingVersion(0),
|
||||
newPersistentDataVersion(invalidVersion), unrecoveredBefore(1), recoveredAt(1), unpoppedRecoveredTags(0),
|
||||
logRouterPopToVersion(0), locality(tagLocalityInvalid), execOpCommitInProgress(false) {
|
||||
: stopped(false), initialized(false), queueCommittingVersion(0), knownCommittedVersion(0),
|
||||
durableKnownCommittedVersion(0), minKnownCommittedVersion(0), queuePoppedVersion(0), minPoppedTagVersion(0),
|
||||
minPoppedTag(invalidTag), unpoppedRecoveredTags(0), cc("TLog", interf.id().toString()),
|
||||
bytesInput("BytesInput", cc), bytesDurable("BytesDurable", cc), logId(interf.id()),
|
||||
protocolVersion(protocolVersion), newPersistentDataVersion(invalidVersion), tLogData(tLogData),
|
||||
unrecoveredBefore(1), recoveredAt(1), logSystem(new AsyncVar<Reference<ILogSystem>>()), remoteTag(remoteTag),
|
||||
isPrimary(isPrimary), logRouterTags(logRouterTags), logRouterPoppedVersion(0), logRouterPopToVersion(0),
|
||||
locality(tagLocalityInvalid), recruitmentID(recruitmentID), logSpillType(logSpillType),
|
||||
allTags(tags.begin(), tags.end()), terminated(tLogData->terminated.getFuture()), execOpCommitInProgress(false),
|
||||
txsTags(txsTags) {
|
||||
startRole(Role::TRANSACTION_LOG,
|
||||
interf.id(),
|
||||
tLogData->workerID,
|
||||
|
|
|
@ -52,7 +52,7 @@ struct OldLogData {
|
|||
std::set<int8_t> pseudoLocalities;
|
||||
LogEpoch epoch;
|
||||
|
||||
OldLogData() : epochBegin(0), epochEnd(0), logRouterTags(0), txsTags(0), epoch(0) {}
|
||||
OldLogData() : logRouterTags(0), txsTags(0), epochBegin(0), epochEnd(0), epoch(0) {}
|
||||
|
||||
// Constructor for T of OldTLogConf and OldTLogCoreData
|
||||
template <class T>
|
||||
|
@ -124,8 +124,8 @@ TLogSet::TLogSet(const LogSet& rhs)
|
|||
}
|
||||
|
||||
OldTLogConf::OldTLogConf(const OldLogData& oldLogData)
|
||||
: logRouterTags(oldLogData.logRouterTags), txsTags(oldLogData.txsTags), epochBegin(oldLogData.epochBegin),
|
||||
epochEnd(oldLogData.epochEnd), pseudoLocalities(oldLogData.pseudoLocalities), epoch(oldLogData.epoch) {
|
||||
: epochBegin(oldLogData.epochBegin), epochEnd(oldLogData.epochEnd), logRouterTags(oldLogData.logRouterTags),
|
||||
txsTags(oldLogData.txsTags), pseudoLocalities(oldLogData.pseudoLocalities), epoch(oldLogData.epoch) {
|
||||
for (const Reference<LogSet>& logSet : oldLogData.tLogs) {
|
||||
tLogs.emplace_back(*logSet);
|
||||
}
|
||||
|
@ -202,9 +202,9 @@ struct TagPartitionedLogSystem final : ILogSystem, ReferenceCounted<TagPartition
|
|||
LogEpoch e,
|
||||
Optional<PromiseStream<Future<Void>>> addActor = Optional<PromiseStream<Future<Void>>>())
|
||||
: dbgid(dbgid), logSystemType(LogSystemType::empty), expectedLogSets(0), logRouterTags(0), txsTags(0),
|
||||
repopulateRegionAntiQuorum(0), epoch(e), oldestBackupEpoch(0), recoveryCompleteWrittenToCoreState(false),
|
||||
locality(locality), remoteLogsWrittenToCoreState(false), hasRemoteServers(false), stopped(false),
|
||||
addActor(addActor), popActors(false) {}
|
||||
repopulateRegionAntiQuorum(0), stopped(false), epoch(e), oldestBackupEpoch(0),
|
||||
recoveryCompleteWrittenToCoreState(false), remoteLogsWrittenToCoreState(false), hasRemoteServers(false),
|
||||
locality(locality), addActor(addActor), popActors(false) {}
|
||||
|
||||
void stopRejoins() final { rejoins = Future<Void>(); }
|
||||
|
||||
|
|
|
@ -61,7 +61,7 @@
|
|||
const uint32_t RESERVED_COUNT = 1U << 29;
|
||||
|
||||
VFSAsyncFile::VFSAsyncFile(std::string const& filename, int flags)
|
||||
: filename(filename), flags(flags), pLockCount(&filename_lockCount_openCount[filename].first), debug_zcrefs(0),
|
||||
: flags(flags), filename(filename), pLockCount(&filename_lockCount_openCount[filename].first), debug_zcrefs(0),
|
||||
debug_zcreads(0), debug_reads(0), chunkSize(0) {
|
||||
filename_lockCount_openCount[filename].second++;
|
||||
|
||||
|
|
|
@ -2050,10 +2050,10 @@ public:
|
|||
int concurrentExtentReads,
|
||||
bool memoryOnly = false,
|
||||
Promise<Void> errorPromise = {})
|
||||
: desiredPageSize(desiredPageSize), desiredExtentSize(desiredExtentSize), filename(filename), pHeader(nullptr),
|
||||
pageCacheBytes(pageCacheSizeBytes), memoryOnly(memoryOnly), remapCleanupWindow(remapCleanupWindow),
|
||||
concurrentExtentReads(new FlowLock(concurrentExtentReads)), errorPromise(errorPromise),
|
||||
ioLock(FLOW_KNOBS->MAX_OUTSTANDING, ioMaxPriority) {
|
||||
: ioLock(FLOW_KNOBS->MAX_OUTSTANDING, ioMaxPriority), pageCacheBytes(pageCacheSizeBytes), pHeader(nullptr),
|
||||
desiredPageSize(desiredPageSize), desiredExtentSize(desiredExtentSize), filename(filename),
|
||||
memoryOnly(memoryOnly), errorPromise(errorPromise), remapCleanupWindow(remapCleanupWindow),
|
||||
concurrentExtentReads(new FlowLock(concurrentExtentReads)) {
|
||||
|
||||
if (!g_redwoodMetricsActor.isValid()) {
|
||||
g_redwoodMetricsActor = redwoodMetricsLogger();
|
||||
|
@ -3550,7 +3550,7 @@ private:
|
|||
class DWALPagerSnapshot : public IPagerSnapshot, public ReferenceCounted<DWALPagerSnapshot> {
|
||||
public:
|
||||
DWALPagerSnapshot(DWALPager* pager, Key meta, Version version, Future<Void> expiredFuture)
|
||||
: pager(pager), metaKey(meta), version(version), expired(expiredFuture) {}
|
||||
: pager(pager), expired(expiredFuture), version(version), metaKey(meta) {}
|
||||
~DWALPagerSnapshot() override {}
|
||||
|
||||
Future<Reference<const ArenaPage>> getPhysicalPage(PagerEventReasons reason,
|
||||
|
@ -4477,7 +4477,7 @@ public:
|
|||
Version getLastCommittedVersion() const { return m_lastCommittedVersion; }
|
||||
|
||||
VersionedBTree(IPager2* pager, std::string name)
|
||||
: m_pager(pager), m_writeVersion(invalidVersion), m_lastCommittedVersion(invalidVersion), m_pBuffer(nullptr),
|
||||
: m_pager(pager), m_pBuffer(nullptr), m_writeVersion(invalidVersion), m_lastCommittedVersion(invalidVersion),
|
||||
m_name(name), m_pHeader(nullptr), m_headerSpace(0) {
|
||||
|
||||
m_lazyClearActor = 0;
|
||||
|
@ -5642,7 +5642,7 @@ private:
|
|||
struct InternalPageModifier {
|
||||
InternalPageModifier() {}
|
||||
InternalPageModifier(Reference<const ArenaPage> p, bool alreadyCloned, bool updating, ParentInfo* parentInfo)
|
||||
: page(p), clonedPage(alreadyCloned), updating(updating), changesMade(false), parentInfo(parentInfo) {}
|
||||
: updating(updating), page(p), clonedPage(alreadyCloned), changesMade(false), parentInfo(parentInfo) {}
|
||||
|
||||
// Whether updating the existing page is allowed
|
||||
bool updating;
|
||||
|
@ -8746,7 +8746,7 @@ TEST_CASE("/redwood/correctness/unit/deltaTree/IntIntPair") {
|
|||
}
|
||||
|
||||
struct SimpleCounter {
|
||||
SimpleCounter() : x(0), xt(0), t(timer()), start(t) {}
|
||||
SimpleCounter() : x(0), t(timer()), start(t), xt(0) {}
|
||||
void operator+=(int n) { x += n; }
|
||||
void operator++() { x++; }
|
||||
int64_t get() { return x; }
|
||||
|
|
|
@ -746,7 +746,7 @@ struct EventLogRequest {
|
|||
ReplyPromise<TraceEventFields> reply;
|
||||
|
||||
EventLogRequest() : getLastError(true) {}
|
||||
explicit EventLogRequest(Standalone<StringRef> eventName) : eventName(eventName), getLastError(false) {}
|
||||
explicit EventLogRequest(Standalone<StringRef> eventName) : getLastError(false), eventName(eventName) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
|
@ -762,8 +762,8 @@ struct DebugEntryRef {
|
|||
MutationRef mutation;
|
||||
DebugEntryRef() {}
|
||||
DebugEntryRef(const char* c, Version v, MutationRef const& m)
|
||||
: context((const uint8_t*)c, strlen(c)), version(v), mutation(m), time(now()),
|
||||
address(g_network->getLocalAddress()) {}
|
||||
: time(now()), address(g_network->getLocalAddress()), context((const uint8_t*)c, strlen(c)), version(v),
|
||||
mutation(m) {}
|
||||
DebugEntryRef(Arena& a, DebugEntryRef const& d)
|
||||
: time(d.time), address(d.address), context(d.context), version(d.version), mutation(a, d.mutation) {}
|
||||
|
||||
|
|
|
@ -95,8 +95,8 @@ public:
|
|||
ReusableCoordinatedState(ServerCoordinators const& coordinators,
|
||||
PromiseStream<Future<Void>> 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<Void> read() { return _read(this); }
|
||||
|
||||
|
@ -262,14 +262,16 @@ struct MasterData : NonCopyable, ReferenceCounted<MasterData> {
|
|||
Standalone<StringRef> const& dbId,
|
||||
PromiseStream<Future<Void>> const& addActor,
|
||||
bool forceRecovery)
|
||||
: dbgid(myInterface.id()), myInterface(myInterface), dbInfo(dbInfo), cstate(coordinators, addActor, dbgid),
|
||||
coordinators(coordinators), clusterController(clusterController), dbId(dbId), forceRecovery(forceRecovery),
|
||||
safeLocality(tagLocalityInvalid), primaryLocality(tagLocalityInvalid), neverCreated(false),
|
||||
lastEpochEnd(invalidVersion), liveCommittedVersion(invalidVersion), databaseLocked(false),
|
||||
minKnownCommittedVersion(invalidVersion), recoveryTransactionVersion(invalidVersion), lastCommitTime(0),
|
||||
registrationCount(0), version(invalidVersion), lastVersionTime(0), txnStateStore(nullptr), memoryLimit(2e9),
|
||||
addActor(addActor), hasConfiguration(false), recruitmentStalled(makeReference<AsyncVar<bool>>(false)),
|
||||
cc("Master", dbgid.toString()), changeCoordinatorsRequests("ChangeCoordinatorsRequests", cc),
|
||||
|
||||
: dbgid(myInterface.id()), lastEpochEnd(invalidVersion), recoveryTransactionVersion(invalidVersion),
|
||||
lastCommitTime(0), liveCommittedVersion(invalidVersion), databaseLocked(false),
|
||||
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<AsyncVar<bool>>(false)), forceRecovery(forceRecovery), neverCreated(false),
|
||||
safeLocality(tagLocalityInvalid), primaryLocality(tagLocalityInvalid), cc("Master", dbgid.toString()),
|
||||
changeCoordinatorsRequests("ChangeCoordinatorsRequests", cc),
|
||||
getCommitVersionRequests("GetCommitVersionRequests", cc),
|
||||
backupWorkerDoneRequests("BackupWorkerDoneRequests", cc),
|
||||
getLiveCommittedVersionRequests("GetLiveCommittedVersionRequests", cc),
|
||||
|
|
|
@ -764,9 +764,9 @@ public:
|
|||
LatencyBands readLatencyBands;
|
||||
|
||||
Counters(StorageServer* self)
|
||||
: cc("StorageServer", self->thisServerID.toString()), getKeyQueries("GetKeyQueries", cc),
|
||||
getValueQueries("GetValueQueries", cc), getRangeQueries("GetRangeQueries", cc),
|
||||
getRangeStreamQueries("GetRangeStreamQueries", cc), allQueries("QueryQueue", cc),
|
||||
: cc("StorageServer", self->thisServerID.toString()), allQueries("QueryQueue", cc),
|
||||
getKeyQueries("GetKeyQueries", cc), getValueQueries("GetValueQueries", cc),
|
||||
getRangeQueries("GetRangeQueries", cc), getRangeStreamQueries("GetRangeStreamQueries", cc),
|
||||
finishedQueries("FinishedQueries", cc), lowPriorityQueries("LowPriorityQueries", cc),
|
||||
rowsQueried("RowsQueried", cc), bytesQueried("BytesQueried", cc), watchQueries("WatchQueries", cc),
|
||||
emptyQueries("EmptyQueries", cc), bytesInput("BytesInput", cc), bytesDurable("BytesDurable", cc),
|
||||
|
@ -808,18 +808,7 @@ public:
|
|||
StorageServer(IKeyValueStore* storage,
|
||||
Reference<AsyncVar<ServerDBInfo> const> const& db,
|
||||
StorageServerInterface const& ssi)
|
||||
: fetchKeysHistograms(), instanceID(deterministicRandom()->randomUniqueID().first()), storage(this, storage),
|
||||
db(db), actors(false), lastTLogVersion(0), lastVersionWithData(0), restoredVersion(0),
|
||||
rebootAfterDurableVersion(std::numeric_limits<Version>::max()), durableInProgress(Void()), versionLag(0),
|
||||
primaryLocality(tagLocalityInvalid), updateEagerReads(0), shardChangeCounter(0),
|
||||
fetchKeysParallelismLock(SERVER_KNOBS->FETCH_KEYS_PARALLELISM),
|
||||
fetchKeysBytesBudget(SERVER_KNOBS->STORAGE_FETCH_BYTES), fetchKeysBudgetUsed(false), shuttingDown(false),
|
||||
debug_inApplyUpdate(false), debug_lastValidateTime(0), watchBytes(0), numWatches(0), logProtocol(0),
|
||||
counters(this), tag(invalidTag), maxQueryQueue(0), thisServerID(ssi.id()), tssInQuarantine(false),
|
||||
readQueueSizeMetric(LiteralStringRef("StorageServer.ReadQueueSize")), behind(false), versionBehind(false),
|
||||
byteSampleClears(false, LiteralStringRef("\xff\xff\xff")), noRecentUpdates(false), lastUpdate(now()),
|
||||
poppedAllAfter(std::numeric_limits<Version>::max()), cpuUsage(0.0), diskUsage(0.0),
|
||||
tlogCursorReadsLatencyHistogram(Histogram::getHistogram(STORAGESERVER_HISTOGRAM_GROUP,
|
||||
: tlogCursorReadsLatencyHistogram(Histogram::getHistogram(STORAGESERVER_HISTOGRAM_GROUP,
|
||||
TLOG_CURSOR_READS_LATENCY_HISTOGRAM,
|
||||
Histogram::Unit::microseconds)),
|
||||
ssVersionLockLatencyHistogram(Histogram::getHistogram(STORAGESERVER_HISTOGRAM_GROUP,
|
||||
|
@ -842,7 +831,18 @@ public:
|
|||
Histogram::Unit::microseconds)),
|
||||
ssDurableVersionUpdateLatencyHistogram(Histogram::getHistogram(STORAGESERVER_HISTOGRAM_GROUP,
|
||||
SS_DURABLE_VERSION_UPDATE_LATENCY_HISTOGRAM,
|
||||
Histogram::Unit::microseconds)) {
|
||||
Histogram::Unit::microseconds)),
|
||||
tag(invalidTag), poppedAllAfter(std::numeric_limits<Version>::max()), cpuUsage(0.0), diskUsage(0.0),
|
||||
storage(this, storage), shardChangeCounter(0), lastTLogVersion(0), lastVersionWithData(0), restoredVersion(0),
|
||||
rebootAfterDurableVersion(std::numeric_limits<Version>::max()), primaryLocality(tagLocalityInvalid),
|
||||
versionLag(0), logProtocol(0), thisServerID(ssi.id()), tssInQuarantine(false), db(db), actors(false),
|
||||
byteSampleClears(false, LiteralStringRef("\xff\xff\xff")), durableInProgress(Void()), watchBytes(0),
|
||||
numWatches(0), noRecentUpdates(false), lastUpdate(now()),
|
||||
readQueueSizeMetric(LiteralStringRef("StorageServer.ReadQueueSize")), updateEagerReads(nullptr),
|
||||
fetchKeysParallelismLock(SERVER_KNOBS->FETCH_KEYS_PARALLELISM),
|
||||
fetchKeysBytesBudget(SERVER_KNOBS->STORAGE_FETCH_BYTES), fetchKeysBudgetUsed(false),
|
||||
instanceID(deterministicRandom()->randomUniqueID().first()), shuttingDown(false), behind(false),
|
||||
versionBehind(false), debug_inApplyUpdate(false), debug_lastValidateTime(0), maxQueryQueue(0), counters(this) {
|
||||
version.initMetric(LiteralStringRef("StorageServer.Version"), counters.cc.id);
|
||||
oldestVersion.initMetric(LiteralStringRef("StorageServer.OldestVersion"), counters.cc.id);
|
||||
durableVersion.initMetric(LiteralStringRef("StorageServer.DurableVersion"), counters.cc.id);
|
||||
|
@ -3154,7 +3154,7 @@ ACTOR Future<Void> fetchKeys(StorageServer* data, AddingShard* shard) {
|
|||
};
|
||||
|
||||
AddingShard::AddingShard(StorageServer* server, KeyRangeRef const& keys)
|
||||
: server(server), keys(keys), transferredVersion(invalidVersion), phase(WaitPrevious) {
|
||||
: keys(keys), server(server), transferredVersion(invalidVersion), phase(WaitPrevious) {
|
||||
fetchClient = fetchKeys(server, this);
|
||||
}
|
||||
|
||||
|
@ -3402,10 +3402,10 @@ static const KeyRef persistPrimaryLocality = LiteralStringRef(PERSIST_PREFIX "Pr
|
|||
class StorageUpdater {
|
||||
public:
|
||||
StorageUpdater()
|
||||
: fromVersion(invalidVersion), currentVersion(invalidVersion), restoredVersion(invalidVersion),
|
||||
: currentVersion(invalidVersion), fromVersion(invalidVersion), restoredVersion(invalidVersion),
|
||||
processedStartKey(false), processedCacheStartKey(false) {}
|
||||
StorageUpdater(Version fromVersion, Version restoredVersion)
|
||||
: fromVersion(fromVersion), currentVersion(fromVersion), restoredVersion(restoredVersion),
|
||||
: currentVersion(fromVersion), fromVersion(fromVersion), restoredVersion(restoredVersion),
|
||||
processedStartKey(false), processedCacheStartKey(false) {}
|
||||
|
||||
void applyMutation(StorageServer* data, MutationRef const& m, Version ver) {
|
||||
|
|
|
@ -48,8 +48,8 @@ using namespace std;
|
|||
WorkloadContext::WorkloadContext() {}
|
||||
|
||||
WorkloadContext::WorkloadContext(const WorkloadContext& r)
|
||||
: options(r.options), clientId(r.clientId), clientCount(r.clientCount), dbInfo(r.dbInfo),
|
||||
sharedRandomNumber(r.sharedRandomNumber) {}
|
||||
: options(r.options), clientId(r.clientId), clientCount(r.clientCount), sharedRandomNumber(r.sharedRandomNumber),
|
||||
dbInfo(r.dbInfo) {}
|
||||
|
||||
WorkloadContext::~WorkloadContext() {}
|
||||
|
||||
|
|
|
@ -223,7 +223,7 @@ struct ApiWorkload : TestWorkload {
|
|||
Database extraDB;
|
||||
|
||||
ApiWorkload(WorkloadContext const& wcx, int maxClients = -1)
|
||||
: TestWorkload(wcx), success(true), transactionFactory(nullptr), maxClients(maxClients) {
|
||||
: TestWorkload(wcx), maxClients(maxClients), success(true), transactionFactory(nullptr) {
|
||||
clientPrefixInt = getOption(options, LiteralStringRef("clientId"), clientId);
|
||||
clientPrefix = format("%010d", clientPrefixInt);
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -40,8 +40,8 @@ struct CycleWorkload : TestWorkload {
|
|||
PerfDoubleCounter totalLatency;
|
||||
|
||||
CycleWorkload(WorkloadContext const& wcx)
|
||||
: TestWorkload(wcx), transactions("Transactions"), retries("Retries"), totalLatency("Latency"),
|
||||
tooOldRetries("Retries.too_old"), commitFailedRetries("Retries.commit_failed") {
|
||||
: TestWorkload(wcx), transactions("Transactions"), retries("Retries"), tooOldRetries("Retries.too_old"),
|
||||
commitFailedRetries("Retries.commit_failed"), totalLatency("Latency") {
|
||||
testDuration = getOption(options, "testDuration"_sr, 10.0);
|
||||
transactionsPerSecond = getOption(options, "transactionsPerSecond"_sr, 5000.0) / clientCount;
|
||||
actorCount = getOption(options, "actorsPerClient"_sr, transactionsPerSecond / 5);
|
||||
|
|
|
@ -35,7 +35,7 @@ struct DDBalanceWorkload : TestWorkload {
|
|||
ContinuousSample<double> latencies;
|
||||
|
||||
DDBalanceWorkload(WorkloadContext const& wcx)
|
||||
: TestWorkload(wcx), latencies(2000), bin_shifts("Bin_Shifts"), operations("Operations"), retries("Retries") {
|
||||
: TestWorkload(wcx), bin_shifts("Bin_Shifts"), operations("Operations"), retries("Retries"), latencies(2000) {
|
||||
testDuration = getOption(options, LiteralStringRef("testDuration"), 10.0);
|
||||
binCount = getOption(options, LiteralStringRef("binCount"), 1000);
|
||||
writesPerTransaction = getOption(options, LiteralStringRef("writesPerTransaction"), 1);
|
||||
|
|
|
@ -43,7 +43,7 @@ struct FileSystemWorkload : TestWorkload {
|
|||
};
|
||||
|
||||
FileSystemWorkload(WorkloadContext const& wcx)
|
||||
: TestWorkload(wcx), latencies(2500), writeLatencies(1000), queries("Queries"), writes("Latency") {
|
||||
: TestWorkload(wcx), queries("Queries"), writes("Latency"), latencies(2500), writeLatencies(1000) {
|
||||
testDuration = getOption(options, LiteralStringRef("testDuration"), 10.0);
|
||||
transactionsPerSecond = getOption(options, LiteralStringRef("transactionsPerSecond"), 5000.0) / clientCount;
|
||||
double allowedLatency = getOption(options, LiteralStringRef("allowedLatency"), 0.250);
|
||||
|
|
|
@ -33,8 +33,8 @@ struct Increment : TestWorkload {
|
|||
PerfDoubleCounter totalLatency;
|
||||
|
||||
Increment(WorkloadContext const& wcx)
|
||||
: TestWorkload(wcx), transactions("Transactions"), retries("Retries"), totalLatency("Latency"),
|
||||
tooOldRetries("Retries.too_old"), commitFailedRetries("Retries.commit_failed") {
|
||||
: TestWorkload(wcx), transactions("Transactions"), retries("Retries"), tooOldRetries("Retries.too_old"),
|
||||
commitFailedRetries("Retries.commit_failed"), totalLatency("Latency") {
|
||||
testDuration = getOption(options, LiteralStringRef("testDuration"), 10.0);
|
||||
transactionsPerSecond = getOption(options, LiteralStringRef("transactionsPerSecond"), 5000.0);
|
||||
actorCount = getOption(options, LiteralStringRef("actorsPerClient"), transactionsPerSecond / 5);
|
||||
|
|
|
@ -33,7 +33,7 @@ struct IndexScanWorkload : KVWorkload {
|
|||
bool singleProcess, readYourWrites;
|
||||
|
||||
IndexScanWorkload(WorkloadContext const& wcx)
|
||||
: KVWorkload(wcx), failedTransactions(0), rowsRead(0), chunks(0), scans(0) {
|
||||
: KVWorkload(wcx), rowsRead(0), chunks(0), failedTransactions(0), scans(0) {
|
||||
testDuration = getOption(options, LiteralStringRef("testDuration"), 10.0);
|
||||
bytesPerRead = getOption(options, LiteralStringRef("bytesPerRead"), 80000);
|
||||
transactionDuration = getOption(options, LiteralStringRef("transactionDuration"), 1.0);
|
||||
|
|
|
@ -110,8 +110,8 @@ struct KVTest {
|
|||
bool dispose;
|
||||
|
||||
explicit KVTest(int nodeCount, bool dispose, int keyBytes)
|
||||
: store(nullptr), dispose(dispose), startVersion(Version(time(nullptr)) << 30), lastSet(startVersion),
|
||||
lastCommit(startVersion), lastDurable(startVersion), nodeCount(nodeCount), keyBytes(keyBytes) {}
|
||||
: store(nullptr), startVersion(Version(time(nullptr)) << 30), lastSet(startVersion), lastCommit(startVersion),
|
||||
lastDurable(startVersion), nodeCount(nodeCount), keyBytes(keyBytes), dispose(dispose) {}
|
||||
~KVTest() { close(); }
|
||||
void close() {
|
||||
if (store) {
|
||||
|
|
|
@ -52,8 +52,8 @@ struct MakoWorkload : TestWorkload {
|
|||
"CLEAR", "SETCLEAR", "CLEARRANGE", "SETCLEARRANGE",
|
||||
"COMMIT" };
|
||||
MakoWorkload(WorkloadContext const& wcx)
|
||||
: TestWorkload(wcx), xacts("Transactions"), retries("Retries"), conflicts("Conflicts"), commits("Commits"),
|
||||
totalOps("Operations"), loadTime(0.0) {
|
||||
: TestWorkload(wcx), loadTime(0.0), xacts("Transactions"), retries("Retries"), conflicts("Conflicts"),
|
||||
commits("Commits"), totalOps("Operations") {
|
||||
// init parameters from test file
|
||||
// Number of rows populated
|
||||
rowCount = getOption(options, LiteralStringRef("rows"), 10000);
|
||||
|
|
|
@ -39,7 +39,7 @@ struct QueuePushWorkload : TestWorkload {
|
|||
ContinuousSample<double> commitLatencies, GRVLatencies;
|
||||
|
||||
QueuePushWorkload(WorkloadContext const& wcx)
|
||||
: TestWorkload(wcx), commitLatencies(2000), GRVLatencies(2000), transactions("Transactions"), retries("Retries") {
|
||||
: TestWorkload(wcx), transactions("Transactions"), retries("Retries"), commitLatencies(2000), GRVLatencies(2000) {
|
||||
testDuration = getOption(options, LiteralStringRef("testDuration"), 10.0);
|
||||
actorCount = getOption(options, LiteralStringRef("actorCount"), 50);
|
||||
|
||||
|
|
|
@ -121,12 +121,12 @@ struct ReadWriteWorkload : KVWorkload {
|
|||
bool doSetup;
|
||||
|
||||
ReadWriteWorkload(WorkloadContext const& wcx)
|
||||
: KVWorkload(wcx), latencies(sampleSize), readLatencies(sampleSize), fullReadLatencies(sampleSize),
|
||||
commitLatencies(sampleSize), GRVLatencies(sampleSize), readLatencyTotal(0), readLatencyCount(0), loadTime(0.0),
|
||||
dependentReads(false), adjacentReads(false), adjacentWrites(false), clientBegin(0),
|
||||
aTransactions("A Transactions"), bTransactions("B Transactions"), retries("Retries"),
|
||||
totalReadsMetric(LiteralStringRef("RWWorkload.TotalReads")),
|
||||
totalRetriesMetric(LiteralStringRef("RWWorkload.TotalRetries")) {
|
||||
: KVWorkload(wcx), loadTime(0.0), clientBegin(0), dependentReads(false), adjacentReads(false),
|
||||
adjacentWrites(false), totalReadsMetric(LiteralStringRef("RWWorkload.TotalReads")),
|
||||
totalRetriesMetric(LiteralStringRef("RWWorkload.TotalRetries")), aTransactions("A Transactions"),
|
||||
bTransactions("B Transactions"), retries("Retries"), latencies(sampleSize), readLatencies(sampleSize),
|
||||
commitLatencies(sampleSize), GRVLatencies(sampleSize), fullReadLatencies(sampleSize), readLatencyTotal(0),
|
||||
readLatencyCount(0) {
|
||||
transactionSuccessMetric.init(LiteralStringRef("RWWorkload.SuccessfulTransaction"));
|
||||
transactionFailureMetric.init(LiteralStringRef("RWWorkload.FailedTransaction"));
|
||||
readMetric.init(LiteralStringRef("RWWorkload.Read"));
|
||||
|
|
|
@ -38,7 +38,7 @@ struct ReportConflictingKeysWorkload : TestWorkload {
|
|||
PerfIntCounter invalidReports, commits, conflicts, xacts;
|
||||
|
||||
ReportConflictingKeysWorkload(WorkloadContext const& wcx)
|
||||
: TestWorkload(wcx), invalidReports("InvalidReports"), conflicts("Conflicts"), commits("Commits"),
|
||||
: TestWorkload(wcx), invalidReports("InvalidReports"), commits("commits"), conflicts("Conflicts"),
|
||||
xacts("Transactions") {
|
||||
testDuration = getOption(options, LiteralStringRef("testDuration"), 10.0);
|
||||
// transactionsPerSecond = getOption(options, LiteralStringRef("transactionsPerSecond"), 5000.0) / clientCount;
|
||||
|
|
|
@ -41,8 +41,8 @@ struct StorefrontWorkload : TestWorkload {
|
|||
PerfDoubleCounter totalLatency;
|
||||
|
||||
StorefrontWorkload(WorkloadContext const& wcx)
|
||||
: TestWorkload(wcx), transactions("Transactions"), retries("Retries"), totalLatency("Total Latency"),
|
||||
spuriousCommitFailures("Spurious Commit Failures") {
|
||||
: TestWorkload(wcx), transactions("Transactions"), retries("Retries"),
|
||||
spuriousCommitFailures("Spurious Commit Failures"), totalLatency("Total Latency") {
|
||||
testDuration = getOption(options, LiteralStringRef("testDuration"), 10.0);
|
||||
transactionsPerSecond = getOption(options, LiteralStringRef("transactionsPerSecond"), 1000.0);
|
||||
actorCount =
|
||||
|
|
|
@ -31,23 +31,23 @@ namespace {
|
|||
struct TPCCMetrics {
|
||||
static constexpr int latenciesStored = 1000;
|
||||
|
||||
uint64_t successfulStockLevelTransactions, failedStockLevelTransactions, successfulDeliveryTransactions,
|
||||
failedDeliveryTransactions, successfulOrderStatusTransactions, failedOrderStatusTransactions,
|
||||
successfulPaymentTransactions, failedPaymentTransactions, successfulNewOrderTransactions,
|
||||
failedNewOrderTransactions;
|
||||
double stockLevelResponseTime, deliveryResponseTime, orderStatusResponseTime, paymentResponseTime,
|
||||
newOrderResponseTime;
|
||||
uint64_t successfulStockLevelTransactions{ 0 };
|
||||
uint64_t failedStockLevelTransactions{ 0 };
|
||||
uint64_t successfulDeliveryTransactions{ 0 };
|
||||
uint64_t failedDeliveryTransactions{ 0 };
|
||||
uint64_t successfulOrderStatusTransactions{ 0 };
|
||||
uint64_t failedOrderStatusTransactions{ 0 };
|
||||
uint64_t successfulPaymentTransactions{ 0 };
|
||||
uint64_t failedPaymentTransactions{ 0 };
|
||||
uint64_t successfulNewOrderTransactions{ 0 };
|
||||
uint64_t failedNewOrderTransactions{ 0 };
|
||||
double stockLevelResponseTime{ 0.0 };
|
||||
double deliveryResponseTime{ 0.0 };
|
||||
double orderStatusResponseTime{ 0.0 };
|
||||
double paymentResponseTime{ 0.0 };
|
||||
double newOrderResponseTime{ 0.0 };
|
||||
std::vector<double> stockLevelLatencies, deliveryLatencies, orderStatusLatencies, paymentLatencies,
|
||||
newOrderLatencies;
|
||||
TPCCMetrics()
|
||||
: successfulStockLevelTransactions(0), successfulDeliveryTransactions(0), successfulOrderStatusTransactions(0),
|
||||
successfulPaymentTransactions(0), successfulNewOrderTransactions(0), failedStockLevelTransactions(0),
|
||||
failedDeliveryTransactions(0), failedOrderStatusTransactions(0), failedPaymentTransactions(0),
|
||||
failedNewOrderTransactions(0), stockLevelResponseTime(0.0), deliveryResponseTime(0.0),
|
||||
orderStatusResponseTime(0.0), paymentResponseTime(0.0), newOrderResponseTime(0.0),
|
||||
stockLevelLatencies(latenciesStored, 0.0), deliveryLatencies(latenciesStored, 0.0),
|
||||
orderStatusLatencies(latenciesStored, 0.0), paymentLatencies(latenciesStored, 0.0),
|
||||
newOrderLatencies(latenciesStored, 0.0) {}
|
||||
|
||||
void sort() {
|
||||
std::sort(stockLevelLatencies.begin(), stockLevelLatencies.end());
|
||||
|
|
|
@ -121,7 +121,7 @@ struct ThreadSafetyWorkload : TestWorkload {
|
|||
|
||||
Reference<ITransaction> tr;
|
||||
|
||||
ThreadSafetyWorkload(WorkloadContext const& wcx) : TestWorkload(wcx), tr(nullptr), stopped(false) {
|
||||
ThreadSafetyWorkload(WorkloadContext const& wcx) : TestWorkload(wcx), stopped(false) {
|
||||
|
||||
threadsPerClient = getOption(options, LiteralStringRef("threadsPerClient"), 3);
|
||||
threadDuration = getOption(options, LiteralStringRef("threadDuration"), 60.0);
|
||||
|
|
|
@ -57,8 +57,8 @@ struct RWTransactor : ITransactor {
|
|||
int keyCount, keyBytes;
|
||||
|
||||
RWTransactor(int reads, int writes, int keyCount, int keyBytes, int minValueBytes, int maxValueBytes)
|
||||
: reads(reads), writes(writes), keyCount(keyCount), keyBytes(keyBytes), minValueBytes(minValueBytes),
|
||||
maxValueBytes(maxValueBytes) {
|
||||
: reads(reads), writes(writes), minValueBytes(minValueBytes), maxValueBytes(maxValueBytes), keyCount(keyCount),
|
||||
keyBytes(keyBytes) {
|
||||
ASSERT(minValueBytes <= maxValueBytes);
|
||||
valueString = std::string(maxValueBytes, '.');
|
||||
}
|
||||
|
@ -138,7 +138,7 @@ struct ABTransactor : ITransactor {
|
|||
Reference<ITransactor> a, b;
|
||||
double alpha; // 0.0 = all a, 1.0 = all b
|
||||
|
||||
ABTransactor(double alpha, Reference<ITransactor> a, Reference<ITransactor> b) : alpha(alpha), a(a), b(b) {}
|
||||
ABTransactor(double alpha, Reference<ITransactor> a, Reference<ITransactor> b) : a(a), b(b), alpha(alpha) {}
|
||||
|
||||
Future<Void> doTransaction(Database const& db, Stats* stats) override {
|
||||
return deterministicRandom()->random01() >= alpha ? a->doTransaction(db, stats) : b->doTransaction(db, stats);
|
||||
|
@ -154,7 +154,7 @@ struct SweepTransactor : ITransactor {
|
|||
double duration;
|
||||
|
||||
SweepTransactor(double duration, double startDelay, Reference<ITransactor> a, Reference<ITransactor> b)
|
||||
: a(a), b(b), duration(duration), startTime(-1), startDelay(startDelay) {}
|
||||
: a(a), b(b), startTime(-1), startDelay(startDelay), duration(duration) {}
|
||||
|
||||
Future<Void> doTransaction(Database const& db, Stats* stats) override {
|
||||
if (startTime == -1)
|
||||
|
|
|
@ -38,8 +38,8 @@ struct WriteBandwidthWorkload : KVWorkload {
|
|||
ContinuousSample<double> commitLatencies, GRVLatencies;
|
||||
|
||||
WriteBandwidthWorkload(WorkloadContext const& wcx)
|
||||
: KVWorkload(wcx), commitLatencies(2000), GRVLatencies(2000), loadTime(0.0), transactions("Transactions"),
|
||||
retries("Retries") {
|
||||
: KVWorkload(wcx), loadTime(0.0), transactions("Transactions"), retries("Retries"), commitLatencies(2000),
|
||||
GRVLatencies(2000) {
|
||||
testDuration = getOption(options, LiteralStringRef("testDuration"), 10.0);
|
||||
keysPerTransaction = getOption(options, LiteralStringRef("keysPerTransaction"), 100);
|
||||
valueString = std::string(maxValueBytes, '.');
|
||||
|
|
|
@ -64,8 +64,8 @@ struct WriteTagThrottlingWorkload : KVWorkload {
|
|||
static constexpr int MIN_TRANSACTION_TAG_LENGTH = 2;
|
||||
|
||||
WriteTagThrottlingWorkload(WorkloadContext const& wcx)
|
||||
: KVWorkload(wcx), badActorCommitLatency(SAMPLE_SIZE), badActorReadLatency(SAMPLE_SIZE),
|
||||
goodActorCommitLatency(SAMPLE_SIZE), goodActorReadLatency(SAMPLE_SIZE) {
|
||||
: KVWorkload(wcx), badActorReadLatency(SAMPLE_SIZE), goodActorReadLatency(SAMPLE_SIZE),
|
||||
badActorCommitLatency(SAMPLE_SIZE), goodActorCommitLatency(SAMPLE_SIZE) {
|
||||
testDuration = getOption(options, LiteralStringRef("testDuration"), 120.0);
|
||||
badOpRate = getOption(options, LiteralStringRef("badOpRate"), 0.9);
|
||||
numWritePerTr = getOption(options, LiteralStringRef("numWritePerTr"), 1);
|
||||
|
|
|
@ -166,7 +166,7 @@ public:
|
|||
double startDelay = 30.0,
|
||||
bool useDB = true,
|
||||
double databasePingDelay = -1.0)
|
||||
: title(title), dumpAfterTest(dump), clearAfterTest(clear), startDelay(startDelay), useDB(useDB), timeout(600),
|
||||
: title(title), dumpAfterTest(dump), clearAfterTest(clear), useDB(useDB), startDelay(startDelay), timeout(600),
|
||||
databasePingDelay(databasePingDelay), runConsistencyCheck(g_network->isSimulated()),
|
||||
runConsistencyCheckOnCache(false), runConsistencyCheckOnTSS(false), waitForQuiescenceBegin(true),
|
||||
waitForQuiescenceEnd(true), restorePerpetualWiggleSetting(true), simCheckRelocationDuration(false),
|
||||
|
|
|
@ -81,7 +81,7 @@ public:
|
|||
}
|
||||
}
|
||||
|
||||
Deque(Deque&& r) noexcept : begin(r.begin), end(r.end), mask(r.mask), arr(r.arr) {
|
||||
Deque(Deque&& r) noexcept : arr(r.arr), begin(r.begin), end(r.end), mask(r.mask) {
|
||||
r.arr = nullptr;
|
||||
r.begin = r.end = 0;
|
||||
r.mask = -1;
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue