From f596a81073113533b7e6f33929f3ee951150bba3 Mon Sep 17 00:00:00 2001 From: Steve Atherton Date: Sat, 17 Jul 2021 00:11:40 -0700 Subject: [PATCH] Rename ::TRUE and ::FALSE in BooleanParams to ::True and ::False so as to not conflict with the TRUE and FALSE macros provided by the Windows and MacOS SDKs. --- fdbbackup/FileConverter.actor.cpp | 2 +- fdbbackup/FileDecoder.actor.cpp | 2 +- fdbbackup/backup.actor.cpp | 34 +++---- fdbcli/fdbcli.actor.cpp | 6 +- fdbclient/BackupAgent.actor.h | 90 +++++++++---------- fdbclient/BackupAgentBase.actor.cpp | 2 +- fdbclient/BackupContainer.actor.cpp | 4 +- fdbclient/ClientKnobs.cpp | 4 +- fdbclient/DatabaseBackupAgent.actor.cpp | 58 ++++++------ fdbclient/DatabaseContext.h | 16 ++-- fdbclient/FileBackupAgent.actor.cpp | 74 +++++++-------- fdbclient/IConfigTransaction.h | 2 +- fdbclient/IKnobCollection.cpp | 8 +- fdbclient/ISingleThreadTransaction.h | 12 +-- fdbclient/KeyBackedTypes.h | 28 +++--- fdbclient/KeyRangeMap.actor.cpp | 8 +- fdbclient/ManagementAPI.actor.cpp | 4 +- fdbclient/NativeAPI.actor.cpp | 56 ++++++------ fdbclient/NativeAPI.actor.h | 52 +++++------ fdbclient/PaxosConfigTransaction.h | 10 +-- fdbclient/ReadYourWrites.actor.cpp | 22 ++--- fdbclient/ReadYourWrites.h | 20 ++--- fdbclient/SimpleConfigTransaction.h | 10 +-- fdbclient/SpecialKeySpace.actor.cpp | 2 +- fdbclient/SpecialKeySpace.actor.h | 2 +- fdbclient/TaskBucket.actor.cpp | 6 +- fdbclient/TaskBucket.h | 8 +- fdbclient/ThreadSafeTransaction.cpp | 2 +- fdbrpc/LoadBalance.actor.h | 12 +-- fdbserver/BackupWorker.actor.cpp | 2 +- fdbserver/ClusterController.actor.cpp | 6 +- fdbserver/ConfigDatabaseUnitTests.actor.cpp | 4 +- fdbserver/DataDistribution.actor.cpp | 10 +-- fdbserver/GrvProxyServer.actor.cpp | 2 +- fdbserver/LocalConfiguration.actor.cpp | 12 +-- fdbserver/LocalConfiguration.h | 2 +- fdbserver/MetricLogger.actor.cpp | 2 +- fdbserver/MoveKeys.actor.cpp | 4 +- fdbserver/OldTLogServer_6_0.actor.cpp | 2 +- fdbserver/OldTLogServer_6_2.actor.cpp | 4 +- fdbserver/ProxyCommitData.actor.h | 2 +- fdbserver/QuietDatabase.actor.cpp | 2 +- fdbserver/Ratekeeper.actor.cpp | 2 +- fdbserver/RestoreCommon.actor.cpp | 4 +- fdbserver/RestoreWorker.actor.cpp | 2 +- fdbserver/StorageCache.actor.cpp | 6 +- fdbserver/TLogServer.actor.cpp | 4 +- fdbserver/WorkerInterface.actor.h | 4 +- fdbserver/fdbserver.actor.cpp | 8 +- fdbserver/masterserver.actor.cpp | 4 +- fdbserver/storageserver.actor.cpp | 10 +-- fdbserver/tester.actor.cpp | 4 +- fdbserver/worker.actor.cpp | 4 +- fdbserver/workloads/ApiWorkload.actor.cpp | 4 +- fdbserver/workloads/ApiWorkload.h | 8 +- fdbserver/workloads/AtomicRestore.actor.cpp | 4 +- .../workloads/AtomicSwitchover.actor.cpp | 10 +-- ...kupAndParallelRestoreCorrectness.actor.cpp | 16 ++-- .../workloads/BackupCorrectness.actor.cpp | 50 +++++------ fdbserver/workloads/BackupToBlob.actor.cpp | 2 +- fdbserver/workloads/BackupToDBAbort.actor.cpp | 6 +- .../workloads/BackupToDBCorrectness.actor.cpp | 16 ++-- .../workloads/BackupToDBUpgrade.actor.cpp | 6 +- .../workloads/IncrementalBackup.actor.cpp | 20 ++--- fdbserver/workloads/Mako.actor.cpp | 10 +-- fdbserver/workloads/MemoryLifetime.actor.cpp | 4 +- fdbserver/workloads/PopulateTPCC.actor.cpp | 20 ++--- fdbserver/workloads/QueuePush.actor.cpp | 4 +- fdbserver/workloads/RandomSelector.actor.cpp | 6 +- fdbserver/workloads/RestoreBackup.actor.cpp | 4 +- fdbserver/workloads/RyowCorrectness.actor.cpp | 2 +- .../workloads/SelectorCorrectness.actor.cpp | 6 +- fdbserver/workloads/Serializability.actor.cpp | 8 +- fdbserver/workloads/SnapTest.actor.cpp | 2 +- .../SpecialKeySpaceCorrectness.actor.cpp | 10 +-- fdbserver/workloads/SubmitBackup.actor.cpp | 2 +- fdbserver/workloads/TPCC.actor.cpp | 2 +- fdbserver/workloads/Unreadable.actor.cpp | 4 +- fdbserver/workloads/WriteBandwidth.actor.cpp | 2 +- flow/BooleanParam.h | 6 +- flow/Knobs.cpp | 2 +- 81 files changed, 449 insertions(+), 449 deletions(-) diff --git a/fdbbackup/FileConverter.actor.cpp b/fdbbackup/FileConverter.actor.cpp index d875b2345c..cf05cf95eb 100644 --- a/fdbbackup/FileConverter.actor.cpp +++ b/fdbbackup/FileConverter.actor.cpp @@ -598,7 +598,7 @@ int main(int argc, char** argv) { Error::init(); StringRef url(param.container_url); - setupNetwork(0, UseMetrics::TRUE); + setupNetwork(0, UseMetrics::True); TraceEvent::setNetworkThread(); openTraceFile(NetworkAddress(), 10 << 20, 10 << 20, param.log_dir, "convert", param.trace_log_group); diff --git a/fdbbackup/FileDecoder.actor.cpp b/fdbbackup/FileDecoder.actor.cpp index ecf8963950..b8e4bc138f 100644 --- a/fdbbackup/FileDecoder.actor.cpp +++ b/fdbbackup/FileDecoder.actor.cpp @@ -579,7 +579,7 @@ int main(int argc, char** argv) { Error::init(); StringRef url(param.container_url); - setupNetwork(0, UseMetrics::TRUE); + setupNetwork(0, UseMetrics::True); TraceEvent::setNetworkThread(); openTraceFile(NetworkAddress(), 10 << 20, 10 << 20, param.log_dir, "decode", param.trace_log_group); diff --git a/fdbbackup/backup.actor.cpp b/fdbbackup/backup.actor.cpp index 62ddd24179..caab2918b3 100644 --- a/fdbbackup/backup.actor.cpp +++ b/fdbbackup/backup.actor.cpp @@ -1470,7 +1470,7 @@ ACTOR Future getLayerStatus(Reference tr std::string id, ProgramExe exe, Database dest, - Snapshot snapshot = Snapshot::FALSE) { + Snapshot snapshot = Snapshot::False) { // This process will write a document that looks like this: // { backup : { $expires : {}, version: } // so that the value under 'backup' will eventually expire to null and thus be ignored by @@ -1646,7 +1646,7 @@ ACTOR Future cleanupStatus(Reference tr, std::string name, std::string id, int limit = 1) { - state RangeResult docs = wait(tr->getRange(KeyRangeRef(rootKey, strinc(rootKey)), limit, Snapshot::TRUE)); + state RangeResult docs = wait(tr->getRange(KeyRangeRef(rootKey, strinc(rootKey)), limit, Snapshot::True)); state bool readMore = false; state int i; for (i = 0; i < docs.size(); ++i) { @@ -1675,7 +1675,7 @@ ACTOR Future cleanupStatus(Reference tr, } if (readMore) { limit = 10000; - RangeResult docs2 = wait(tr->getRange(KeyRangeRef(rootKey, strinc(rootKey)), limit, Snapshot::TRUE)); + RangeResult docs2 = wait(tr->getRange(KeyRangeRef(rootKey, strinc(rootKey)), limit, Snapshot::True)); docs = std::move(docs2); readMore = false; } @@ -1768,7 +1768,7 @@ ACTOR Future statusUpdateActor(Database statusUpdateDest, tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::LOCK_AWARE); state Future futureStatusDoc = - getLayerStatus(tr, name, id, exe, taskDest, Snapshot::TRUE); + getLayerStatus(tr, name, id, exe, taskDest, Snapshot::True); wait(cleanupStatus(tr, rootKey, name, id)); std::string statusdoc = wait(futureStatusDoc); tr->set(instanceKey, statusdoc); @@ -1858,7 +1858,7 @@ ACTOR Future submitDBBackup(Database src, } wait(backupAgent.submitBackup( - dest, KeyRef(tagName), backupRanges, StopWhenDone::FALSE, StringRef(), StringRef(), LockDB::TRUE)); + dest, KeyRef(tagName), backupRanges, StopWhenDone::False, StringRef(), StringRef(), LockDB::True)); // Check if a backup agent is running bool agentRunning = wait(backupAgent.checkActive(dest)); @@ -2083,7 +2083,7 @@ ACTOR Future abortDBBackup(Database src, try { state DatabaseBackupAgent backupAgent(src); - wait(backupAgent.abortBackup(dest, Key(tagName), partial, AbortOldBackup::FALSE, dstOnly)); + wait(backupAgent.abortBackup(dest, Key(tagName), partial, AbortOldBackup::False, dstOnly)); wait(backupAgent.unlockBackup(dest, Key(tagName))); printf("The DR on tag `%s' was successfully aborted.\n", printable(StringRef(tagName)).c_str()); @@ -2350,7 +2350,7 @@ ACTOR Future runRestore(Database db, verbose, KeyRef(addPrefix), KeyRef(removePrefix), - LockDB::TRUE, + LockDB::True, onlyApplyMutationLogs, inconsistentSnapshotOnly, beginVersion, @@ -2434,7 +2434,7 @@ ACTOR Future runFastRestoreTool(Database db, ranges, KeyRef(container), dbVersion, - LockDB::TRUE, + LockDB::True, randomUID, LiteralStringRef(""), LiteralStringRef(""))); @@ -2861,7 +2861,7 @@ ACTOR Future modifyBackup(Database db, std::string tagName, BackupModifyOp } state BackupConfig config(uidFlag.get().first); - EBackupState s = wait(config.stateEnum().getOrThrow(tr, Snapshot::FALSE, backup_invalid_info())); + EBackupState s = wait(config.stateEnum().getOrThrow(tr, Snapshot::False, backup_invalid_info())); if (!FileBackupAgent::isRunnable(s)) { fprintf(stderr, "Backup on tag '%s' is not runnable.\n", tagName.c_str()); throw backup_error(); @@ -2881,7 +2881,7 @@ ACTOR Future modifyBackup(Database db, std::string tagName, BackupModifyOp } if (options.activeSnapshotIntervalSeconds.present()) { - Version begin = wait(config.snapshotBeginVersion().getOrThrow(tr, Snapshot::FALSE, backup_error())); + Version begin = wait(config.snapshotBeginVersion().getOrThrow(tr, Snapshot::False, backup_error())); config.snapshotTargetEndVersion().set(tr, begin + ((int64_t)options.activeSnapshotIntervalSeconds.get() * CLIENT_KNOBS->CORE_VERSIONSPERSECOND)); @@ -3731,7 +3731,7 @@ int main(int argc, char* argv[]) { } } - IKnobCollection::setGlobalKnobCollection(IKnobCollection::Type::CLIENT, Randomize::FALSE, IsSimulated::FALSE); + IKnobCollection::setGlobalKnobCollection(IKnobCollection::Type::CLIENT, Randomize::False, IsSimulated::False); auto& g_knobs = IKnobCollection::getMutableGlobalKnobCollection(); for (const auto& [knobName, knobValueString] : knobs) { try { @@ -3758,7 +3758,7 @@ int main(int argc, char* argv[]) { } // Reinitialize knobs in order to update knobs that are dependent on explicitly set knobs - g_knobs.initialize(Randomize::FALSE, IsSimulated::FALSE); + g_knobs.initialize(Randomize::False, IsSimulated::False); if (trace) { if (!traceLogGroup.empty()) @@ -3796,7 +3796,7 @@ int main(int argc, char* argv[]) { Reference c; try { - setupNetwork(0, UseMetrics::TRUE); + setupNetwork(0, UseMetrics::True); } catch (Error& e) { fprintf(stderr, "ERROR: %s\n", e.what()); return FDB_EXIT_ERROR; @@ -3840,7 +3840,7 @@ int main(int argc, char* argv[]) { } try { - db = Database::createDatabase(ccf, -1, IsInternal::TRUE, localities); + db = Database::createDatabase(ccf, -1, IsInternal::True, localities); } catch (Error& e) { fprintf(stderr, "ERROR: %s\n", e.what()); fprintf(stderr, "ERROR: Unable to connect to cluster from `%s'\n", ccf->getFilename().c_str()); @@ -3860,7 +3860,7 @@ int main(int argc, char* argv[]) { } try { - sourceDb = Database::createDatabase(sourceCcf, -1, IsInternal::TRUE, localities); + sourceDb = Database::createDatabase(sourceCcf, -1, IsInternal::True, localities); } catch (Error& e) { fprintf(stderr, "ERROR: %s\n", e.what()); fprintf(stderr, "ERROR: Unable to connect to cluster from `%s'\n", sourceCcf->getFilename().c_str()); @@ -3906,7 +3906,7 @@ int main(int argc, char* argv[]) { case BackupType::STATUS: if (!initCluster()) return FDB_EXIT_ERROR; - f = stopAfter(statusBackup(db, tagName, ShowErrors::TRUE, jsonOutput)); + f = stopAfter(statusBackup(db, tagName, ShowErrors::True, jsonOutput)); break; case BackupType::ABORT: @@ -4067,7 +4067,7 @@ int main(int argc, char* argv[]) { encryptionKeyFile)); break; case RestoreType::WAIT: - f = stopAfter(success(ba.waitRestore(db, KeyRef(tagName), Verbose::TRUE))); + f = stopAfter(success(ba.waitRestore(db, KeyRef(tagName), Verbose::True))); break; case RestoreType::ABORT: f = stopAfter( diff --git a/fdbcli/fdbcli.actor.cpp b/fdbcli/fdbcli.actor.cpp index d193d7ea24..bef9e3d43c 100644 --- a/fdbcli/fdbcli.actor.cpp +++ b/fdbcli/fdbcli.actor.cpp @@ -3151,7 +3151,7 @@ struct CLIOptions { } // Reinitialize knobs in order to update knobs that are dependent on explicitly set knobs - g_knobs.initialize(Randomize::FALSE, IsSimulated::FALSE); + g_knobs.initialize(Randomize::False, IsSimulated::False); } int processArg(CSimpleOpt& args) { @@ -3322,7 +3322,7 @@ ACTOR Future cli(CLIOptions opt, LineNoise* plinenoise) { TraceEvent::setNetworkThread(); try { - db = Database::createDatabase(ccf, -1, IsInternal::FALSE); + db = Database::createDatabase(ccf, -1, IsInternal::False); if (!opt.exec.present()) { printf("Using cluster file `%s'.\n", ccf->getFilename().c_str()); } @@ -4924,7 +4924,7 @@ int main(int argc, char** argv) { registerCrashHandler(); - IKnobCollection::setGlobalKnobCollection(IKnobCollection::Type::CLIENT, Randomize::FALSE, IsSimulated::FALSE); + IKnobCollection::setGlobalKnobCollection(IKnobCollection::Type::CLIENT, Randomize::False, IsSimulated::False); #ifdef __unixish__ struct sigaction act; diff --git a/fdbclient/BackupAgent.actor.h b/fdbclient/BackupAgent.actor.h index 69c2caa53f..e23360b531 100644 --- a/fdbclient/BackupAgent.actor.h +++ b/fdbclient/BackupAgent.actor.h @@ -160,7 +160,7 @@ public: static Key getPauseKey(); // parallel restore - Future parallelRestoreFinish(Database cx, UID randomUID, UnlockDB = UnlockDB::TRUE); + Future parallelRestoreFinish(Database cx, UID randomUID, UnlockDB = UnlockDB::True); Future submitParallelRestore(Database cx, Key backupTag, Standalone> backupRanges, @@ -188,29 +188,29 @@ public: Key tagName, Key url, Standalone> ranges, - WaitForComplete = WaitForComplete::TRUE, + WaitForComplete = WaitForComplete::True, Version targetVersion = ::invalidVersion, - Verbose = Verbose::TRUE, + Verbose = Verbose::True, Key addPrefix = Key(), Key removePrefix = Key(), - LockDB = LockDB::TRUE, - OnlyApplyMutationLogs = OnlyApplyMutationLogs::FALSE, - InconsistentSnapshotOnly = InconsistentSnapshotOnly::FALSE, + LockDB = LockDB::True, + OnlyApplyMutationLogs = OnlyApplyMutationLogs::False, + InconsistentSnapshotOnly = InconsistentSnapshotOnly::False, Version beginVersion = ::invalidVersion, Optional const& encryptionKeyFileName = {}); Future restore(Database cx, Optional cxOrig, Key tagName, Key url, - WaitForComplete waitForComplete = WaitForComplete::TRUE, + WaitForComplete waitForComplete = WaitForComplete::True, Version targetVersion = ::invalidVersion, - Verbose verbose = Verbose::TRUE, + Verbose verbose = Verbose::True, KeyRange range = normalKeys, Key addPrefix = Key(), Key removePrefix = Key(), - LockDB lockDB = LockDB::TRUE, - OnlyApplyMutationLogs onlyApplyMutationLogs = OnlyApplyMutationLogs::FALSE, - InconsistentSnapshotOnly inconsistentSnapshotOnly = InconsistentSnapshotOnly::FALSE, + LockDB lockDB = LockDB::True, + OnlyApplyMutationLogs onlyApplyMutationLogs = OnlyApplyMutationLogs::False, + InconsistentSnapshotOnly inconsistentSnapshotOnly = InconsistentSnapshotOnly::False, Version beginVersion = ::invalidVersion, Optional const& encryptionKeyFileName = {}) { Standalone> rangeRef; @@ -267,9 +267,9 @@ public: int snapshotIntervalSeconds, std::string const& tagName, Standalone> backupRanges, - StopWhenDone = StopWhenDone::TRUE, - UsePartitionedLog = UsePartitionedLog::FALSE, - IncrementalBackupOnly = IncrementalBackupOnly::FALSE, + StopWhenDone = StopWhenDone::True, + UsePartitionedLog = UsePartitionedLog::False, + IncrementalBackupOnly = IncrementalBackupOnly::False, Optional const& encryptionKeyFileName = {}); Future submitBackup(Database cx, Key outContainer, @@ -277,9 +277,9 @@ public: int snapshotIntervalSeconds, std::string const& tagName, Standalone> backupRanges, - StopWhenDone stopWhenDone = StopWhenDone::TRUE, - UsePartitionedLog partitionedLog = UsePartitionedLog::FALSE, - IncrementalBackupOnly incrementalBackupOnly = IncrementalBackupOnly::FALSE, + StopWhenDone stopWhenDone = StopWhenDone::True, + UsePartitionedLog partitionedLog = UsePartitionedLog::False, + IncrementalBackupOnly incrementalBackupOnly = IncrementalBackupOnly::False, Optional const& encryptionKeyFileName = {}) { return runRYWTransactionFailIfLocked(cx, [=](Reference tr) { return submitBackup(tr, @@ -318,14 +318,14 @@ public: Future> getLastRestorable(Reference tr, Key tagName, - Snapshot = Snapshot::FALSE); + Snapshot = Snapshot::False); void setLastRestorable(Reference tr, Key tagName, Version version); // stopWhenDone will return when the backup is stopped, if enabled. Otherwise, it // will return when the backup directory is restorable. Future waitBackup(Database cx, std::string tagName, - StopWhenDone = StopWhenDone::TRUE, + StopWhenDone = StopWhenDone::True, Reference* pContainer = nullptr, UID* pUID = nullptr); @@ -397,7 +397,7 @@ public: Standalone> backupRanges, Key addPrefix, Key removePrefix, - ForceAction = ForceAction::FALSE); + ForceAction = ForceAction::False); Future unlockBackup(Reference tr, Key tagName); Future unlockBackup(Database cx, Key tagName) { @@ -416,18 +416,18 @@ public: Future submitBackup(Reference tr, Key tagName, Standalone> backupRanges, - StopWhenDone = StopWhenDone::TRUE, + StopWhenDone = StopWhenDone::True, Key addPrefix = StringRef(), Key removePrefix = StringRef(), - LockDB lockDatabase = LockDB::FALSE, + LockDB lockDatabase = LockDB::False, PreBackupAction backupAction = PreBackupAction::VERIFY); Future submitBackup(Database cx, Key tagName, Standalone> backupRanges, - StopWhenDone stopWhenDone = StopWhenDone::TRUE, + StopWhenDone stopWhenDone = StopWhenDone::True, Key addPrefix = StringRef(), Key removePrefix = StringRef(), - LockDB lockDatabase = LockDB::FALSE, + LockDB lockDatabase = LockDB::False, PreBackupAction backupAction = PreBackupAction::VERIFY) { return runRYWTransaction(cx, [=](Reference tr) { return submitBackup( @@ -443,36 +443,36 @@ public: Future abortBackup(Database cx, Key tagName, - PartialBackup = PartialBackup::FALSE, - AbortOldBackup = AbortOldBackup::FALSE, - DstOnly = DstOnly::FALSE, - WaitForDestUID = WaitForDestUID::FALSE); + PartialBackup = PartialBackup::False, + AbortOldBackup = AbortOldBackup::False, + DstOnly = DstOnly::False, + WaitForDestUID = WaitForDestUID::False); Future getStatus(Database cx, int errorLimit, Key tagName); - Future getStateValue(Reference tr, UID logUid, Snapshot = Snapshot::FALSE); + Future getStateValue(Reference tr, UID logUid, Snapshot = Snapshot::False); Future getStateValue(Database cx, UID logUid) { return runRYWTransaction(cx, [=](Reference tr) { return getStateValue(tr, logUid); }); } - Future getDestUid(Reference tr, UID logUid, Snapshot = Snapshot::FALSE); + Future getDestUid(Reference tr, UID logUid, Snapshot = Snapshot::False); Future getDestUid(Database cx, UID logUid) { return runRYWTransaction(cx, [=](Reference tr) { return getDestUid(tr, logUid); }); } - Future getLogUid(Reference tr, Key tagName, Snapshot = Snapshot::FALSE); + Future getLogUid(Reference tr, Key tagName, Snapshot = Snapshot::False); Future getLogUid(Database cx, Key tagName) { return runRYWTransaction(cx, [=](Reference tr) { return getLogUid(tr, tagName); }); } Future getRangeBytesWritten(Reference tr, UID logUid, - Snapshot = Snapshot::FALSE); - Future getLogBytesWritten(Reference tr, UID logUid, Snapshot = Snapshot::FALSE); + Snapshot = Snapshot::False); + Future getLogBytesWritten(Reference tr, UID logUid, Snapshot = Snapshot::False); // stopWhenDone will return when the backup is stopped, if enabled. Otherwise, it // will return when the backup directory is restorable. - Future waitBackup(Database cx, Key tagName, StopWhenDone = StopWhenDone::TRUE); + Future waitBackup(Database cx, Key tagName, StopWhenDone = StopWhenDone::True); Future waitSubmitted(Database cx, Key tagName); Future waitUpgradeToLatestDrVersion(Database cx, Key tagName); @@ -530,7 +530,7 @@ Future eraseLogData(Reference tr, Key logUidValue, Key destUidValue, Optional endVersion = Optional(), - CheckBackupUID = CheckBackupUID::FALSE, + CheckBackupUID = CheckBackupUID::False, Version backupUid = 0); Key getApplyKey(Version version, Key backupUid); Version getLogKeyVersion(Key key); @@ -542,18 +542,18 @@ ACTOR Future readCommitted(Database cx, PromiseStream results, Reference lock, KeyRangeRef range, - Terminator terminator = Terminator::TRUE, - AccessSystemKeys systemAccess = AccessSystemKeys::FALSE, - LockAware lockAware = LockAware::FALSE); + Terminator terminator = Terminator::True, + AccessSystemKeys systemAccess = AccessSystemKeys::False, + LockAware lockAware = LockAware::False); ACTOR Future readCommitted(Database cx, PromiseStream results, Future active, Reference lock, KeyRangeRef range, std::function(Key key)> groupBy, - Terminator terminator = Terminator::TRUE, - AccessSystemKeys systemAccess = AccessSystemKeys::FALSE, - LockAware lockAware = LockAware::FALSE); + Terminator terminator = Terminator::True, + AccessSystemKeys systemAccess = AccessSystemKeys::False, + LockAware lockAware = LockAware::False); ACTOR Future applyMutations(Database cx, Key uid, Key addPrefix, @@ -614,7 +614,7 @@ public: TagUidMap(const StringRef& prefix) : TagMap(LiteralStringRef("tag->uid/").withPrefix(prefix)), prefix(prefix) {} Future> getAll(Reference tr, - Snapshot snapshot = Snapshot::FALSE) { + Snapshot snapshot = Snapshot::False) { return getAll_impl(this, tr, snapshot); } @@ -630,12 +630,12 @@ static inline KeyBackedTag makeBackupTag(std::string tagName) { } static inline Future> getAllRestoreTags(Reference tr, - Snapshot snapshot = Snapshot::FALSE) { + Snapshot snapshot = Snapshot::False) { return TagUidMap(fileRestorePrefixRange.begin).getAll(tr, snapshot); } static inline Future> getAllBackupTags(Reference tr, - Snapshot snapshot = Snapshot::FALSE) { + Snapshot snapshot = Snapshot::False) { return TagUidMap(fileBackupPrefixRange.begin).getAll(tr, snapshot); } @@ -652,7 +652,7 @@ public: Future toTask(Reference tr, Reference task, - SetValidation setValidation = SetValidation::TRUE) { + SetValidation setValidation = SetValidation::True) { // Set the uid task parameter TaskParams.uid().set(task, uid); diff --git a/fdbclient/BackupAgentBase.actor.cpp b/fdbclient/BackupAgentBase.actor.cpp index fdb374d7ca..774f606344 100644 --- a/fdbclient/BackupAgentBase.actor.cpp +++ b/fdbclient/BackupAgentBase.actor.cpp @@ -579,7 +579,7 @@ Future readCommitted(Database cx, KeyRangeRef range, std::function(Key key)> groupBy) { return readCommitted( - cx, results, Void(), lock, range, groupBy, Terminator::TRUE, AccessSystemKeys::TRUE, LockAware::TRUE); + cx, results, Void(), lock, range, groupBy, Terminator::True, AccessSystemKeys::True, LockAware::True); } ACTOR Future dumpData(Database cx, diff --git a/fdbclient/BackupContainer.actor.cpp b/fdbclient/BackupContainer.actor.cpp index ce2923945e..8f97d6e56e 100644 --- a/fdbclient/BackupContainer.actor.cpp +++ b/fdbclient/BackupContainer.actor.cpp @@ -389,7 +389,7 @@ ACTOR Future timeKeeperVersionFromDatetime(std::string datetime, Databa tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::LOCK_AWARE); state std::vector> results = - wait(versionMap.getRange(tr, 0, time, 1, Snapshot::FALSE, Reverse::TRUE)); + wait(versionMap.getRange(tr, 0, time, 1, Snapshot::False, Reverse::True)); if (results.size() != 1) { // No key less than time was found in the database // Look for a key >= time. @@ -428,7 +428,7 @@ ACTOR Future> timeKeeperEpochsFromVersion(Version v, Reference // Find the highest time < mid state std::vector> results = - wait(versionMap.getRange(tr, min, mid, 1, Snapshot::FALSE, Reverse::TRUE)); + wait(versionMap.getRange(tr, min, mid, 1, Snapshot::False, Reverse::True)); if (results.size() != 1) { if (mid == min) { diff --git a/fdbclient/ClientKnobs.cpp b/fdbclient/ClientKnobs.cpp index 4c735ef9c4..238b99bfc3 100644 --- a/fdbclient/ClientKnobs.cpp +++ b/fdbclient/ClientKnobs.cpp @@ -252,13 +252,13 @@ void ClientKnobs::initialize(Randomize randomize) { TEST_CASE("/fdbclient/knobs/initialize") { // This test depends on TASKBUCKET_TIMEOUT_VERSIONS being defined as a constant multiple of CORE_VERSIONSPERSECOND - ClientKnobs clientKnobs(Randomize::FALSE); + ClientKnobs clientKnobs(Randomize::False); int64_t initialCoreVersionsPerSecond = clientKnobs.CORE_VERSIONSPERSECOND; int initialTaskBucketTimeoutVersions = clientKnobs.TASKBUCKET_TIMEOUT_VERSIONS; clientKnobs.setKnob("core_versionspersecond", initialCoreVersionsPerSecond * 2); ASSERT_EQ(clientKnobs.CORE_VERSIONSPERSECOND, initialCoreVersionsPerSecond * 2); ASSERT_EQ(clientKnobs.TASKBUCKET_TIMEOUT_VERSIONS, initialTaskBucketTimeoutVersions); - clientKnobs.initialize(Randomize::FALSE); + clientKnobs.initialize(Randomize::False); ASSERT_EQ(clientKnobs.CORE_VERSIONSPERSECOND, initialCoreVersionsPerSecond * 2); ASSERT_EQ(clientKnobs.TASKBUCKET_TIMEOUT_VERSIONS, initialTaskBucketTimeoutVersions * 2); return Void(); diff --git a/fdbclient/DatabaseBackupAgent.actor.cpp b/fdbclient/DatabaseBackupAgent.actor.cpp index 356b9538e2..a8de6819dd 100644 --- a/fdbclient/DatabaseBackupAgent.actor.cpp +++ b/fdbclient/DatabaseBackupAgent.actor.cpp @@ -48,10 +48,10 @@ DatabaseBackupAgent::DatabaseBackupAgent() states(subspace.get(BackupAgentBase::keyStates)), config(subspace.get(BackupAgentBase::keyConfig)), errors(subspace.get(BackupAgentBase::keyErrors)), ranges(subspace.get(BackupAgentBase::keyRanges)), taskBucket(new TaskBucket(subspace.get(BackupAgentBase::keyTasks), - AccessSystemKeys::TRUE, - PriorityBatch::FALSE, - LockAware::TRUE)), - futureBucket(new FutureBucket(subspace.get(BackupAgentBase::keyFutures), AccessSystemKeys::TRUE, LockAware::TRUE)), + 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)) {} @@ -60,10 +60,10 @@ DatabaseBackupAgent::DatabaseBackupAgent(Database src) states(subspace.get(BackupAgentBase::keyStates)), config(subspace.get(BackupAgentBase::keyConfig)), errors(subspace.get(BackupAgentBase::keyErrors)), ranges(subspace.get(BackupAgentBase::keyRanges)), taskBucket(new TaskBucket(subspace.get(BackupAgentBase::keyTasks), - AccessSystemKeys::TRUE, - PriorityBatch::FALSE, - LockAware::TRUE)), - futureBucket(new FutureBucket(subspace.get(BackupAgentBase::keyFutures), AccessSystemKeys::TRUE, LockAware::TRUE)), + 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)) { taskBucket->src = src; @@ -241,7 +241,7 @@ struct BackupRangeTaskFunc : TaskFuncBase { state PromiseStream results; state Future rc = readCommitted( - taskBucket->src, results, lock, range, Terminator::TRUE, AccessSystemKeys::TRUE, LockAware::TRUE); + taskBucket->src, results, lock, range, Terminator::True, AccessSystemKeys::True, LockAware::True); state Key rangeBegin = range.begin; state Key rangeEnd; state bool endOfStream = false; @@ -325,18 +325,18 @@ struct BackupRangeTaskFunc : TaskFuncBase { krmGetRanges(tr, prefix, KeyRangeRef(rangeBegin, rangeEnd), BUGGIFY ? 2 : 2000, 1e5); state Future> logVersionValue = tr->get(task->params[BackupAgentBase::keyConfigLogUid].withPrefix(applyMutationsEndRange.begin), - Snapshot::TRUE); - state Future> rangeCountValue = tr->get(rangeCountKey, Snapshot::TRUE); + Snapshot::True); + state Future> rangeCountValue = tr->get(rangeCountKey, Snapshot::True); state Future prevRange = tr->getRange(firstGreaterOrEqual(prefix), lastLessOrEqual(rangeBegin.withPrefix(prefix)), 1, - Snapshot::TRUE, - Reverse::TRUE); + Snapshot::True, + Reverse::True); state Future nextRange = tr->getRange(firstGreaterOrEqual(rangeEnd.withPrefix(prefix)), firstGreaterOrEqual(strinc(prefix)), 1, - Snapshot::TRUE, - Reverse::FALSE); + Snapshot::True, + Reverse::False); state Future verified = taskBucket->keepRunning(tr, task); wait(checkDatabaseLock(tr, @@ -644,7 +644,7 @@ struct EraseLogRangeTaskFunc : TaskFuncBase { task->params[BackupAgentBase::keyConfigLogUid], task->params[BackupAgentBase::destUid], Optional(endVersion), - CheckBackupUID::TRUE, + CheckBackupUID::True, BinaryReader::fromStringRef(task->params[BackupAgentBase::keyFolderId], Unversioned()))); wait(tr->commit()); return Void(); @@ -897,9 +897,9 @@ struct CopyLogRangeTaskFunc : TaskFuncBase { locks[j], ranges[j], decodeBKMutationLogKey, - Terminator::TRUE, - AccessSystemKeys::TRUE, - LockAware::TRUE)); + Terminator::True, + AccessSystemKeys::True, + LockAware::True)); } // copy the range @@ -1202,7 +1202,7 @@ struct FinishedFullBackupTaskFunc : TaskFuncBase { task->params[DatabaseBackupAgent::keyFolderId], Unversioned())) return Void(); - wait(eraseLogData(tr, logUidValue, destUidValue, Optional(), CheckBackupUID::TRUE, backupUid)); + wait(eraseLogData(tr, logUidValue, destUidValue, Optional(), CheckBackupUID::True, backupUid)); wait(tr->commit()); return Void(); } catch (Error& e) { @@ -1607,9 +1607,9 @@ struct OldCopyLogRangeTaskFunc : TaskFuncBase { lock, ranges[i], decodeBKMutationLogKey, - Terminator::TRUE, - AccessSystemKeys::TRUE, - LockAware::TRUE)); + Terminator::True, + AccessSystemKeys::True, + LockAware::True)); dump.push_back(dumpData(cx, task, results[i], lock.getPtr(), taskBucket)); } @@ -1716,7 +1716,7 @@ struct AbortOldBackupTaskFunc : TaskFuncBase { } TraceEvent("DBA_AbortOldBackup").detail("TagName", tagNameKey.printable()); - wait(srcDrAgent.abortBackup(cx, tagNameKey, PartialBackup::FALSE, AbortOldBackup::TRUE)); + wait(srcDrAgent.abortBackup(cx, tagNameKey, PartialBackup::False, AbortOldBackup::True)); return Void(); } @@ -2766,7 +2766,7 @@ public: throw; } - wait(success(backupAgent->waitBackup(dest, tagName, StopWhenDone::TRUE))); + wait(success(backupAgent->waitBackup(dest, tagName, StopWhenDone::True))); TraceEvent("DBA_SwitchoverStopped"); @@ -2795,10 +2795,10 @@ public: wait(drAgent.submitBackup(backupAgent->taskBucket->src, tagName, backupRanges, - StopWhenDone::FALSE, + StopWhenDone::False, addPrefix, removePrefix, - LockDB::TRUE, + LockDB::True, DatabaseBackupAgent::PreBackupAction::NONE)); } catch (Error& e) { if (e.code() != error_code_backup_duplicate) @@ -3078,8 +3078,8 @@ public: errorLimit > 0 ? tr->getRange(backupAgent->errors.get(BinaryWriter::toValue(logUid, Unversioned())).range(), errorLimit, - Snapshot::FALSE, - Reverse::TRUE) + Snapshot::False, + Reverse::True) : Future(); state Future> fBackupUid = tr->get(backupAgent->states.get(BinaryWriter::toValue(logUid, Unversioned())) diff --git a/fdbclient/DatabaseContext.h b/fdbclient/DatabaseContext.h index d95ca71c32..2b1383cb98 100644 --- a/fdbclient/DatabaseContext.h +++ b/fdbclient/DatabaseContext.h @@ -159,9 +159,9 @@ public: LocalityData clientLocality, EnableLocalityLoadBalance, TaskPriority taskID = TaskPriority::DefaultEndpoint, - LockAware = LockAware::FALSE, + LockAware = LockAware::False, int apiVersion = Database::API_VERSION_LATEST, - IsSwitchable = IsSwitchable::FALSE); + IsSwitchable = IsSwitchable::False); ~DatabaseContext(); @@ -180,13 +180,13 @@ public: switchable)); } - std::pair> getCachedLocation(const KeyRef&, Reverse isBackward = Reverse::FALSE); + std::pair> getCachedLocation(const KeyRef&, Reverse isBackward = Reverse::False); bool getCachedLocations(const KeyRangeRef&, vector>>&, int limit, Reverse reverse); Reference setCachedLocation(const KeyRangeRef&, const vector&); - void invalidateCache(const KeyRef&, Reverse isBackward = Reverse::FALSE); + void invalidateCache(const KeyRef&, Reverse isBackward = Reverse::False); void invalidateCache(const KeyRangeRef&); bool sampleReadTags() const; @@ -217,7 +217,7 @@ public: void setOption(FDBDatabaseOptions::Option option, Optional value); Error deferredError; - LockAware lockAware{ LockAware::FALSE }; + LockAware lockAware{ LockAware::False }; bool isError() const { return deferredError.code() != invalid_error_code; } @@ -261,9 +261,9 @@ public: LocalityData const& clientLocality, EnableLocalityLoadBalance, LockAware, - IsInternal = IsInternal::TRUE, + IsInternal = IsInternal::True, int apiVersion = Database::API_VERSION_LATEST, - IsSwitchable = IsSwitchable::FALSE); + IsSwitchable = IsSwitchable::False); explicit DatabaseContext(const Error& err); @@ -282,7 +282,7 @@ public: UID proxiesLastChange; LocalityData clientLocality; QueueModel queueModel; - EnableLocalityLoadBalance enableLocalityLoadBalance{ EnableLocalityLoadBalance::FALSE }; + EnableLocalityLoadBalance enableLocalityLoadBalance{ EnableLocalityLoadBalance::False }; struct VersionRequest { SpanID spanContext; diff --git a/fdbclient/FileBackupAgent.actor.cpp b/fdbclient/FileBackupAgent.actor.cpp index 63d4f1aee3..90cc1ab321 100644 --- a/fdbclient/FileBackupAgent.actor.cpp +++ b/fdbclient/FileBackupAgent.actor.cpp @@ -252,8 +252,8 @@ public: ACTOR static Future getApplyVersionLag_impl(Reference tr, UID uid) { state Future> beginVal = - tr->get(uidPrefixKey(applyMutationsBeginRange.begin, uid), Snapshot::TRUE); - state Future> endVal = tr->get(uidPrefixKey(applyMutationsEndRange.begin, uid), Snapshot::TRUE); + tr->get(uidPrefixKey(applyMutationsBeginRange.begin, uid), Snapshot::True); + state Future> endVal = tr->get(uidPrefixKey(applyMutationsEndRange.begin, uid), Snapshot::True); wait(success(beginVal) && success(endVal)); if (!beginVal.get().present() || !endVal.get().present()) @@ -444,10 +444,10 @@ FileBackupAgent::FileBackupAgent() , config(subspace.get(BackupAgentBase::keyConfig)), lastRestorable(subspace.get(FileBackupAgent::keyLastRestorable)), taskBucket(new TaskBucket(subspace.get(BackupAgentBase::keyTasks), - AccessSystemKeys::TRUE, - PriorityBatch::FALSE, - LockAware::TRUE)), - futureBucket(new FutureBucket(subspace.get(BackupAgentBase::keyFutures), AccessSystemKeys::TRUE, LockAware::TRUE)) { + AccessSystemKeys::True, + PriorityBatch::False, + LockAware::True)), + futureBucket(new FutureBucket(subspace.get(BackupAgentBase::keyFutures), AccessSystemKeys::True, LockAware::True)) { } namespace fileBackup { @@ -870,10 +870,10 @@ ACTOR static Future abortFiveOneBackup(FileBackupAgent* backupAgent, tr->setOption(FDBTransactionOptions::LOCK_AWARE); state KeyBackedTag tag = makeBackupTag(tagName); - state UidAndAbortedFlagT current = wait(tag.getOrThrow(tr, Snapshot::FALSE, backup_unneeded())); + state UidAndAbortedFlagT current = wait(tag.getOrThrow(tr, Snapshot::False, backup_unneeded())); state BackupConfig config(current.first); - EBackupState status = wait(config.stateEnum().getD(tr, Snapshot::FALSE, EBackupState::STATE_NEVERRAN)); + EBackupState status = wait(config.stateEnum().getD(tr, Snapshot::False, EBackupState::STATE_NEVERRAN)); if (!backupAgent->isRunnable(status)) { throw backup_unneeded(); @@ -959,7 +959,7 @@ ACTOR static Future addBackupTask(StringRef name, Reference waitFor = Reference(), std::function)> setupTaskFn = NOP_SETUP_TASK_FN, int priority = 0, - SetValidation setValidation = SetValidation::TRUE) { + SetValidation setValidation = SetValidation::True) { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::LOCK_AWARE); @@ -1114,7 +1114,7 @@ struct BackupRangeTaskFunc : BackupTaskFuncBase { Params.beginKey().set(task, range.end); // Save and extend the task with the new begin parameter - state Version newTimeout = wait(taskBucket->extendTimeout(tr, task, UpdateParams::TRUE)); + state Version newTimeout = wait(taskBucket->extendTimeout(tr, task, UpdateParams::True)); // Update the range bytes written in the backup config backup.rangeBytesWritten().atomicOp(tr, file->size(), MutationRef::AddValue); @@ -1212,9 +1212,9 @@ struct BackupRangeTaskFunc : BackupTaskFuncBase { results, lock, KeyRangeRef(beginKey, endKey), - Terminator::TRUE, - AccessSystemKeys::TRUE, - LockAware::TRUE); + Terminator::True, + AccessSystemKeys::True, + LockAware::True); state RangeFileWriter rangeFile; state BackupConfig backup(task); @@ -2058,7 +2058,7 @@ struct BackupLogRangeTaskFunc : BackupTaskFuncBase { for (auto& range : ranges) { rc.push_back( - readCommitted(cx, results, lock, range, Terminator::FALSE, AccessSystemKeys::TRUE, LockAware::TRUE)); + readCommitted(cx, results, lock, range, Terminator::False, AccessSystemKeys::True, LockAware::True)); } state Future sendEOS = map(errorOr(waitForAll(rc)), [=](ErrorOr const& result) { @@ -2236,7 +2236,7 @@ struct EraseLogRangeTaskFunc : BackupTaskFuncBase { Params.destUidValue().set(task, destUidValue); }, 0, - SetValidation::FALSE)); + SetValidation::False)); return key; } @@ -4072,13 +4072,13 @@ struct StartFullRestoreTaskFunc : RestoreTaskFuncBase { tr->setOption(FDBTransactionOptions::LOCK_AWARE); wait(checkTaskVersion(tr->getDatabase(), task, name, version)); - wait(store(beginVersion, restore.beginVersion().getD(tr, Snapshot::FALSE, ::invalidVersion))); + wait(store(beginVersion, restore.beginVersion().getD(tr, Snapshot::False, ::invalidVersion))); wait(store(restoreVersion, restore.restoreVersion().getOrThrow(tr))); wait(store(ranges, restore.getRestoreRangesOrDefault(tr))); - wait(store(logsOnly, restore.onlyApplyMutationLogs().getD(tr, Snapshot::FALSE, false))); + wait(store(logsOnly, restore.onlyApplyMutationLogs().getD(tr, Snapshot::False, false))); wait(store(inconsistentSnapshotOnly, - restore.inconsistentSnapshotOnly().getD(tr, Snapshot::FALSE, false))); + restore.inconsistentSnapshotOnly().getD(tr, Snapshot::False, false))); wait(taskBucket->keepRunning(tr, task)); @@ -4329,7 +4329,7 @@ public: static constexpr int MAX_RESTORABLE_FILE_METASECTION_BYTES = 1024 * 8; // Parallel restore - ACTOR static Future parallelRestoreFinish(Database cx, UID randomUID, UnlockDB unlockDB = UnlockDB::TRUE) { + ACTOR static Future parallelRestoreFinish(Database cx, UID randomUID, UnlockDB unlockDB = UnlockDB::True) { state ReadYourWritesTransaction tr(cx); state Optional restoreRequestDoneKeyValue; TraceEvent("FastRestoreToolWaitForRestoreToFinish").detail("DBLock", randomUID); @@ -4492,7 +4492,7 @@ public: state BackupConfig config(oldUidAndAborted.get().first); state EBackupState status = - wait(config.stateEnum().getD(tr, Snapshot::FALSE, EBackupState::STATE_NEVERRAN)); + wait(config.stateEnum().getD(tr, Snapshot::False, EBackupState::STATE_NEVERRAN)); // Break, if one of the following is true // - no longer runnable @@ -4502,7 +4502,7 @@ public: if (pContainer != nullptr) { Reference c = - wait(config.backupContainer().getOrThrow(tr, Snapshot::FALSE, backup_invalid_info())); + wait(config.backupContainer().getOrThrow(tr, Snapshot::False, backup_invalid_info())); *pContainer = c; } @@ -4549,7 +4549,7 @@ public: if (uidAndAbortedFlag.present()) { state BackupConfig prevConfig(uidAndAbortedFlag.get().first); state EBackupState prevBackupStatus = - wait(prevConfig.stateEnum().getD(tr, Snapshot::FALSE, EBackupState::STATE_NEVERRAN)); + wait(prevConfig.stateEnum().getD(tr, Snapshot::False, EBackupState::STATE_NEVERRAN)); if (FileBackupAgent::isRunnable(prevBackupStatus)) { throw backup_duplicate(); } @@ -4812,9 +4812,9 @@ public: tr->setOption(FDBTransactionOptions::LOCK_AWARE); state KeyBackedTag tag = makeBackupTag(tagName.toString()); - state UidAndAbortedFlagT current = wait(tag.getOrThrow(tr, Snapshot::FALSE, backup_unneeded())); + state UidAndAbortedFlagT current = wait(tag.getOrThrow(tr, Snapshot::False, backup_unneeded())); state BackupConfig config(current.first); - state EBackupState status = wait(config.stateEnum().getD(tr, Snapshot::FALSE, EBackupState::STATE_NEVERRAN)); + state EBackupState status = wait(config.stateEnum().getD(tr, Snapshot::False, EBackupState::STATE_NEVERRAN)); if (!FileBackupAgent::isRunnable(status)) { throw backup_unneeded(); @@ -4863,11 +4863,11 @@ public: tr->setOption(FDBTransactionOptions::LOCK_AWARE); state KeyBackedTag tag = makeBackupTag(tagName); - state UidAndAbortedFlagT current = wait(tag.getOrThrow(tr, Snapshot::FALSE, backup_unneeded())); + state UidAndAbortedFlagT current = wait(tag.getOrThrow(tr, Snapshot::False, backup_unneeded())); state BackupConfig config(current.first); state Key destUidValue = wait(config.destUidValue().getOrThrow(tr)); - EBackupState status = wait(config.stateEnum().getD(tr, Snapshot::FALSE, EBackupState::STATE_NEVERRAN)); + EBackupState status = wait(config.stateEnum().getD(tr, Snapshot::False, EBackupState::STATE_NEVERRAN)); if (!backupAgent->isRunnable(status)) { throw backup_unneeded(); @@ -4969,7 +4969,7 @@ public: state BackupConfig config(uidAndAbortedFlag.get().first); state EBackupState backupState = - wait(config.stateEnum().getD(tr, Snapshot::FALSE, EBackupState::STATE_NEVERRAN)); + wait(config.stateEnum().getD(tr, Snapshot::False, EBackupState::STATE_NEVERRAN)); JsonBuilderObject statusDoc; statusDoc.setKey("Name", BackupAgentBase::getStateName(backupState)); statusDoc.setKey("Description", BackupAgentBase::getStateText(backupState)); @@ -5114,7 +5114,7 @@ public: if (uidAndAbortedFlag.present()) { config = BackupConfig(uidAndAbortedFlag.get().first); EBackupState status = - wait(config.stateEnum().getD(tr, Snapshot::FALSE, EBackupState::STATE_NEVERRAN)); + wait(config.stateEnum().getD(tr, Snapshot::False, EBackupState::STATE_NEVERRAN)); backupState = status; } @@ -5488,7 +5488,7 @@ public: } } - wait(success(waitBackup(backupAgent, cx, tagName.toString(), StopWhenDone::TRUE))); + wait(success(waitBackup(backupAgent, cx, tagName.toString(), StopWhenDone::True))); TraceEvent("AS_BackupStopped"); ryw_tr->reset(); @@ -5519,7 +5519,7 @@ public: ranges, KeyRef(bc->getURL()), targetVersion, - LockDB::TRUE, + LockDB::True, randomUid, addPrefix, removePrefix)); @@ -5540,14 +5540,14 @@ public: tagName, KeyRef(bc->getURL()), ranges, - WaitForComplete::TRUE, + WaitForComplete::True, ::invalidVersion, - Verbose::TRUE, + Verbose::True, addPrefix, removePrefix, - LockDB::TRUE, - OnlyApplyMutationLogs::FALSE, - InconsistentSnapshotOnly::FALSE, + LockDB::True, + OnlyApplyMutationLogs::False, + InconsistentSnapshotOnly::False, ::invalidVersion, {}, randomUid)); @@ -5565,7 +5565,7 @@ public: Key addPrefix, Key removePrefix) { return success( - atomicRestore(backupAgent, cx, tagName, ranges, addPrefix, removePrefix, UsePartitionedLog::TRUE)); + atomicRestore(backupAgent, cx, tagName, ranges, addPrefix, removePrefix, UsePartitionedLog::True)); } }; @@ -5637,7 +5637,7 @@ Future FileBackupAgent::atomicRestore(Database cx, Key addPrefix, Key removePrefix) { return FileBackupAgentImpl::atomicRestore( - this, cx, tagName, ranges, addPrefix, removePrefix, UsePartitionedLog::FALSE); + this, cx, tagName, ranges, addPrefix, removePrefix, UsePartitionedLog::False); } Future FileBackupAgent::abortRestore(Reference tr, Key tagName) { diff --git a/fdbclient/IConfigTransaction.h b/fdbclient/IConfigTransaction.h index e0c4a64f71..a46c914682 100644 --- a/fdbclient/IConfigTransaction.h +++ b/fdbclient/IConfigTransaction.h @@ -43,7 +43,7 @@ public: // Not implemented: void setVersion(Version) override { throw client_invalid_operation(); } - Future getKey(KeySelector const& key, Snapshot snapshot = Snapshot::FALSE) override { + Future getKey(KeySelector const& key, Snapshot snapshot = Snapshot::False) override { throw client_invalid_operation(); } Future>> getAddressesForKey(Key const& key) override { diff --git a/fdbclient/IKnobCollection.cpp b/fdbclient/IKnobCollection.cpp index aee94dbc97..26a0dcb22d 100644 --- a/fdbclient/IKnobCollection.cpp +++ b/fdbclient/IKnobCollection.cpp @@ -56,17 +56,17 @@ KnobValue IKnobCollection::parseKnobValue(std::string const& knobName, std::stri static std::unique_ptr clientKnobCollection, serverKnobCollection, testKnobCollection; if (type == Type::CLIENT) { if (!clientKnobCollection) { - clientKnobCollection = create(type, Randomize::FALSE, IsSimulated::FALSE); + clientKnobCollection = create(type, Randomize::False, IsSimulated::False); } return clientKnobCollection->parseKnobValue(knobName, knobValue); } else if (type == Type::SERVER) { if (!serverKnobCollection) { - serverKnobCollection = create(type, Randomize::FALSE, IsSimulated::FALSE); + serverKnobCollection = create(type, Randomize::False, IsSimulated::False); } return serverKnobCollection->parseKnobValue(knobName, knobValue); } else if (type == Type::TEST) { if (!testKnobCollection) { - testKnobCollection = create(type, Randomize::FALSE, IsSimulated::FALSE); + testKnobCollection = create(type, Randomize::False, IsSimulated::False); } return testKnobCollection->parseKnobValue(knobName, knobValue); } @@ -74,7 +74,7 @@ KnobValue IKnobCollection::parseKnobValue(std::string const& knobName, std::stri } std::unique_ptr IKnobCollection::globalKnobCollection = - IKnobCollection::create(IKnobCollection::Type::CLIENT, Randomize::FALSE, IsSimulated::FALSE); + IKnobCollection::create(IKnobCollection::Type::CLIENT, Randomize::False, IsSimulated::False); void IKnobCollection::setGlobalKnobCollection(Type type, Randomize randomize, IsSimulated isSimulated) { globalKnobCollection = create(type, randomize, isSimulated); diff --git a/fdbclient/ISingleThreadTransaction.h b/fdbclient/ISingleThreadTransaction.h index 4f219cfdbd..950e723e11 100644 --- a/fdbclient/ISingleThreadTransaction.h +++ b/fdbclient/ISingleThreadTransaction.h @@ -50,18 +50,18 @@ public: virtual void setVersion(Version v) = 0; virtual Future getReadVersion() = 0; virtual Optional getCachedReadVersion() const = 0; - virtual Future> get(const Key& key, Snapshot = Snapshot::FALSE) = 0; - virtual Future getKey(const KeySelector& key, Snapshot = Snapshot::FALSE) = 0; + virtual Future> get(const Key& key, Snapshot = Snapshot::False) = 0; + virtual Future getKey(const KeySelector& key, Snapshot = Snapshot::False) = 0; virtual Future> getRange(const KeySelector& begin, const KeySelector& end, int limit, - Snapshot = Snapshot::FALSE, - Reverse = Reverse::FALSE) = 0; + Snapshot = Snapshot::False, + Reverse = Reverse::False) = 0; virtual Future> getRange(KeySelector begin, KeySelector end, GetRangeLimits limits, - Snapshot = Snapshot::FALSE, - Reverse = Reverse::FALSE) = 0; + Snapshot = Snapshot::False, + Reverse = Reverse::False) = 0; virtual Future>> getAddressesForKey(Key const& key) = 0; virtual Future>> getRangeSplitPoints(KeyRange const& range, int64_t chunkSize) = 0; virtual Future getEstimatedRangeSizeBytes(KeyRange const& keys) = 0; diff --git a/fdbclient/KeyBackedTypes.h b/fdbclient/KeyBackedTypes.h index 7fa26cbea9..b82b5c7357 100644 --- a/fdbclient/KeyBackedTypes.h +++ b/fdbclient/KeyBackedTypes.h @@ -150,7 +150,7 @@ template class KeyBackedProperty { public: KeyBackedProperty(KeyRef key) : key(key) {} - Future> get(Reference tr, Snapshot snapshot = Snapshot::FALSE) const { + Future> get(Reference tr, Snapshot snapshot = Snapshot::False) const { return map(tr->get(key, snapshot), [](Optional const& val) -> Optional { if (val.present()) return Codec::unpack(Tuple::unpack(val.get())); @@ -159,13 +159,13 @@ public: } // Get property's value or defaultValue if it doesn't exist Future getD(Reference tr, - Snapshot snapshot = Snapshot::FALSE, + Snapshot snapshot = Snapshot::False, T defaultValue = T()) const { return map(get(tr, snapshot), [=](Optional val) -> T { return val.present() ? val.get() : defaultValue; }); } // Get property's value or throw error if it doesn't exist Future getOrThrow(Reference tr, - Snapshot snapshot = Snapshot::FALSE, + Snapshot snapshot = Snapshot::False, Error err = key_not_found()) const { auto keyCopy = key; auto backtrace = platform::get_backtrace(); @@ -182,7 +182,7 @@ public: }); } - Future> get(Database cx, Snapshot snapshot = Snapshot::FALSE) const { + Future> get(Database cx, Snapshot snapshot = Snapshot::False) const { auto& copy = *this; return runRYWTransaction(cx, [=](Reference tr) { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); @@ -192,7 +192,7 @@ public: }); } - Future getD(Database cx, Snapshot snapshot = Snapshot::FALSE, T defaultValue = T()) const { + Future getD(Database cx, Snapshot snapshot = Snapshot::False, T defaultValue = T()) const { auto& copy = *this; return runRYWTransaction(cx, [=](Reference tr) { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); @@ -202,7 +202,7 @@ public: }); } - Future getOrThrow(Database cx, Snapshot snapshot = Snapshot::FALSE, Error err = key_not_found()) const { + Future getOrThrow(Database cx, Snapshot snapshot = Snapshot::False, Error err = key_not_found()) const { auto& copy = *this; return runRYWTransaction(cx, [=](Reference tr) { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); @@ -237,7 +237,7 @@ template class KeyBackedBinaryValue { public: KeyBackedBinaryValue(KeyRef key) : key(key) {} - Future> get(Reference tr, Snapshot snapshot = Snapshot::FALSE) const { + Future> get(Reference tr, Snapshot snapshot = Snapshot::False) const { return map(tr->get(key, snapshot), [](Optional const& val) -> Optional { if (val.present()) return BinaryReader::fromStringRef(val.get(), Unversioned()); @@ -246,9 +246,9 @@ public: } // Get property's value or defaultValue if it doesn't exist Future getD(Reference tr, - Snapshot snapshot = Snapshot::FALSE, + Snapshot snapshot = Snapshot::False, T defaultValue = T()) const { - return map(get(tr, Snapshot::FALSE), + return map(get(tr, Snapshot::False), [=](Optional val) -> T { return val.present() ? val.get() : defaultValue; }); } void set(Reference tr, T const& val) { @@ -278,8 +278,8 @@ public: KeyType const& begin, Optional const& end, int limit, - Snapshot snapshot = Snapshot::FALSE, - Reverse reverse = Reverse::FALSE) const { + Snapshot snapshot = Snapshot::False, + Reverse reverse = Reverse::False) const { Subspace s = space; // 'this' could be invalid inside lambda Key endKey = end.present() ? s.pack(Codec::pack(end.get())) : space.range().end; return map( @@ -298,7 +298,7 @@ public: Future> get(Reference tr, KeyType const& key, - Snapshot snapshot = Snapshot::FALSE) const { + Snapshot snapshot = Snapshot::False) const { return map(tr->get(space.pack(Codec::pack(key)), snapshot), [](Optional const& val) -> Optional { if (val.present()) @@ -344,7 +344,7 @@ public: ValueType const& begin, Optional const& end, int limit, - Snapshot snapshot = Snapshot::FALSE) const { + Snapshot snapshot = Snapshot::False) const { Subspace s = space; // 'this' could be invalid inside lambda Key endKey = end.present() ? s.pack(Codec::pack(end.get())) : space.range().end; return map( @@ -360,7 +360,7 @@ public: Future exists(Reference tr, ValueType const& val, - Snapshot snapshot = Snapshot::FALSE) const { + Snapshot snapshot = Snapshot::False) const { return map(tr->get(space.pack(Codec::pack(val)), snapshot), [](Optional const& val) -> bool { return val.present(); }); } diff --git a/fdbclient/KeyRangeMap.actor.cpp b/fdbclient/KeyRangeMap.actor.cpp index 3ca129872e..607bc56d97 100644 --- a/fdbclient/KeyRangeMap.actor.cpp +++ b/fdbclient/KeyRangeMap.actor.cpp @@ -120,7 +120,7 @@ ACTOR Future krmSetRange(Transaction* tr, Key mapPrefix, KeyRange range, V state KeyRange withPrefix = KeyRangeRef(mapPrefix.toString() + range.begin.toString(), mapPrefix.toString() + range.end.toString()); RangeResult old = - wait(tr->getRange(lastLessOrEqual(withPrefix.end), firstGreaterThan(withPrefix.end), 1, Snapshot::TRUE)); + wait(tr->getRange(lastLessOrEqual(withPrefix.end), firstGreaterThan(withPrefix.end), 1, Snapshot::True)); Value oldValue; bool hasResult = old.size() > 0 && old[0].key.startsWith(mapPrefix); @@ -142,7 +142,7 @@ ACTOR Future krmSetRange(Reference tr, Key mapP state KeyRange withPrefix = KeyRangeRef(mapPrefix.toString() + range.begin.toString(), mapPrefix.toString() + range.end.toString()); RangeResult old = - wait(tr->getRange(lastLessOrEqual(withPrefix.end), firstGreaterThan(withPrefix.end), 1, Snapshot::TRUE)); + wait(tr->getRange(lastLessOrEqual(withPrefix.end), firstGreaterThan(withPrefix.end), 1, Snapshot::True)); Value oldValue; bool hasResult = old.size() > 0 && old[0].key.startsWith(mapPrefix); @@ -178,9 +178,9 @@ static Future krmSetRangeCoalescing_(Transaction* tr, state vector> keys; keys.push_back( - tr->getRange(lastLessThan(withPrefix.begin), firstGreaterOrEqual(withPrefix.begin), 1, Snapshot::TRUE)); + tr->getRange(lastLessThan(withPrefix.begin), firstGreaterOrEqual(withPrefix.begin), 1, Snapshot::True)); keys.push_back( - tr->getRange(lastLessOrEqual(withPrefix.end), firstGreaterThan(withPrefix.end) + 1, 2, Snapshot::TRUE)); + tr->getRange(lastLessOrEqual(withPrefix.end), firstGreaterThan(withPrefix.end) + 1, 2, Snapshot::True)); wait(waitForAll(keys)); // Determine how far to extend this range at the beginning diff --git a/fdbclient/ManagementAPI.actor.cpp b/fdbclient/ManagementAPI.actor.cpp index 38f79e8ac1..7015b6fa92 100644 --- a/fdbclient/ManagementAPI.actor.cpp +++ b/fdbclient/ManagementAPI.actor.cpp @@ -2474,7 +2474,7 @@ ACTOR Future changeCachedRange(Database cx, KeyRangeRef range, bool add) { tr.clear(privateRange); tr.addReadConflictRange(privateRange); RangeResult previous = - wait(tr.getRange(KeyRangeRef(storageCachePrefix, sysRange.begin), 1, Snapshot::TRUE)); + wait(tr.getRange(KeyRangeRef(storageCachePrefix, sysRange.begin), 1, Snapshot::True)); bool prevIsCached = false; if (!previous.empty()) { std::vector prevVal; @@ -2490,7 +2490,7 @@ ACTOR Future changeCachedRange(Database cx, KeyRangeRef range, bool add) { tr.set(sysRange.begin, trueValue); tr.set(privateRange.begin, serverKeysTrue); } - RangeResult after = wait(tr.getRange(KeyRangeRef(sysRange.end, storageCacheKeys.end), 1, Snapshot::FALSE)); + RangeResult after = wait(tr.getRange(KeyRangeRef(sysRange.end, storageCacheKeys.end), 1, Snapshot::False)); bool afterIsCached = false; if (!after.empty()) { std::vector afterVal; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index f588f31418..305ec8fc63 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -95,7 +95,7 @@ Future loadBalance( const Request& request = Request(), TaskPriority taskID = TaskPriority::DefaultPromiseEndpoint, AtMostOnce atMostOnce = - AtMostOnce::FALSE, // if true, throws request_maybe_delivered() instead of retrying automatically + AtMostOnce::False, // if true, throws request_maybe_delivered() instead of retrying automatically QueueModel* model = nullptr) { if (alternatives->hasCaches) { return loadBalance(alternatives->locations(), channel, request, taskID, atMostOnce, model); @@ -494,7 +494,7 @@ ACTOR static Future delExcessClntTxnEntriesActor(Transaction* tr, int64_t tr->reset(); tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::LOCK_AWARE); - Optional ctrValue = wait(tr->get(KeyRef(clientLatencyAtomicCtr), Snapshot::TRUE)); + Optional ctrValue = wait(tr->get(KeyRef(clientLatencyAtomicCtr), Snapshot::True)); if (!ctrValue.present()) { TraceEvent(SevInfo, "NumClntTxnEntriesNotFound"); return Void(); @@ -1369,7 +1369,7 @@ DatabaseContext::DatabaseContext(const Error& err) 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), + transactionsExpensiveClearCostEstCount("ExpensiveClearCostEstCount", cc), internal(IsInternal::False), transactionTracingEnabled(true) {} // Static constructor used by server processes to create a DatabaseContext @@ -1390,7 +1390,7 @@ Database DatabaseContext::create(Reference> clientInfo, clientLocality, enableLocalityLoadBalance, lockAware, - IsInternal::TRUE, + IsInternal::True, apiVersion, switchable)); } @@ -1739,11 +1739,11 @@ Database Database::createDatabase(Reference connFile, clientInfoMonitor, TaskPriority::DefaultEndpoint, clientLocality, - EnableLocalityLoadBalance::TRUE, - LockAware::FALSE, + EnableLocalityLoadBalance::True, + LockAware::False, internal, apiVersion, - IsSwitchable::TRUE); + IsSwitchable::True); } else { db = new DatabaseContext(connectionFile, clientInfo, @@ -1751,11 +1751,11 @@ Database Database::createDatabase(Reference connFile, clientInfoMonitor, TaskPriority::DefaultEndpoint, clientLocality, - EnableLocalityLoadBalance::TRUE, - LockAware::FALSE, + EnableLocalityLoadBalance::True, + LockAware::False, internal, apiVersion, - IsSwitchable::TRUE); + IsSwitchable::True); } auto database = Database(db); @@ -2248,7 +2248,7 @@ void updateTssMappings(Database cx, const GetKeyServerLocationsReply& reply) { ACTOR Future>> getKeyLocation_internal(Database cx, Key key, TransactionInfo info, - Reverse isBackward = Reverse::FALSE) { + Reverse isBackward = Reverse::False) { state Span span("NAPI:getKeyLocation"_loc, info.spanID); if (isBackward) { ASSERT(key != allKeys.begin && key <= allKeys.end); @@ -2287,7 +2287,7 @@ Future>> getKeyLocation(Database const& c Key const& key, F StorageServerInterface::*member, TransactionInfo const& info, - Reverse isBackward = Reverse::FALSE) { + Reverse isBackward = Reverse::False) { // we first check whether this range is cached auto ssi = cx->getCachedLocation(key, isBackward); if (!ssi.second) { @@ -2395,7 +2395,7 @@ ACTOR Future warmRange_impl(Transaction* self, Database cx, KeyRange keys) state int totalRequests = 0; loop { vector>> locations = wait( - getKeyRangeLocations_internal(cx, keys, CLIENT_KNOBS->WARM_RANGE_SHARD_LIMIT, Reverse::FALSE, self->info)); + getKeyRangeLocations_internal(cx, keys, CLIENT_KNOBS->WARM_RANGE_SHARD_LIMIT, Reverse::False, self->info)); totalRanges += CLIENT_KNOBS->WARM_RANGE_SHARD_LIMIT; totalRequests++; if (locations.size() == 0 || totalRanges >= cx->locationCacheSize || @@ -2478,7 +2478,7 @@ ACTOR Future> getValue(Future version, GetValueRequest( span.context, key, ver, cx->sampleReadTags() ? tags : Optional(), getValueID), TaskPriority::DefaultPromiseEndpoint, - AtMostOnce::FALSE, + AtMostOnce::False, cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr))) { reply = _reply; } @@ -2590,7 +2590,7 @@ ACTOR Future getKey(Database cx, KeySelector k, Future version, Tr &StorageServerInterface::getKey, req, TaskPriority::DefaultPromiseEndpoint, - AtMostOnce::FALSE, + AtMostOnce::False, cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr))) { reply = _reply; } @@ -2983,7 +2983,7 @@ ACTOR Future getExactRange(Database cx, &StorageServerInterface::getKeyValues, req, TaskPriority::DefaultPromiseEndpoint, - AtMostOnce::FALSE, + AtMostOnce::False, cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr))) { rep = _rep; } @@ -3334,7 +3334,7 @@ ACTOR Future getRange(Database cx, &StorageServerInterface::getKeyValues, req, TaskPriority::DefaultPromiseEndpoint, - AtMostOnce::FALSE, + AtMostOnce::False, cx->enableLocalityLoadBalance ? &cx->queueModel : nullptr)); rep = _rep; ++cx->transactionPhysicalReadsCompleted; @@ -4045,7 +4045,7 @@ Future getRange(Database const& cx, end, limits, Promise>(), - Snapshot::TRUE, + Snapshot::True, reverse, info, tags); @@ -4264,7 +4264,7 @@ ACTOR Future>> getAddressesForKeyActor(Key key lastLessOrEqual(serverTagKeys.begin), firstGreaterThan(serverTagKeys.end), GetRangeLimits(CLIENT_KNOBS->TOO_MANY), - Reverse::FALSE, + Reverse::False, info, options.readTags)); ASSERT(!serverTagResult.more && serverTagResult.size() < CLIENT_KNOBS->TOO_MANY); @@ -4273,7 +4273,7 @@ ACTOR Future>> getAddressesForKeyActor(Key key lastLessOrEqual(ksKey), firstGreaterThan(ksKey), GetRangeLimits(1), - Reverse::FALSE, + Reverse::False, info, options.readTags); RangeResult serverUids = wait(futureServerUids); @@ -4917,7 +4917,7 @@ ACTOR Future> estimateCommitCosts(Transac wait(getKeyRangeLocations(self->getDatabase(), keyRange, CLIENT_KNOBS->TOO_MANY, - Reverse::FALSE, + Reverse::False, &StorageServerInterface::getShardState, self->info)); if (locations.empty()) { @@ -5018,7 +5018,7 @@ ACTOR static Future tryCommit(Database cx, &CommitProxyInterface::commit, req, TaskPriority::DefaultPromiseEndpoint, - AtMostOnce::TRUE); + AtMostOnce::True); } choose { @@ -5911,7 +5911,7 @@ ACTOR Future getStorageMetricsLargeKeyRange(Database cx, KeyRang wait(getKeyRangeLocations(cx, keys, std::numeric_limits::max(), - Reverse::FALSE, + Reverse::False, &StorageServerInterface::waitMetrics, TransactionInfo(TaskPriority::DataDistribution, span.context))); state int nLocs = locations.size(); @@ -6010,7 +6010,7 @@ ACTOR Future>> getReadHotRanges(Da wait(getKeyRangeLocations(cx, keys, shardLimit, - Reverse::FALSE, + Reverse::False, &StorageServerInterface::getReadHotRanges, TransactionInfo(TaskPriority::DataDistribution, span.context))); try { @@ -6078,7 +6078,7 @@ ACTOR Future, int>> waitStorageMetrics(Databa wait(getKeyRangeLocations(cx, keys, shardLimit, - Reverse::FALSE, + Reverse::False, &StorageServerInterface::waitMetrics, TransactionInfo(TaskPriority::DataDistribution, span.context))); if (expectedShardCount >= 0 && locations.size() != expectedShardCount) { @@ -6170,7 +6170,7 @@ ACTOR Future>> getRangeSplitPoints(Database cx, Key wait(getKeyRangeLocations(cx, keys, CLIENT_KNOBS->TOO_MANY, - Reverse::FALSE, + Reverse::False, &StorageServerInterface::getRangeSplitPoints, TransactionInfo(TaskPriority::DataDistribution, span.context))); try { @@ -6231,7 +6231,7 @@ ACTOR Future>> splitStorageMetrics(Database cx, wait(getKeyRangeLocations(cx, keys, CLIENT_KNOBS->STORAGE_METRICS_SHARD_LIMIT, - Reverse::FALSE, + Reverse::False, &StorageServerInterface::splitMetrics, TransactionInfo(TaskPriority::DataDistribution, span.context))); state StorageMetrics used; @@ -6338,7 +6338,7 @@ ACTOR Future snapCreate(Database cx, Standalone snapCmd, UID sn &CommitProxyInterface::proxySnapReq, ProxySnapRequest(snapCmd, snapUID, snapUID), cx->taskID, - AtMostOnce::TRUE))) { + AtMostOnce::True))) { TraceEvent("SnapCreateExit").detail("SnapCmd", snapCmd.toString()).detail("UID", snapUID); return Void(); } diff --git a/fdbclient/NativeAPI.actor.h b/fdbclient/NativeAPI.actor.h index 9709b02d1b..6357bb7d80 100644 --- a/fdbclient/NativeAPI.actor.h +++ b/fdbclient/NativeAPI.actor.h @@ -84,13 +84,13 @@ public: // on another thread static Database createDatabase(Reference connFile, int apiVersion, - IsInternal internal = IsInternal::TRUE, + IsInternal internal = IsInternal::True, LocalityData const& clientLocality = LocalityData(), DatabaseContext* preallocatedDb = nullptr); static Database createDatabase(std::string connFileName, int apiVersion, - IsInternal internal = IsInternal::TRUE, + IsInternal internal = IsInternal::True, LocalityData const& clientLocality = LocalityData()); Database() {} // an uninitialized database can be destructed or reassigned safely; that's it @@ -115,7 +115,7 @@ private: void setNetworkOption(FDBNetworkOptions::Option option, Optional value = Optional()); // Configures the global networking machinery -void setupNetwork(uint64_t transportId = 0, UseMetrics = UseMetrics::FALSE); +void setupNetwork(uint64_t transportId = 0, UseMetrics = UseMetrics::False); // This call blocks while the network is running. To use the API in a single-threaded // environment, the calling program must have ACTORs already launched that are waiting @@ -249,24 +249,24 @@ public: Future getRawReadVersion(); Optional getCachedReadVersion() const; - [[nodiscard]] Future> get(const Key& key, Snapshot = Snapshot::FALSE); + [[nodiscard]] Future> get(const Key& key, Snapshot = Snapshot::False); [[nodiscard]] Future watch(Reference watch); - [[nodiscard]] Future getKey(const KeySelector& key, Snapshot = Snapshot::FALSE); + [[nodiscard]] Future getKey(const KeySelector& key, Snapshot = Snapshot::False); // Future< Optional > get( const KeySelectorRef& key ); [[nodiscard]] Future getRange(const KeySelector& begin, const KeySelector& end, int limit, - Snapshot = Snapshot::FALSE, - Reverse = Reverse::FALSE); + Snapshot = Snapshot::False, + Reverse = Reverse::False); [[nodiscard]] Future getRange(const KeySelector& begin, const KeySelector& end, GetRangeLimits limits, - Snapshot = Snapshot::FALSE, - Reverse = Reverse::FALSE); + Snapshot = Snapshot::False, + Reverse = Reverse::False); [[nodiscard]] Future getRange(const KeyRange& keys, int limit, - Snapshot snapshot = Snapshot::FALSE, - Reverse reverse = Reverse::FALSE) { + Snapshot snapshot = Snapshot::False, + Reverse reverse = Reverse::False) { return getRange(KeySelector(firstGreaterOrEqual(keys.begin), keys.arena()), KeySelector(firstGreaterOrEqual(keys.end), keys.arena()), limit, @@ -275,8 +275,8 @@ public: } [[nodiscard]] Future getRange(const KeyRange& keys, GetRangeLimits limits, - Snapshot snapshot = Snapshot::FALSE, - Reverse reverse = Reverse::FALSE) { + Snapshot snapshot = Snapshot::False, + Reverse reverse = Reverse::False) { return getRange(KeySelector(firstGreaterOrEqual(keys.begin), keys.arena()), KeySelector(firstGreaterOrEqual(keys.end), keys.arena()), limits, @@ -290,19 +290,19 @@ public: const KeySelector& begin, const KeySelector& end, int limit, - Snapshot = Snapshot::FALSE, - Reverse = Reverse::FALSE); + Snapshot = Snapshot::False, + Reverse = Reverse::False); [[nodiscard]] Future getRangeStream(const PromiseStream>& results, const KeySelector& begin, const KeySelector& end, GetRangeLimits limits, - Snapshot = Snapshot::FALSE, - Reverse = Reverse::FALSE); + Snapshot = Snapshot::False, + Reverse = Reverse::False); [[nodiscard]] Future getRangeStream(const PromiseStream>& results, const KeyRange& keys, int limit, - Snapshot snapshot = Snapshot::FALSE, - Reverse reverse = Reverse::FALSE) { + Snapshot snapshot = Snapshot::False, + Reverse reverse = Reverse::False) { return getRangeStream(results, KeySelector(firstGreaterOrEqual(keys.begin), keys.arena()), KeySelector(firstGreaterOrEqual(keys.end), keys.arena()), @@ -313,8 +313,8 @@ public: [[nodiscard]] Future getRangeStream(const PromiseStream>& results, const KeyRange& keys, GetRangeLimits limits, - Snapshot snapshot = Snapshot::FALSE, - Reverse reverse = Reverse::FALSE) { + Snapshot snapshot = Snapshot::False, + Reverse reverse = Reverse::False) { return getRangeStream(results, KeySelector(firstGreaterOrEqual(keys.begin), keys.arena()), KeySelector(firstGreaterOrEqual(keys.end), keys.arena()), @@ -349,13 +349,13 @@ public: // The returned list would still be in form of [keys.begin, splitPoint1, splitPoint2, ... , keys.end] Future>> getRangeSplitPoints(KeyRange const& keys, int64_t chunkSize); // If checkWriteConflictRanges is true, existing write conflict ranges will be searched for this key - void set(const KeyRef& key, const ValueRef& value, AddConflictRange = AddConflictRange::TRUE); + void set(const KeyRef& key, const ValueRef& value, AddConflictRange = AddConflictRange::True); void atomicOp(const KeyRef& key, const ValueRef& value, MutationRef::Type operationType, - AddConflictRange = AddConflictRange::TRUE); - void clear(const KeyRangeRef& range, AddConflictRange = AddConflictRange::TRUE); - void clear(const KeyRef& key, AddConflictRange = AddConflictRange::TRUE); + AddConflictRange = AddConflictRange::True); + void clear(const KeyRangeRef& range, AddConflictRange = AddConflictRange::True); + void clear(const KeyRef& key, AddConflictRange = AddConflictRange::True); [[nodiscard]] Future commit(); // Throws not_committed or commit_unknown_result errors in normal operation void setOption(FDBTransactionOptions::Option option, Optional value = Optional()); @@ -452,7 +452,7 @@ inline uint64_t getWriteOperationCost(uint64_t bytes) { // Create a transaction to set the value of system key \xff/conf/perpetual_storage_wiggle. If enable == true, the value // will be 1. Otherwise, the value will be 0. -ACTOR Future setPerpetualStorageWiggle(Database cx, bool enable, LockAware lockAware = LockAware::FALSE); +ACTOR Future setPerpetualStorageWiggle(Database cx, bool enable, LockAware lockAware = LockAware::False); #include "flow/unactorcompiler.h" #endif diff --git a/fdbclient/PaxosConfigTransaction.h b/fdbclient/PaxosConfigTransaction.h index ea3c130f8f..7c68fcba05 100644 --- a/fdbclient/PaxosConfigTransaction.h +++ b/fdbclient/PaxosConfigTransaction.h @@ -39,17 +39,17 @@ public: Future getReadVersion() override; Optional getCachedReadVersion() const override; - Future> get(Key const& key, Snapshot = Snapshot::FALSE) override; + Future> get(Key const& key, Snapshot = Snapshot::False) override; Future> getRange(KeySelector const& begin, KeySelector const& end, int limit, - Snapshot = Snapshot::FALSE, - Reverse = Reverse::FALSE) override; + Snapshot = Snapshot::False, + Reverse = Reverse::False) override; Future> getRange(KeySelector begin, KeySelector end, GetRangeLimits limits, - Snapshot = Snapshot::FALSE, - Reverse = Reverse::FALSE) override; + Snapshot = Snapshot::False, + Reverse = Reverse::False) override; void set(KeyRef const& key, ValueRef const& value) override; void clear(KeyRangeRef const&) override { throw client_invalid_operation(); } void clear(KeyRef const&) override; diff --git a/fdbclient/ReadYourWrites.actor.cpp b/fdbclient/ReadYourWrites.actor.cpp index d559cb1f99..f3f9a391c7 100644 --- a/fdbclient/ReadYourWrites.actor.cpp +++ b/fdbclient/ReadYourWrites.actor.cpp @@ -99,7 +99,7 @@ public: } else if (it->is_empty_range()) { return Optional(); } else { - Optional res = wait(ryw->tr.get(read.key, Snapshot::TRUE)); + Optional res = wait(ryw->tr.get(read.key, Snapshot::True)); KeyRef k(ryw->arena, read.key); if (res.present()) { @@ -188,7 +188,7 @@ public: } RangeResult v = wait( - ryw->tr.getRange(read.begin, read.end, read.limits, snapshot, backwards ? Reverse::TRUE : Reverse::FALSE)); + ryw->tr.getRange(read.begin, read.end, read.limits, snapshot, backwards ? Reverse::True : Reverse::False)); KeyRef maxKey = ryw->getMaxReadKey(); if (v.size() > 0) { if (!backwards && v[v.size() - 1].key >= maxKey) { @@ -694,7 +694,7 @@ public: additionalRows = 0; RangeResult snapshot_read = - wait(ryw->tr.getRange(read_begin, read_end, requestLimit, Snapshot::TRUE, Reverse::FALSE)); + wait(ryw->tr.getRange(read_begin, read_end, requestLimit, Snapshot::True, Reverse::False)); KeyRangeRef range = getKnownKeyRange(snapshot_read, read_begin, read_end, ryw->arena); //TraceEvent("RYWCacheInsert", randomID).detail("Range", range).detail("ExpectedSize", snapshot_read.expectedSize()).detail("Rows", snapshot_read.size()).detail("Results", snapshot_read).detail("More", snapshot_read.more).detail("ReadToBegin", snapshot_read.readToBegin).detail("ReadThroughEnd", snapshot_read.readThroughEnd).detail("ReadThrough", snapshot_read.readThrough); @@ -998,7 +998,7 @@ public: additionalRows = 0; RangeResult snapshot_read = - wait(ryw->tr.getRange(read_begin, read_end, requestLimit, Snapshot::TRUE, Reverse::TRUE)); + wait(ryw->tr.getRange(read_begin, read_end, requestLimit, Snapshot::True, Reverse::True)); KeyRangeRef range = getKnownKeyRangeBack(snapshot_read, read_begin, read_end, ryw->arena); //TraceEvent("RYWCacheInsert", randomID).detail("Range", range).detail("ExpectedSize", snapshot_read.expectedSize()).detail("Rows", snapshot_read.size()).detail("Results", snapshot_read).detail("More", snapshot_read.more).detail("ReadToBegin", snapshot_read.readToBegin).detail("ReadThroughEnd", snapshot_read.readThroughEnd).detail("ReadThrough", snapshot_read.readThrough); @@ -1115,7 +1115,7 @@ public: if (!ryw->options.readYourWritesDisabled) { ryw->watchMap[key].push_back(watch); - val = readWithConflictRange(ryw, GetValueReq(key), Snapshot::FALSE); + val = readWithConflictRange(ryw, GetValueReq(key), Snapshot::False); } else { ryw->approximateSize += 2 * key.expectedSize() + 1; val = ryw->tr.get(key); @@ -1637,13 +1637,13 @@ void ReadYourWritesTransaction::writeRangeToNativeTransaction(KeyRangeRef const& inClearRange = true; } else if (!it.is_cleared_range() && inClearRange) { tr.clear(KeyRangeRef(clearBegin.toArenaOrRef(arena), it.beginKey().toArenaOrRef(arena)), - AddConflictRange::FALSE); + AddConflictRange::False); inClearRange = false; } } if (inClearRange) { - tr.clear(KeyRangeRef(clearBegin.toArenaOrRef(arena), keys.end), AddConflictRange::FALSE); + tr.clear(KeyRangeRef(clearBegin.toArenaOrRef(arena), keys.end), AddConflictRange::False); } it.skip(keys.begin); @@ -1667,9 +1667,9 @@ void ReadYourWritesTransaction::writeRangeToNativeTransaction(KeyRangeRef const& switch (op[i].type) { case MutationRef::SetValue: if (op[i].value.present()) { - tr.set(it.beginKey().assertRef(), op[i].value.get(), AddConflictRange::FALSE); + tr.set(it.beginKey().assertRef(), op[i].value.get(), AddConflictRange::False); } else { - tr.clear(it.beginKey().assertRef(), AddConflictRange::FALSE); + tr.clear(it.beginKey().assertRef(), AddConflictRange::False); } break; case MutationRef::AddValue: @@ -1686,7 +1686,7 @@ void ReadYourWritesTransaction::writeRangeToNativeTransaction(KeyRangeRef const& case MutationRef::MinV2: case MutationRef::AndV2: case MutationRef::CompareAndClear: - tr.atomicOp(it.beginKey().assertRef(), op[i].value.get(), op[i].type, AddConflictRange::FALSE); + tr.atomicOp(it.beginKey().assertRef(), op[i].value.get(), op[i].type, AddConflictRange::False); break; default: break; @@ -1899,7 +1899,7 @@ void ReadYourWritesTransaction::atomicOp(const KeyRef& key, const ValueRef& oper // this does validation of the key and needs to be performed before the readYourWritesDisabled path KeyRangeRef range = getVersionstampKeyRange(arena, k, tr.getCachedReadVersion().orDefault(0), getMaxReadKey()); versionStampKeys.push_back(arena, k); - addWriteConflict = AddConflictRange::FALSE; + addWriteConflict = AddConflictRange::False; if (!options.readYourWritesDisabled) { writeRangeToNativeTransaction(range); writes.addUnmodifiedAndUnreadableRange(range); diff --git a/fdbclient/ReadYourWrites.h b/fdbclient/ReadYourWrites.h index c0bbb95f55..092a67793e 100644 --- a/fdbclient/ReadYourWrites.h +++ b/fdbclient/ReadYourWrites.h @@ -72,22 +72,22 @@ public: void setVersion(Version v) override { tr.setVersion(v); } Future getReadVersion() override; Optional getCachedReadVersion() const override { return tr.getCachedReadVersion(); } - Future> get(const Key& key, Snapshot = Snapshot::FALSE) override; - Future getKey(const KeySelector& key, Snapshot = Snapshot::FALSE) override; + Future> get(const Key& key, Snapshot = Snapshot::False) override; + Future getKey(const KeySelector& key, Snapshot = Snapshot::False) override; Future> getRange(const KeySelector& begin, const KeySelector& end, int limit, - Snapshot = Snapshot::FALSE, - Reverse = Reverse::FALSE) override; + Snapshot = Snapshot::False, + Reverse = Reverse::False) override; Future> getRange(KeySelector begin, KeySelector end, GetRangeLimits limits, - Snapshot = Snapshot::FALSE, - Reverse = Reverse::FALSE) override; + Snapshot = Snapshot::False, + Reverse = Reverse::False) override; Future> getRange(const KeyRange& keys, int limit, - Snapshot snapshot = Snapshot::FALSE, - Reverse reverse = Reverse::FALSE) { + Snapshot snapshot = Snapshot::False, + Reverse reverse = Reverse::False) { return getRange(KeySelector(firstGreaterOrEqual(keys.begin), keys.arena()), KeySelector(firstGreaterOrEqual(keys.end), keys.arena()), limit, @@ -96,8 +96,8 @@ public: } Future getRange(const KeyRange& keys, GetRangeLimits limits, - Snapshot snapshot = Snapshot::FALSE, - Reverse reverse = Reverse::FALSE) { + Snapshot snapshot = Snapshot::False, + Reverse reverse = Reverse::False) { return getRange(KeySelector(firstGreaterOrEqual(keys.begin), keys.arena()), KeySelector(firstGreaterOrEqual(keys.end), keys.arena()), limits, diff --git a/fdbclient/SimpleConfigTransaction.h b/fdbclient/SimpleConfigTransaction.h index a84ced8dbb..faecd2f8b0 100644 --- a/fdbclient/SimpleConfigTransaction.h +++ b/fdbclient/SimpleConfigTransaction.h @@ -49,17 +49,17 @@ public: Future getReadVersion() override; Optional getCachedReadVersion() const override; - Future> get(Key const& key, Snapshot = Snapshot::FALSE) override; + Future> get(Key const& key, Snapshot = Snapshot::False) override; Future> getRange(KeySelector const& begin, KeySelector const& end, int limit, - Snapshot = Snapshot::FALSE, - Reverse = Reverse::FALSE) override; + Snapshot = Snapshot::False, + Reverse = Reverse::False) override; Future> getRange(KeySelector begin, KeySelector end, GetRangeLimits limits, - Snapshot = Snapshot::FALSE, - Reverse = Reverse::FALSE) override; + Snapshot = Snapshot::False, + Reverse = Reverse::False) override; Future commit() override; Version getCommittedVersion() const override; void setOption(FDBTransactionOptions::Option option, Optional value = Optional()) override; diff --git a/fdbclient/SpecialKeySpace.actor.cpp b/fdbclient/SpecialKeySpace.actor.cpp index 11997ca5c5..fc6bf0b2fe 100644 --- a/fdbclient/SpecialKeySpace.actor.cpp +++ b/fdbclient/SpecialKeySpace.actor.cpp @@ -441,7 +441,7 @@ ACTOR Future> SpecialKeySpace::getActor(SpecialKeySpace* sks, KeySelector(firstGreaterOrEqual(key)), KeySelector(firstGreaterOrEqual(keyAfter(key))), GetRangeLimits(CLIENT_KNOBS->TOO_MANY), - Reverse::FALSE)); + Reverse::False)); ASSERT(result.size() <= 1); if (result.size()) { return Optional(result[0].value); diff --git a/fdbclient/SpecialKeySpace.actor.h b/fdbclient/SpecialKeySpace.actor.h index 88dc855769..ed7e6da46a 100644 --- a/fdbclient/SpecialKeySpace.actor.h +++ b/fdbclient/SpecialKeySpace.actor.h @@ -168,7 +168,7 @@ public: KeySelector begin, KeySelector end, GetRangeLimits limits, - Reverse = Reverse::FALSE); + Reverse = Reverse::False); void set(ReadYourWritesTransaction* ryw, const KeyRef& key, const ValueRef& value); diff --git a/fdbclient/TaskBucket.actor.cpp b/fdbclient/TaskBucket.actor.cpp index 759bf82842..f7e4ed7e24 100644 --- a/fdbclient/TaskBucket.actor.cpp +++ b/fdbclient/TaskBucket.actor.cpp @@ -173,14 +173,14 @@ public: { // Get a task key that is <= a random UID task key, if successful then return it - Key k = wait(tr->getKey(lastLessOrEqual(space.pack(uid)), Snapshot::TRUE)); + Key k = wait(tr->getKey(lastLessOrEqual(space.pack(uid)), Snapshot::True)); if (space.contains(k)) return Optional(k); } { // Get a task key that is <= the maximum possible UID, if successful return it. - Key k = wait(tr->getKey(lastLessOrEqual(space.pack(maxUIDKey)), Snapshot::TRUE)); + Key k = wait(tr->getKey(lastLessOrEqual(space.pack(maxUIDKey)), Snapshot::True)); if (space.contains(k)) return Optional(k); } @@ -395,7 +395,7 @@ public: taskBucket->setOptions(tr); // Attempt to extend the task's timeout - state Version newTimeout = wait(taskBucket->extendTimeout(tr, task, UpdateParams::FALSE)); + state Version newTimeout = wait(taskBucket->extendTimeout(tr, task, UpdateParams::False)); wait(tr->commit()); task->timeoutVersion = newTimeout; versionNow = tr->getCommittedVersion(); diff --git a/fdbclient/TaskBucket.h b/fdbclient/TaskBucket.h index 3158d0a1ae..e492f26226 100644 --- a/fdbclient/TaskBucket.h +++ b/fdbclient/TaskBucket.h @@ -140,9 +140,9 @@ class FutureBucket; class TaskBucket : public ReferenceCounted { public: TaskBucket(const Subspace& subspace, - AccessSystemKeys = AccessSystemKeys::FALSE, - PriorityBatch = PriorityBatch::FALSE, - LockAware = LockAware::FALSE); + AccessSystemKeys = AccessSystemKeys::False, + PriorityBatch = PriorityBatch::False, + LockAware = LockAware::False); virtual ~TaskBucket(); void setOptions(Reference tr) { @@ -311,7 +311,7 @@ class TaskFuture; class FutureBucket : public ReferenceCounted { public: - FutureBucket(const Subspace& subspace, AccessSystemKeys = AccessSystemKeys::FALSE, LockAware = LockAware::FALSE); + FutureBucket(const Subspace& subspace, AccessSystemKeys = AccessSystemKeys::False, LockAware = LockAware::False); virtual ~FutureBucket(); void setOptions(Reference tr) { diff --git a/fdbclient/ThreadSafeTransaction.cpp b/fdbclient/ThreadSafeTransaction.cpp index cc696ee553..47649747a9 100644 --- a/fdbclient/ThreadSafeTransaction.cpp +++ b/fdbclient/ThreadSafeTransaction.cpp @@ -122,7 +122,7 @@ ThreadSafeDatabase::ThreadSafeDatabase(std::string connFilename, int apiVersion) [db, connFile, apiVersion]() { try { Database::createDatabase( - Reference(connFile), apiVersion, IsInternal::FALSE, LocalityData(), db) + Reference(connFile), apiVersion, IsInternal::False, LocalityData(), db) .extractPtr(); } catch (Error& e) { new (db) DatabaseContext(e); diff --git a/fdbrpc/LoadBalance.actor.h b/fdbrpc/LoadBalance.actor.h index 7aeb491416..84a04caa70 100644 --- a/fdbrpc/LoadBalance.actor.h +++ b/fdbrpc/LoadBalance.actor.h @@ -419,7 +419,7 @@ struct RequestData : NonCopyable { Reference holderCapture = std::move(modelHolder); auto triedAllOptionsCapture = triedAllOptions; Future updateModel = map(response, [holderCapture, triedAllOptionsCapture](Reply result) { - checkAndProcessResultImpl(result, holderCapture, AtMostOnce::FALSE, triedAllOptionsCapture); + checkAndProcessResultImpl(result, holderCapture, AtMostOnce::False, triedAllOptionsCapture); return Void(); }); model->addActor.send(updateModel); @@ -447,7 +447,7 @@ Future loadBalance( Request request = Request(), TaskPriority taskID = TaskPriority::DefaultPromiseEndpoint, AtMostOnce atMostOnce = - AtMostOnce::FALSE, // if true, throws request_maybe_delivered() instead of retrying automatically + AtMostOnce::False, // if true, throws request_maybe_delivered() instead of retrying automatically QueueModel* model = nullptr) { state RequestData firstRequestData; @@ -459,7 +459,7 @@ Future loadBalance( state Promise requestFinished; state double startTime = now(); - state TriedAllOptions triedAllOptions = TriedAllOptions::FALSE; + state TriedAllOptions triedAllOptions = TriedAllOptions::False; setReplyPriority(request, taskID); if (!alternatives) @@ -602,7 +602,7 @@ Future loadBalance( break; nextAlt = (nextAlt + 1) % alternatives->size(); if (nextAlt == startAlt) - triedAllOptions = TriedAllOptions::TRUE; + triedAllOptions = TriedAllOptions::True; stream = nullptr; } @@ -709,7 +709,7 @@ Future loadBalance( nextAlt = (nextAlt + 1) % alternatives->size(); if (nextAlt == startAlt) - triedAllOptions = TriedAllOptions::TRUE; + triedAllOptions = TriedAllOptions::True; resetReply(request, taskID); secondDelay = Never(); } @@ -731,7 +731,7 @@ Future basicLoadBalance(Reference> al RequestStream Interface::*channel, Request request = Request(), TaskPriority taskID = TaskPriority::DefaultPromiseEndpoint, - AtMostOnce atMostOnce = AtMostOnce::FALSE) { + AtMostOnce atMostOnce = AtMostOnce::False) { setReplyPriority(request, taskID); if (!alternatives) return Never(); diff --git a/fdbserver/BackupWorker.actor.cpp b/fdbserver/BackupWorker.actor.cpp index 8a1f2c952a..ff783dafca 100644 --- a/fdbserver/BackupWorker.actor.cpp +++ b/fdbserver/BackupWorker.actor.cpp @@ -243,7 +243,7 @@ struct BackupData { 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)) { - cx = openDBOnServer(db, TaskPriority::DefaultEndpoint, LockAware::TRUE); + cx = openDBOnServer(db, TaskPriority::DefaultEndpoint, LockAware::True); specialCounter(cc, "SavedVersion", [this]() { return this->savedVersion; }); specialCounter(cc, "MinKnownCommittedVersion", [this]() { return this->minKnownCommittedVersion; }); diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index 8b0376c418..97ef92b1e5 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -133,9 +133,9 @@ public: serverInfo(new AsyncVar()), db(DatabaseContext::create(clientInfo, Future(), LocalityData(), - EnableLocalityLoadBalance::TRUE, + EnableLocalityLoadBalance::True, TaskPriority::DefaultEndpoint, - LockAware::TRUE)) // SOMEDAY: Locality! + LockAware::True)) // SOMEDAY: Locality! {} void setDistributor(const DataDistributorInterface& interf) { @@ -3078,7 +3078,7 @@ public: serverInfo.clusterInterface = ccInterface; serverInfo.myLocality = locality; db.serverInfo->set(serverInfo); - cx = openDBOnServer(db.serverInfo, TaskPriority::DefaultEndpoint, LockAware::TRUE); + cx = openDBOnServer(db.serverInfo, TaskPriority::DefaultEndpoint, LockAware::True); } ~ClusterControllerData() { diff --git a/fdbserver/ConfigDatabaseUnitTests.actor.cpp b/fdbserver/ConfigDatabaseUnitTests.actor.cpp index e315156da0..a99f4560cd 100644 --- a/fdbserver/ConfigDatabaseUnitTests.actor.cpp +++ b/fdbserver/ConfigDatabaseUnitTests.actor.cpp @@ -158,13 +158,13 @@ public: ReadFromLocalConfigEnvironment(std::string const& dataDir, std::string const& configPath, std::map const& manualKnobOverrides) - : dataDir(dataDir), localConfiguration(dataDir, configPath, manualKnobOverrides, IsTest::TRUE), + : dataDir(dataDir), localConfiguration(dataDir, configPath, manualKnobOverrides, IsTest::True), consumer(Never()) {} Future setup() { return setup(this); } Future restartLocalConfig(std::string const& newConfigPath) { - localConfiguration = LocalConfiguration(dataDir, newConfigPath, {}, IsTest::TRUE); + localConfiguration = LocalConfiguration(dataDir, newConfigPath, {}, IsTest::True); return setup(); } diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index 6b882bb1e1..175b1518a2 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -5781,7 +5781,7 @@ ACTOR Future dataDistribution(Reference self, state double lastLimited = 0; self->addActor.send(monitorBatchLimitedTime(self->dbInfo, &lastLimited)); - state Database cx = openDBOnServer(self->dbInfo, TaskPriority::DataDistributionLaunch, LockAware::TRUE); + state Database cx = openDBOnServer(self->dbInfo, TaskPriority::DataDistributionLaunch, LockAware::True); cx->locationCacheSize = SERVER_KNOBS->DD_LOCATION_CACHE_SIZE; // cx->setOption( FDBDatabaseOptions::LOCATION_CACHE_SIZE, StringRef((uint8_t*) @@ -6122,7 +6122,7 @@ static std::set const& normalDataDistributorErrors() { } ACTOR Future ddSnapCreateCore(DistributorSnapRequest snapReq, Reference> db) { - state Database cx = openDBOnServer(db, TaskPriority::DefaultDelay, LockAware::TRUE); + state Database cx = openDBOnServer(db, TaskPriority::DefaultDelay, LockAware::True); state ReadYourWritesTransaction tr(cx); loop { try { @@ -6463,7 +6463,7 @@ ACTOR Future dataDistributor(DataDistributorInterface di, Reference self(new DataDistributorData(db, di.id())); state Future collection = actorCollection(self->addActor.getFuture()); state PromiseStream getShardMetricsList; - state Database cx = openDBOnServer(db, TaskPriority::DefaultDelay, LockAware::TRUE); + state Database cx = openDBOnServer(db, TaskPriority::DefaultDelay, LockAware::True); state ActorCollection actors(false); state DDEnabledState ddEnabledState; self->addActor.send(actors.getResult()); @@ -6515,7 +6515,7 @@ std::unique_ptr testTeamCollection(int teamSize, Reference policy, int processCount) { Database database = DatabaseContext::create( - makeReference>(), Never(), LocalityData(), EnableLocalityLoadBalance::FALSE); + makeReference>(), Never(), LocalityData(), EnableLocalityLoadBalance::False); DatabaseConfiguration conf; conf.storageTeamSize = teamSize; @@ -6558,7 +6558,7 @@ std::unique_ptr testMachineTeamCollection(int teamSize, Reference policy, int processCount) { Database database = DatabaseContext::create( - makeReference>(), Never(), LocalityData(), EnableLocalityLoadBalance::FALSE); + makeReference>(), Never(), LocalityData(), EnableLocalityLoadBalance::False); DatabaseConfiguration conf; conf.storageTeamSize = teamSize; diff --git a/fdbserver/GrvProxyServer.actor.cpp b/fdbserver/GrvProxyServer.actor.cpp index 0a7614a52c..415ae9a310 100644 --- a/fdbserver/GrvProxyServer.actor.cpp +++ b/fdbserver/GrvProxyServer.actor.cpp @@ -253,7 +253,7 @@ struct GrvProxyData { RequestStream getConsistentReadVersion, Reference> db) : dbgid(dbgid), stats(dbgid), master(master), getConsistentReadVersion(getConsistentReadVersion), - cx(openDBOnServer(db, TaskPriority::DefaultEndpoint, LockAware::TRUE)), db(db), lastStartCommit(0), + cx(openDBOnServer(db, TaskPriority::DefaultEndpoint, LockAware::True)), db(db), lastStartCommit(0), lastCommitLatency(SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION), updateCommitRequests(0), lastCommitTime(0), minKnownCommittedVersion(invalidVersion) {} }; diff --git a/fdbserver/LocalConfiguration.actor.cpp b/fdbserver/LocalConfiguration.actor.cpp index 30974a2d19..b522ca1fca 100644 --- a/fdbserver/LocalConfiguration.actor.cpp +++ b/fdbserver/LocalConfiguration.actor.cpp @@ -230,11 +230,11 @@ class LocalConfigurationImpl { void updateInMemoryState(Version lastSeenVersion) { this->lastSeenVersion = lastSeenVersion; // TODO: Support randomization? - getKnobs().reset(Randomize::FALSE, g_network->isSimulated() ? IsSimulated::TRUE : IsSimulated::FALSE); + getKnobs().reset(Randomize::False, g_network->isSimulated() ? IsSimulated::True : IsSimulated::False); configKnobOverrides.update(getKnobs()); manualKnobOverrides.update(getKnobs()); // Must reinitialize in order to update dependent knobs - getKnobs().initialize(Randomize::FALSE, g_network->isSimulated() ? IsSimulated::TRUE : IsSimulated::FALSE); + getKnobs().initialize(Randomize::False, g_network->isSimulated() ? IsSimulated::True : IsSimulated::False); } ACTOR static Future setSnapshot(LocalConfigurationImpl* self, @@ -334,8 +334,8 @@ public: if (isTest) { testKnobCollection = IKnobCollection::create(IKnobCollection::Type::TEST, - Randomize::FALSE, - g_network->isSimulated() ? IsSimulated::TRUE : IsSimulated::FALSE); + Randomize::False, + g_network->isSimulated() ? IsSimulated::True : IsSimulated::False); } logger = traceCounters( "LocalConfigurationMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "LocalConfigurationMetrics"); @@ -405,7 +405,7 @@ public: configKnobOverrides.set( {}, "knob_name_that_does_not_exist"_sr, KnobValueRef::create(ParsedKnobValue(int{ 1 }))); auto testKnobCollection = - IKnobCollection::create(IKnobCollection::Type::TEST, Randomize::FALSE, IsSimulated::FALSE); + IKnobCollection::create(IKnobCollection::Type::TEST, Randomize::False, IsSimulated::False); // Should only trace and not throw an error: configKnobOverrides.update(*testKnobCollection); } @@ -414,7 +414,7 @@ public: ConfigKnobOverrides configKnobOverrides; configKnobOverrides.set({}, "test_int"_sr, KnobValueRef::create(ParsedKnobValue("not_an_int"))); auto testKnobCollection = - IKnobCollection::create(IKnobCollection::Type::TEST, Randomize::FALSE, IsSimulated::FALSE); + IKnobCollection::create(IKnobCollection::Type::TEST, Randomize::False, IsSimulated::False); // Should only trace and not throw an error: configKnobOverrides.update(*testKnobCollection); } diff --git a/fdbserver/LocalConfiguration.h b/fdbserver/LocalConfiguration.h index 6f9ecabc8f..b2f73641c3 100644 --- a/fdbserver/LocalConfiguration.h +++ b/fdbserver/LocalConfiguration.h @@ -51,7 +51,7 @@ public: LocalConfiguration(std::string const& dataFolder, std::string const& configPath, std::map const& manualKnobOverrides, - IsTest = IsTest::FALSE); + IsTest = IsTest::False); LocalConfiguration(LocalConfiguration&&); LocalConfiguration& operator=(LocalConfiguration&&); ~LocalConfiguration(); diff --git a/fdbserver/MetricLogger.actor.cpp b/fdbserver/MetricLogger.actor.cpp index 7c1ddf1b6f..aee9ea67d6 100644 --- a/fdbserver/MetricLogger.actor.cpp +++ b/fdbserver/MetricLogger.actor.cpp @@ -182,7 +182,7 @@ public: // levelKey is the prefix for the entire level, no timestamp at the end ACTOR static Future>> getLastBlock_impl(ReadYourWritesTransaction* tr, Standalone levelKey) { - RangeResult results = wait(tr->getRange(normalKeys.withPrefix(levelKey), 1, Snapshot::TRUE, Reverse::TRUE)); + RangeResult results = wait(tr->getRange(normalKeys.withPrefix(levelKey), 1, Snapshot::True, Reverse::True)); if (results.size() == 1) return results[0].value; return Optional>(); diff --git a/fdbserver/MoveKeys.actor.cpp b/fdbserver/MoveKeys.actor.cpp index a120433ac1..906bca4b16 100644 --- a/fdbserver/MoveKeys.actor.cpp +++ b/fdbserver/MoveKeys.actor.cpp @@ -1039,9 +1039,9 @@ ACTOR Future> addStorageServer(Database cx, StorageServe LocalityData::ExcludeLocalityPrefix.toString() + l.first + ":" + l.second)))); } - state Future fTags = tr->getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY, Snapshot::TRUE); + state Future fTags = tr->getRange(serverTagKeys, CLIENT_KNOBS->TOO_MANY, Snapshot::True); state Future fHistoryTags = - tr->getRange(serverTagHistoryKeys, CLIENT_KNOBS->TOO_MANY, Snapshot::TRUE); + tr->getRange(serverTagHistoryKeys, CLIENT_KNOBS->TOO_MANY, Snapshot::True); wait(success(fTagLocalities) && success(fv) && success(fTags) && success(fHistoryTags) && success(fExclProc) && success(fExclIP) && success(fFailProc) && success(fFailIP) && diff --git a/fdbserver/OldTLogServer_6_0.actor.cpp b/fdbserver/OldTLogServer_6_0.actor.cpp index 24c97f741c..0d503a0844 100644 --- a/fdbserver/OldTLogServer_6_0.actor.cpp +++ b/fdbserver/OldTLogServer_6_0.actor.cpp @@ -311,7 +311,7 @@ struct TLogData : NonCopyable { targetVolatileBytes(SERVER_KNOBS->TLOG_SPILL_THRESHOLD), overheadBytesInput(0), overheadBytesDurable(0), concurrentLogRouterReads(SERVER_KNOBS->CONCURRENT_LOG_ROUTER_READS), ignorePopRequest(false), ignorePopDeadline(), ignorePopUid(), dataFolder(folder), toBePopped() { - cx = openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::TRUE); + cx = openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::True); } }; diff --git a/fdbserver/OldTLogServer_6_2.actor.cpp b/fdbserver/OldTLogServer_6_2.actor.cpp index 68c125858f..aa88157fb4 100644 --- a/fdbserver/OldTLogServer_6_2.actor.cpp +++ b/fdbserver/OldTLogServer_6_2.actor.cpp @@ -375,7 +375,7 @@ struct TLogData : NonCopyable { peekMemoryLimiter(SERVER_KNOBS->TLOG_SPILL_REFERENCE_MAX_PEEK_MEMORY_BYTES), concurrentLogRouterReads(SERVER_KNOBS->CONCURRENT_LOG_ROUTER_READS), ignorePopRequest(false), ignorePopDeadline(), ignorePopUid(), dataFolder(folder), toBePopped() { - cx = openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::TRUE); + cx = openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::True); } }; @@ -1757,7 +1757,7 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen state std::vector>> messageReads; messageReads.reserve(commitLocations.size()); for (const auto& pair : commitLocations) { - messageReads.push_back(self->rawPersistentQueue->read(pair.first, pair.second, CheckHashes::TRUE)); + messageReads.push_back(self->rawPersistentQueue->read(pair.first, pair.second, CheckHashes::True)); } commitLocations.clear(); wait(waitForAll(messageReads)); diff --git a/fdbserver/ProxyCommitData.actor.h b/fdbserver/ProxyCommitData.actor.h index 7a2960022e..43cff6bd47 100644 --- a/fdbserver/ProxyCommitData.actor.h +++ b/fdbserver/ProxyCommitData.actor.h @@ -247,7 +247,7 @@ struct ProxyCommitData { mostRecentProcessedRequestNumber(0), getConsistentReadVersion(getConsistentReadVersion), commit(commit), lastCoalesceTime(0), localCommitBatchesStarted(0), locked(false), commitBatchInterval(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_INTERVAL_MIN), firstProxy(firstProxy), - cx(openDBOnServer(db, TaskPriority::DefaultEndpoint, LockAware::TRUE)), db(db), + cx(openDBOnServer(db, TaskPriority::DefaultEndpoint, LockAware::True)), db(db), singleKeyMutationEvent(LiteralStringRef("SingleKeyMutation")), commitBatchesMemBytesCount(0), lastTxsPop(0), lastStartCommit(0), lastCommitLatency(SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION), lastCommitTime(0), lastMasterReset(now()), lastResolverReset(now()) { diff --git a/fdbserver/QuietDatabase.actor.cpp b/fdbserver/QuietDatabase.actor.cpp index 47b9a9f2f3..7980c382ef 100644 --- a/fdbserver/QuietDatabase.actor.cpp +++ b/fdbserver/QuietDatabase.actor.cpp @@ -637,7 +637,7 @@ ACTOR Future waitForQuietDatabase(Database cx, // The quiet database check (which runs at the end of every test) will always time out due to active data movement. // To get around this, quiet Database will disable the perpetual wiggle in the setup phase. - wait(setPerpetualStorageWiggle(cx, false, LockAware::TRUE)); + wait(setPerpetualStorageWiggle(cx, false, LockAware::True)); // Require 3 consecutive successful quiet database checks spaced 2 second apart state int numSuccesses = 0; diff --git a/fdbserver/Ratekeeper.actor.cpp b/fdbserver/Ratekeeper.actor.cpp index 77bda2577b..a49c4f476d 100644 --- a/fdbserver/Ratekeeper.actor.cpp +++ b/fdbserver/Ratekeeper.actor.cpp @@ -1409,7 +1409,7 @@ ACTOR Future configurationMonitor(RatekeeperData* self) { } ACTOR Future ratekeeper(RatekeeperInterface rkInterf, Reference> dbInfo) { - state RatekeeperData self(rkInterf.id(), openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::TRUE)); + state RatekeeperData self(rkInterf.id(), openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::True)); state Future timeout = Void(); state std::vector> tlogTrackers; state std::vector tlogInterfs; diff --git a/fdbserver/RestoreCommon.actor.cpp b/fdbserver/RestoreCommon.actor.cpp index ace015f24b..31e60edc21 100644 --- a/fdbserver/RestoreCommon.actor.cpp +++ b/fdbserver/RestoreCommon.actor.cpp @@ -141,8 +141,8 @@ Key RestoreConfigFR::applyMutationsMapPrefix() { ACTOR Future RestoreConfigFR::getApplyVersionLag_impl(Reference tr, UID uid) { // Both of these are snapshot reads - state Future> beginVal = tr->get(uidPrefixKey(applyMutationsBeginRange.begin, uid), Snapshot::TRUE); - state Future> endVal = tr->get(uidPrefixKey(applyMutationsEndRange.begin, uid), Snapshot::TRUE); + state Future> beginVal = tr->get(uidPrefixKey(applyMutationsBeginRange.begin, uid), Snapshot::True); + state Future> endVal = tr->get(uidPrefixKey(applyMutationsEndRange.begin, uid), Snapshot::True); wait(success(beginVal) && success(endVal)); if (!beginVal.get().present() || !endVal.get().present()) diff --git a/fdbserver/RestoreWorker.actor.cpp b/fdbserver/RestoreWorker.actor.cpp index 827a58a25d..0e2e6a8be6 100644 --- a/fdbserver/RestoreWorker.actor.cpp +++ b/fdbserver/RestoreWorker.actor.cpp @@ -410,7 +410,7 @@ ACTOR Future restoreWorker(Reference connFile, LocalityData locality, std::string coordFolder) { try { - Database cx = Database::createDatabase(connFile, Database::API_VERSION_LATEST, IsInternal::TRUE, locality); + Database cx = Database::createDatabase(connFile, Database::API_VERSION_LATEST, IsInternal::True, locality); wait(reportErrors(_restoreWorker(cx, locality), "RestoreWorker")); } catch (Error& e) { TraceEvent("FastRestoreWorker").detail("Error", e.what()); diff --git a/fdbserver/StorageCache.actor.cpp b/fdbserver/StorageCache.actor.cpp index 888c94c3b3..3344636683 100644 --- a/fdbserver/StorageCache.actor.cpp +++ b/fdbserver/StorageCache.actor.cpp @@ -251,7 +251,7 @@ public: newestAvailableVersion.insert(allKeys, invalidVersion); newestDirtyVersion.insert(allKeys, invalidVersion); addCacheRange(CacheRangeInfo::newNotAssigned(allKeys)); - cx = openDBOnServer(db, TaskPriority::DefaultEndpoint, LockAware::TRUE); + cx = openDBOnServer(db, TaskPriority::DefaultEndpoint, LockAware::True); } // Puts the given cacheRange into cachedRangeMap. The caller is responsible for adding cacheRanges @@ -1194,7 +1194,7 @@ ACTOR Future tryFetchRange(Database cx, try { loop { - RangeResult rep = wait(tr.getRange(begin, end, limits, Snapshot::TRUE)); + RangeResult rep = wait(tr.getRange(begin, end, limits, Snapshot::True)); limits.decrement(rep); if (limits.isReached() || !rep.more) { @@ -1392,7 +1392,7 @@ ACTOR Future fetchKeys(StorageCacheData* data, AddingCacheRange* cacheRang // TODO: NEELAM: what's this for? // FIXME: remove when we no longer support upgrades from 5.X if (debug_getRangeRetries >= 100) { - data->cx->enableLocalityLoadBalance = EnableLocalityLoadBalance::FALSE; + data->cx->enableLocalityLoadBalance = EnableLocalityLoadBalance::False; } debug_getRangeRetries++; diff --git a/fdbserver/TLogServer.actor.cpp b/fdbserver/TLogServer.actor.cpp index a948ecefb2..730c4c9070 100644 --- a/fdbserver/TLogServer.actor.cpp +++ b/fdbserver/TLogServer.actor.cpp @@ -386,7 +386,7 @@ struct TLogData : NonCopyable { commitLatencyDist(Histogram::getHistogram(LiteralStringRef("tLog"), LiteralStringRef("commit"), Histogram::Unit::microseconds)) { - cx = openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::TRUE); + cx = openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::True); } }; @@ -1798,7 +1798,7 @@ ACTOR Future tLogPeekMessages(TLogData* self, TLogPeekRequest req, Referen state std::vector>> messageReads; messageReads.reserve(commitLocations.size()); for (const auto& pair : commitLocations) { - messageReads.push_back(self->rawPersistentQueue->read(pair.first, pair.second, CheckHashes::TRUE)); + messageReads.push_back(self->rawPersistentQueue->read(pair.first, pair.second, CheckHashes::True)); } commitLocations.clear(); wait(waitForAll(messageReads)); diff --git a/fdbserver/WorkerInterface.actor.h b/fdbserver/WorkerInterface.actor.h index e642d015dd..682801a90c 100644 --- a/fdbserver/WorkerInterface.actor.h +++ b/fdbserver/WorkerInterface.actor.h @@ -833,8 +833,8 @@ struct ServerDBInfo; class Database openDBOnServer(Reference> const& db, TaskPriority taskID = TaskPriority::DefaultEndpoint, - LockAware = LockAware::FALSE, - EnableLocalityLoadBalance = EnableLocalityLoadBalance::TRUE); + LockAware = LockAware::False, + EnableLocalityLoadBalance = EnableLocalityLoadBalance::True); ACTOR Future extractClusterInterface(Reference>> a, Reference>> b); diff --git a/fdbserver/fdbserver.actor.cpp b/fdbserver/fdbserver.actor.cpp index 0a62a0b2fd..1589852b1f 100644 --- a/fdbserver/fdbserver.actor.cpp +++ b/fdbserver/fdbserver.actor.cpp @@ -1640,9 +1640,9 @@ int main(int argc, char* argv[]) { enableBuggify(opts.buggifyEnabled, BuggifyType::General); IKnobCollection::setGlobalKnobCollection(IKnobCollection::Type::SERVER, - Randomize::TRUE, - role == ServerRole::Simulation ? IsSimulated::TRUE - : IsSimulated::FALSE); + Randomize::True, + role == ServerRole::Simulation ? IsSimulated::True + : IsSimulated::False); IKnobCollection::getMutableGlobalKnobCollection().setKnob("log_directory", KnobValue::create(opts.logFolder)); if (role != ServerRole::Simulation) { IKnobCollection::getMutableGlobalKnobCollection().setKnob("commit_batches_mem_bytes_hard_limit", @@ -1677,7 +1677,7 @@ int main(int argc, char* argv[]) { KnobValue::create(int64_t{ opts.memLimit })); // Reinitialize knobs in order to update knobs that are dependent on explicitly set knobs IKnobCollection::getMutableGlobalKnobCollection().initialize( - Randomize::TRUE, role == ServerRole::Simulation ? IsSimulated::TRUE : IsSimulated::FALSE); + Randomize::True, role == ServerRole::Simulation ? IsSimulated::True : IsSimulated::False); // evictionPolicyStringToEnum will throw an exception if the string is not recognized as a valid EvictablePageCache::evictionPolicyStringToEnum(FLOW_KNOBS->CACHE_EVICTION_POLICY); diff --git a/fdbserver/masterserver.actor.cpp b/fdbserver/masterserver.actor.cpp index ea93b35f7f..167491efb5 100644 --- a/fdbserver/masterserver.actor.cpp +++ b/fdbserver/masterserver.actor.cpp @@ -577,7 +577,7 @@ Future sendMasterRegistration(MasterData* self, } ACTOR Future updateRegistration(Reference self, Reference logSystem) { - state Database cx = openDBOnServer(self->dbInfo, TaskPriority::DefaultEndpoint, LockAware::TRUE); + state Database cx = openDBOnServer(self->dbInfo, TaskPriority::DefaultEndpoint, LockAware::True); state Future trigger = self->registrationTrigger.onTrigger(); state Future updateLogsKey; @@ -1965,7 +1965,7 @@ ACTOR Future masterCore(Reference self) { self->addActor.send(resolutionBalancing(self)); self->addActor.send(changeCoordinators(self)); - Database cx = openDBOnServer(self->dbInfo, TaskPriority::DefaultEndpoint, LockAware::TRUE); + Database cx = openDBOnServer(self->dbInfo, TaskPriority::DefaultEndpoint, LockAware::True); self->addActor.send(configurationMonitor(self, cx)); if (self->configuration.backupWorkerEnabled) { self->addActor.send(recruitBackupWorkers(self, cx)); diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 6008251127..3f454e79f4 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -852,7 +852,7 @@ public: newestDirtyVersion.insert(allKeys, invalidVersion); addShard(ShardInfo::newNotAssigned(allKeys)); - cx = openDBOnServer(db, TaskPriority::DefaultEndpoint, LockAware::TRUE); + cx = openDBOnServer(db, TaskPriority::DefaultEndpoint, LockAware::True); } //~StorageServer() { fclose(log); } @@ -2790,7 +2790,7 @@ ACTOR Future tryGetRange(PromiseStream results, Transaction* loop { GetRangeLimits limits(GetRangeLimits::ROW_LIMIT_UNLIMITED, SERVER_KNOBS->FETCH_BLOCK_BYTES); limits.minRows = 0; - state RangeResult rep = wait(tr->getRange(begin, end, limits, Snapshot::TRUE)); + state RangeResult rep = wait(tr->getRange(begin, end, limits, Snapshot::True)); if (!rep.more) { rep.readThrough = keys.end; } @@ -2903,7 +2903,7 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { tr.info.taskID = TaskPriority::FetchKeys; state PromiseStream results; state Future hold = SERVER_KNOBS->FETCH_USING_STREAMING - ? tr.getRangeStream(results, keys, GetRangeLimits(), Snapshot::TRUE) + ? tr.getRangeStream(results, keys, GetRangeLimits(), Snapshot::True) : tryGetRange(results, &tr, keys); state Key nfk = keys.begin; @@ -2970,7 +2970,7 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { // FIXME: remove when we no longer support upgrades from 5.X if (debug_getRangeRetries >= 100) { - data->cx->enableLocalityLoadBalance = EnableLocalityLoadBalance::FALSE; + data->cx->enableLocalityLoadBalance = EnableLocalityLoadBalance::False; TraceEvent(SevWarnAlways, "FKDisableLB").detail("FKID", fetchKeysID); } @@ -3018,7 +3018,7 @@ ACTOR Future fetchKeys(StorageServer* data, AddingShard* shard) { } // FIXME: remove when we no longer support upgrades from 5.X - data->cx->enableLocalityLoadBalance = EnableLocalityLoadBalance::TRUE; + data->cx->enableLocalityLoadBalance = EnableLocalityLoadBalance::True; TraceEvent(SevWarnAlways, "FKReenableLB").detail("FKID", fetchKeysID); // We have completed the fetch and write of the data, now we wait for MVCC window to pass. diff --git a/fdbserver/tester.actor.cpp b/fdbserver/tester.actor.cpp index 3dcd9ae162..44826b4590 100644 --- a/fdbserver/tester.actor.cpp +++ b/fdbserver/tester.actor.cpp @@ -616,7 +616,7 @@ ACTOR Future testerServerWorkload(WorkloadRequest work, startRole(Role::TESTER, workIface.id(), UID(), details); if (work.useDatabase) { - cx = Database::createDatabase(ccf, -1, IsInternal::TRUE, locality); + cx = Database::createDatabase(ccf, -1, IsInternal::True, locality); wait(delay(1.0)); } @@ -1481,7 +1481,7 @@ ACTOR Future runTests(Reference workerServer(Reference connFile, if (metricsConnFile.size() > 0) { try { state Database db = - Database::createDatabase(metricsConnFile, Database::API_VERSION_LATEST, IsInternal::TRUE, locality); + Database::createDatabase(metricsConnFile, Database::API_VERSION_LATEST, IsInternal::True, locality); metricsLogger = runMetrics(db, KeyRef(metricsPrefix)); } catch (Error& e) { TraceEvent(SevWarnAlways, "TDMetricsBadClusterFile").error(e).detail("ConnFile", metricsConnFile); } } else { - auto lockAware = metricsPrefix.size() && metricsPrefix[0] == '\xff' ? LockAware::TRUE : LockAware::FALSE; + auto lockAware = metricsPrefix.size() && metricsPrefix[0] == '\xff' ? LockAware::True : LockAware::False; metricsLogger = runMetrics(openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, lockAware), KeyRef(metricsPrefix)); } diff --git a/fdbserver/workloads/ApiWorkload.actor.cpp b/fdbserver/workloads/ApiWorkload.actor.cpp index 7d223462c9..5b9b0fc100 100644 --- a/fdbserver/workloads/ApiWorkload.actor.cpp +++ b/fdbserver/workloads/ApiWorkload.actor.cpp @@ -168,14 +168,14 @@ ACTOR Future compareDatabaseToMemory(ApiWorkload* self) { loop { // Fetch a subset of the results from each of the database and the memory store and compare them state RangeResult storeResults = - self->store.getRange(KeyRangeRef(startKey, endKey), resultsPerRange, Reverse::FALSE); + self->store.getRange(KeyRangeRef(startKey, endKey), resultsPerRange, Reverse::False); state Reference transaction = self->createTransaction(); state KeyRangeRef range(startKey, endKey); loop { try { - state RangeResult dbResults = wait(transaction->getRange(range, resultsPerRange, Reverse::FALSE)); + state RangeResult dbResults = wait(transaction->getRange(range, resultsPerRange, Reverse::False)); // Compare results of database and memory store Version v = wait(transaction->getReadVersion()); diff --git a/fdbserver/workloads/ApiWorkload.h b/fdbserver/workloads/ApiWorkload.h index 53e2eed431..152ac72205 100644 --- a/fdbserver/workloads/ApiWorkload.h +++ b/fdbserver/workloads/ApiWorkload.h @@ -102,12 +102,12 @@ struct FlowTransactionWrapper : public TransactionWrapper { // Gets a range of key-value pairs from the database specified by a key range Future getRange(KeyRangeRef& keys, int limit, Reverse reverse) override { - return transaction.getRange(keys, limit, Snapshot::FALSE, reverse); + return transaction.getRange(keys, limit, Snapshot::False, reverse); } // Gets a range of key-value pairs from the database specified by a pair of key selectors Future getRange(KeySelectorRef& begin, KeySelectorRef& end, int limit, Reverse reverse) override { - return transaction.getRange(begin, end, limit, Snapshot::FALSE, reverse); + return transaction.getRange(begin, end, limit, Snapshot::False, reverse); } // Gets the key from the database specified by a given key selector @@ -162,12 +162,12 @@ struct ThreadTransactionWrapper : public TransactionWrapper { // Gets a range of key-value pairs from the database specified by a key range Future getRange(KeyRangeRef& keys, int limit, Reverse reverse) override { - return unsafeThreadFutureToFuture(transaction->getRange(keys, limit, Snapshot::FALSE, reverse)); + return unsafeThreadFutureToFuture(transaction->getRange(keys, limit, Snapshot::False, reverse)); } // Gets a range of key-value pairs from the database specified by a pair of key selectors Future getRange(KeySelectorRef& begin, KeySelectorRef& end, int limit, Reverse reverse) override { - return unsafeThreadFutureToFuture(transaction->getRange(begin, end, limit, Snapshot::FALSE, reverse)); + return unsafeThreadFutureToFuture(transaction->getRange(begin, end, limit, Snapshot::False, reverse)); } // Gets the key from the database specified by a given key selector diff --git a/fdbserver/workloads/AtomicRestore.actor.cpp b/fdbserver/workloads/AtomicRestore.actor.cpp index a4100e9e9a..66e35105ab 100644 --- a/fdbserver/workloads/AtomicRestore.actor.cpp +++ b/fdbserver/workloads/AtomicRestore.actor.cpp @@ -97,7 +97,7 @@ struct AtomicRestoreWorkload : TestWorkload { deterministicRandom()->randomInt(0, 100), BackupAgentBase::getDefaultTagName(), self->backupRanges, - StopWhenDone::FALSE, + StopWhenDone::False, self->usePartitionedLogs)); } catch (Error& e) { if (e.code() != error_code_backup_unneeded && e.code() != error_code_backup_duplicate) @@ -105,7 +105,7 @@ struct AtomicRestoreWorkload : TestWorkload { } TraceEvent("AtomicRestore_Wait"); - wait(success(backupAgent.waitBackup(cx, BackupAgentBase::getDefaultTagName(), StopWhenDone::FALSE))); + wait(success(backupAgent.waitBackup(cx, BackupAgentBase::getDefaultTagName(), StopWhenDone::False))); TraceEvent("AtomicRestore_BackupStart"); wait(delay(self->restoreAfter * deterministicRandom()->random01())); TraceEvent("AtomicRestore_RestoreStart"); diff --git a/fdbserver/workloads/AtomicSwitchover.actor.cpp b/fdbserver/workloads/AtomicSwitchover.actor.cpp index d672759817..3bb93fb8bf 100644 --- a/fdbserver/workloads/AtomicSwitchover.actor.cpp +++ b/fdbserver/workloads/AtomicSwitchover.actor.cpp @@ -57,10 +57,10 @@ struct AtomicSwitchoverWorkload : TestWorkload { wait(backupAgent.submitBackup(self->extraDB, BackupAgentBase::getDefaultTag(), self->backupRanges, - StopWhenDone::FALSE, + StopWhenDone::False, StringRef(), StringRef(), - LockDB::TRUE)); + LockDB::True)); TraceEvent("AS_Submit2"); } catch (Error& e) { if (e.code() != error_code_backup_duplicate) @@ -168,21 +168,21 @@ struct AtomicSwitchoverWorkload : TestWorkload { state DatabaseBackupAgent restoreTool(self->extraDB); TraceEvent("AS_Wait1"); - wait(success(backupAgent.waitBackup(self->extraDB, BackupAgentBase::getDefaultTag(), StopWhenDone::FALSE))); + wait(success(backupAgent.waitBackup(self->extraDB, BackupAgentBase::getDefaultTag(), StopWhenDone::False))); TraceEvent("AS_Ready1"); wait(delay(deterministicRandom()->random01() * self->switch1delay)); TraceEvent("AS_Switch1"); wait(backupAgent.atomicSwitchover( self->extraDB, BackupAgentBase::getDefaultTag(), self->backupRanges, StringRef(), StringRef())); TraceEvent("AS_Wait2"); - wait(success(restoreTool.waitBackup(cx, BackupAgentBase::getDefaultTag(), StopWhenDone::FALSE))); + wait(success(restoreTool.waitBackup(cx, BackupAgentBase::getDefaultTag(), StopWhenDone::False))); TraceEvent("AS_Ready2"); wait(delay(deterministicRandom()->random01() * self->switch2delay)); TraceEvent("AS_Switch2"); wait(restoreTool.atomicSwitchover( cx, BackupAgentBase::getDefaultTag(), self->backupRanges, StringRef(), StringRef())); TraceEvent("AS_Wait3"); - wait(success(backupAgent.waitBackup(self->extraDB, BackupAgentBase::getDefaultTag(), StopWhenDone::FALSE))); + wait(success(backupAgent.waitBackup(self->extraDB, BackupAgentBase::getDefaultTag(), StopWhenDone::False))); TraceEvent("AS_Ready3"); wait(delay(deterministicRandom()->random01() * self->stopDelay)); TraceEvent("AS_Abort"); diff --git a/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp b/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp index 1f636bad6b..b465e49939 100644 --- a/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp +++ b/fdbserver/workloads/BackupAndParallelRestoreCorrectness.actor.cpp @@ -179,7 +179,7 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload { ACTOR static Future statusLoop(Database cx, std::string tag) { state FileBackupAgent agent; loop { - std::string status = wait(agent.getStatus(cx, ShowErrors::TRUE, tag)); + std::string status = wait(agent.getStatus(cx, ShowErrors::True, tag)); puts(status.c_str()); wait(delay(2.0)); } @@ -252,7 +252,7 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload { state Reference lastBackupContainer; state UID lastBackupUID; state EBackupState resultWait = wait(backupAgent->waitBackup( - cx, backupTag.tagName, StopWhenDone::FALSE, &lastBackupContainer, &lastBackupUID)); + cx, backupTag.tagName, StopWhenDone::False, &lastBackupContainer, &lastBackupUID)); TraceEvent("BARW_DoBackupWaitForRestorable", randomID) .detail("Tag", backupTag.tagName) @@ -333,11 +333,11 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload { // Wait for the backup to complete TraceEvent("BARW_DoBackupWaitBackup", randomID).detail("Tag", printable(tag)); - state EBackupState statusValue = wait(backupAgent->waitBackup(cx, tag.toString(), StopWhenDone::TRUE)); + state EBackupState statusValue = wait(backupAgent->waitBackup(cx, tag.toString(), StopWhenDone::True)); state std::string statusText; - std::string _statusText = wait(backupAgent->getStatus(cx, ShowErrors::TRUE, tag.toString())); + std::string _statusText = wait(backupAgent->getStatus(cx, ShowErrors::True, tag.toString())); statusText = _statusText; // Can we validate anything about status? @@ -377,9 +377,9 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload { cx, self->backupTag, KeyRef(lastBackupContainer), - WaitForComplete::TRUE, + WaitForComplete::True, ::invalidVersion, - Verbose::TRUE, + Verbose::True, normalKeys, Key(), Key(), @@ -482,8 +482,8 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload { deterministicRandom()->randomInt(0, 100), self->backupTag.toString(), self->backupRanges, - StopWhenDone::TRUE, - UsePartitionedLog::FALSE); + StopWhenDone::True, + UsePartitionedLog::False); } catch (Error& e) { TraceEvent("BARW_SubmitBackup2Exception", randomID) .error(e) diff --git a/fdbserver/workloads/BackupCorrectness.actor.cpp b/fdbserver/workloads/BackupCorrectness.actor.cpp index e629f37085..d655342661 100644 --- a/fdbserver/workloads/BackupCorrectness.actor.cpp +++ b/fdbserver/workloads/BackupCorrectness.actor.cpp @@ -220,7 +220,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { ACTOR static Future statusLoop(Database cx, std::string tag) { state FileBackupAgent agent; loop { - std::string status = wait(agent.getStatus(cx, ShowErrors::TRUE, tag)); + std::string status = wait(agent.getStatus(cx, ShowErrors::True, tag)); puts(status.c_str()); std::string statusJSON = wait(agent.getStatusJSON(cx, tag)); puts(statusJSON.c_str()); @@ -271,8 +271,8 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { tag.toString(), backupRanges, StopWhenDone{ !stopDifferentialDelay }, - UsePartitionedLog::FALSE, - IncrementalBackupOnly::FALSE, + UsePartitionedLog::False, + IncrementalBackupOnly::False, self->encryptionKeyFileName)); } catch (Error& e) { TraceEvent("BARW_DoBackupSubmitBackupException", randomID).error(e).detail("Tag", printable(tag)); @@ -299,7 +299,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { state Reference lastBackupContainer; state UID lastBackupUID; state EBackupState resultWait = wait(backupAgent->waitBackup( - cx, backupTag.tagName, StopWhenDone::FALSE, &lastBackupContainer, &lastBackupUID)); + cx, backupTag.tagName, StopWhenDone::False, &lastBackupContainer, &lastBackupUID)); TraceEvent("BARW_DoBackupWaitForRestorable", randomID) .detail("Tag", backupTag.tagName) @@ -380,11 +380,11 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { // Wait for the backup to complete TraceEvent("BARW_DoBackupWaitBackup", randomID).detail("Tag", printable(tag)); - state EBackupState statusValue = wait(backupAgent->waitBackup(cx, tag.toString(), StopWhenDone::TRUE)); + state EBackupState statusValue = wait(backupAgent->waitBackup(cx, tag.toString(), StopWhenDone::True)); state std::string statusText; - std::string _statusText = wait(backupAgent->getStatus(cx, ShowErrors::TRUE, tag.toString())); + std::string _statusText = wait(backupAgent->getStatus(cx, ShowErrors::True, tag.toString())); statusText = _statusText; // Can we validate anything about status? @@ -423,9 +423,9 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { cx, self->backupTag, KeyRef(lastBackupContainer), - WaitForComplete::TRUE, + WaitForComplete::True, ::invalidVersion, - Verbose::TRUE, + Verbose::True, normalKeys, Key(), Key(), @@ -527,7 +527,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { deterministicRandom()->randomInt(0, 100), self->backupTag.toString(), self->backupRanges, - StopWhenDone::TRUE); + StopWhenDone::True); } catch (Error& e) { TraceEvent("BARW_SubmitBackup2Exception", randomID) .error(e) @@ -593,15 +593,15 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { cx, restoreTag, KeyRef(lastBackupContainer->getURL()), - WaitForComplete::TRUE, + WaitForComplete::True, targetVersion, - Verbose::TRUE, + Verbose::True, range, Key(), Key(), self->locked, - OnlyApplyMutationLogs::FALSE, - InconsistentSnapshotOnly::FALSE, + OnlyApplyMutationLogs::False, + InconsistentSnapshotOnly::False, ::invalidVersion, self->encryptionKeyFileName)); } @@ -617,14 +617,14 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { restoreTag, KeyRef(lastBackupContainer->getURL()), self->restoreRanges, - WaitForComplete::TRUE, + WaitForComplete::True, targetVersion, - Verbose::TRUE, + Verbose::True, Key(), Key(), self->locked, - OnlyApplyMutationLogs::FALSE, - InconsistentSnapshotOnly::FALSE, + OnlyApplyMutationLogs::False, + InconsistentSnapshotOnly::False, ::invalidVersion, self->encryptionKeyFileName)); } @@ -647,14 +647,14 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { restoreTags[restoreIndex], KeyRef(lastBackupContainer->getURL()), self->restoreRanges, - WaitForComplete::TRUE, + WaitForComplete::True, ::invalidVersion, - Verbose::TRUE, + Verbose::True, Key(), Key(), self->locked, - OnlyApplyMutationLogs::FALSE, - InconsistentSnapshotOnly::FALSE, + OnlyApplyMutationLogs::False, + InconsistentSnapshotOnly::False, ::invalidVersion, self->encryptionKeyFileName); } @@ -675,15 +675,15 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload { cx, restoreTags[restoreIndex], KeyRef(lastBackupContainer->getURL()), - WaitForComplete::TRUE, + WaitForComplete::True, ::invalidVersion, - Verbose::TRUE, + Verbose::True, self->restoreRanges[restoreIndex], Key(), Key(), self->locked, - OnlyApplyMutationLogs::FALSE, - InconsistentSnapshotOnly::FALSE, + OnlyApplyMutationLogs::False, + InconsistentSnapshotOnly::False, ::invalidVersion, self->encryptionKeyFileName); } diff --git a/fdbserver/workloads/BackupToBlob.actor.cpp b/fdbserver/workloads/BackupToBlob.actor.cpp index 44cabf8549..bf5bc0db42 100644 --- a/fdbserver/workloads/BackupToBlob.actor.cpp +++ b/fdbserver/workloads/BackupToBlob.actor.cpp @@ -66,7 +66,7 @@ struct BackupToBlobWorkload : TestWorkload { self->snapshotInterval, self->backupTag.toString(), backupRanges)); - EBackupState backupStatus = wait(backupAgent.waitBackup(cx, self->backupTag.toString(), StopWhenDone::TRUE)); + EBackupState backupStatus = wait(backupAgent.waitBackup(cx, self->backupTag.toString(), StopWhenDone::True)); TraceEvent("BackupToBlob_BackupStatus").detail("Status", BackupAgentBase::getStateText(backupStatus)); return Void(); } diff --git a/fdbserver/workloads/BackupToDBAbort.actor.cpp b/fdbserver/workloads/BackupToDBAbort.actor.cpp index c83b38e451..d2d5c3d02d 100644 --- a/fdbserver/workloads/BackupToDBAbort.actor.cpp +++ b/fdbserver/workloads/BackupToDBAbort.actor.cpp @@ -56,10 +56,10 @@ struct BackupToDBAbort : TestWorkload { wait(backupAgent.submitBackup(self->extraDB, BackupAgentBase::getDefaultTag(), self->backupRanges, - StopWhenDone::FALSE, + StopWhenDone::False, StringRef(), StringRef(), - LockDB::TRUE)); + LockDB::True)); TraceEvent("BDBA_Submit2"); } catch (Error& e) { if (e.code() != error_code_backup_duplicate) @@ -80,7 +80,7 @@ struct BackupToDBAbort : TestWorkload { TraceEvent("BDBA_Start").detail("Delay", self->abortDelay); wait(delay(self->abortDelay)); TraceEvent("BDBA_Wait"); - wait(success(backupAgent.waitBackup(self->extraDB, BackupAgentBase::getDefaultTag(), StopWhenDone::FALSE))); + wait(success(backupAgent.waitBackup(self->extraDB, BackupAgentBase::getDefaultTag(), StopWhenDone::False))); TraceEvent("BDBA_Lock"); wait(lockDatabase(cx, self->lockid)); TraceEvent("BDBA_Abort"); diff --git a/fdbserver/workloads/BackupToDBCorrectness.actor.cpp b/fdbserver/workloads/BackupToDBCorrectness.actor.cpp index 385b492a87..138431568e 100644 --- a/fdbserver/workloads/BackupToDBCorrectness.actor.cpp +++ b/fdbserver/workloads/BackupToDBCorrectness.actor.cpp @@ -337,7 +337,7 @@ struct BackupToDBCorrectnessWorkload : TestWorkload { if (BUGGIFY) { TraceEvent("BARW_DoBackupWaitForRestorable", randomID).detail("Tag", printable(tag)); // Wait until the backup is in a restorable state - state EBackupState resultWait = wait(backupAgent->waitBackup(cx, tag, StopWhenDone::FALSE)); + state EBackupState resultWait = wait(backupAgent->waitBackup(cx, tag, StopWhenDone::False)); TraceEvent("BARW_LastBackupFolder", randomID) .detail("BackupTag", printable(tag)) @@ -383,7 +383,7 @@ struct BackupToDBCorrectnessWorkload : TestWorkload { UID _destUid = wait(backupAgent->getDestUid(cx, logUid)); self->destUid = _destUid; - state EBackupState statusValue = wait(backupAgent->waitBackup(cx, tag, StopWhenDone::TRUE)); + state EBackupState statusValue = wait(backupAgent->waitBackup(cx, tag, StopWhenDone::True)); wait(backupAgent->unlockBackup(cx, tag)); state std::string statusText; @@ -617,7 +617,7 @@ struct BackupToDBCorrectnessWorkload : TestWorkload { extraBackup = backupAgent.submitBackup(self->extraDB, self->backupTag, self->backupRanges, - StopWhenDone::TRUE, + StopWhenDone::True, self->extraPrefix, StringRef(), self->locked, @@ -652,7 +652,7 @@ struct BackupToDBCorrectnessWorkload : TestWorkload { wait(restoreTool.submitBackup(cx, self->restoreTag, restoreRange, - StopWhenDone::TRUE, + StopWhenDone::True, StringRef(), self->backupPrefix, self->locked, @@ -704,10 +704,10 @@ struct BackupToDBCorrectnessWorkload : TestWorkload { // not be set yet. Adding "waitForDestUID" flag to avoid the race. wait(backupAgent.abortBackup(self->extraDB, self->backupTag, - PartialBackup::FALSE, - AbortOldBackup::FALSE, - DstOnly::FALSE, - WaitForDestUID::TRUE)); + PartialBackup::False, + AbortOldBackup::False, + DstOnly::False, + WaitForDestUID::True)); } catch (Error& e) { TraceEvent("BARW_AbortBackupExtraException", randomID).error(e); if (e.code() != error_code_backup_unneeded) diff --git a/fdbserver/workloads/BackupToDBUpgrade.actor.cpp b/fdbserver/workloads/BackupToDBUpgrade.actor.cpp index 9cf029e7ed..e0553058ed 100644 --- a/fdbserver/workloads/BackupToDBUpgrade.actor.cpp +++ b/fdbserver/workloads/BackupToDBUpgrade.actor.cpp @@ -117,7 +117,7 @@ struct BackupToDBUpgradeWorkload : TestWorkload { } } wait(backupAgent->submitBackup( - tr, tag, backupRanges, StopWhenDone::FALSE, self->backupPrefix, StringRef())); + tr, tag, backupRanges, StopWhenDone::False, self->backupPrefix, StringRef())); wait(tr->commit()); break; } catch (Error& e) { @@ -133,7 +133,7 @@ struct BackupToDBUpgradeWorkload : TestWorkload { } } - wait(success(backupAgent->waitBackup(self->extraDB, tag, StopWhenDone::FALSE))); + wait(success(backupAgent->waitBackup(self->extraDB, tag, StopWhenDone::False))); return Void(); } @@ -500,7 +500,7 @@ struct BackupToDBUpgradeWorkload : TestWorkload { try { TraceEvent("DRU_RestoreDb").detail("RestoreTag", printable(self->restoreTag)); wait(restoreTool.submitBackup( - cx, self->restoreTag, restoreRanges, StopWhenDone::TRUE, StringRef(), self->backupPrefix)); + cx, self->restoreTag, restoreRanges, StopWhenDone::True, StringRef(), self->backupPrefix)); } catch (Error& e) { TraceEvent("DRU_RestoreSubmitBackupError").error(e).detail("Tag", printable(self->restoreTag)); if (e.code() != error_code_backup_unneeded && e.code() != error_code_backup_duplicate) diff --git a/fdbserver/workloads/IncrementalBackup.actor.cpp b/fdbserver/workloads/IncrementalBackup.actor.cpp index 4087e7923d..c273be09be 100644 --- a/fdbserver/workloads/IncrementalBackup.actor.cpp +++ b/fdbserver/workloads/IncrementalBackup.actor.cpp @@ -94,7 +94,7 @@ struct IncrementalBackupWorkload : TestWorkload { // Wait for backup container to be created and avoid race condition TraceEvent("IBackupWaitContainer"); wait(success(self->backupAgent.waitBackup( - cx, self->tag.toString(), StopWhenDone::FALSE, &backupContainer, &backupUID))); + cx, self->tag.toString(), StopWhenDone::False, &backupContainer, &backupUID))); if (!backupContainer.isValid()) { TraceEvent("IBackupCheckListContainersAttempt"); state std::vector containers = @@ -156,9 +156,9 @@ struct IncrementalBackupWorkload : TestWorkload { 1e8, self->tag.toString(), backupRanges, - StopWhenDone::FALSE, - UsePartitionedLog::FALSE, - IncrementalBackupOnly::TRUE)); + StopWhenDone::False, + UsePartitionedLog::False, + IncrementalBackupOnly::True)); } catch (Error& e) { TraceEvent("IBackupSubmitError").error(e); if (e.code() != error_code_backup_duplicate) { @@ -187,7 +187,7 @@ struct IncrementalBackupWorkload : TestWorkload { state UID backupUID; state Version beginVersion = invalidVersion; wait(success(self->backupAgent.waitBackup( - cx, self->tag.toString(), StopWhenDone::FALSE, &backupContainer, &backupUID))); + cx, self->tag.toString(), StopWhenDone::False, &backupContainer, &backupUID))); if (self->checkBeginVersion) { TraceEvent("IBackupReadSystemKeys"); state Reference tr(new ReadYourWritesTransaction(cx)); @@ -229,15 +229,15 @@ struct IncrementalBackupWorkload : TestWorkload { cx, Key(self->tag.toString()), backupURL, - WaitForComplete::TRUE, + WaitForComplete::True, invalidVersion, - Verbose::TRUE, + Verbose::True, normalKeys, Key(), Key(), - LockDB::TRUE, - OnlyApplyMutationLogs::TRUE, - InconsistentSnapshotOnly::FALSE, + LockDB::True, + OnlyApplyMutationLogs::True, + InconsistentSnapshotOnly::False, beginVersion))); TraceEvent("IBackupRestoreSuccess"); } diff --git a/fdbserver/workloads/Mako.actor.cpp b/fdbserver/workloads/Mako.actor.cpp index e5e267f3b5..7bfd33bf62 100644 --- a/fdbserver/workloads/Mako.actor.cpp +++ b/fdbserver/workloads/Mako.actor.cpp @@ -463,18 +463,18 @@ struct MakoWorkload : TestWorkload { if (i == OP_GETREADVERSION) { wait(logLatency(tr.getReadVersion(), &self->opLatencies[i])); } else if (i == OP_GET) { - wait(logLatency(tr.get(rkey, Snapshot::FALSE), &self->opLatencies[i])); + wait(logLatency(tr.get(rkey, Snapshot::False), &self->opLatencies[i])); } else if (i == OP_GETRANGE) { - wait(logLatency(tr.getRange(rkeyRangeRef, CLIENT_KNOBS->TOO_MANY, Snapshot::FALSE), + wait(logLatency(tr.getRange(rkeyRangeRef, CLIENT_KNOBS->TOO_MANY, Snapshot::False), &self->opLatencies[i])); } else if (i == OP_SGET) { - wait(logLatency(tr.get(rkey, Snapshot::TRUE), &self->opLatencies[i])); + wait(logLatency(tr.get(rkey, Snapshot::True), &self->opLatencies[i])); } else if (i == OP_SGETRANGE) { // do snapshot get range here - wait(logLatency(tr.getRange(rkeyRangeRef, CLIENT_KNOBS->TOO_MANY, Snapshot::TRUE), + wait(logLatency(tr.getRange(rkeyRangeRef, CLIENT_KNOBS->TOO_MANY, Snapshot::True), &self->opLatencies[i])); } else if (i == OP_UPDATE) { - wait(logLatency(tr.get(rkey, Snapshot::FALSE), &self->opLatencies[OP_GET])); + wait(logLatency(tr.get(rkey, Snapshot::False), &self->opLatencies[OP_GET])); if (self->latencyForLocalOperation) { double opBegin = timer(); tr.set(rkey, rval); diff --git a/fdbserver/workloads/MemoryLifetime.actor.cpp b/fdbserver/workloads/MemoryLifetime.actor.cpp index bd8c5685ca..15d8526edb 100644 --- a/fdbserver/workloads/MemoryLifetime.actor.cpp +++ b/fdbserver/workloads/MemoryLifetime.actor.cpp @@ -70,8 +70,8 @@ struct MemoryLifetime : KVWorkload { ACTOR Future _start(Database cx, MemoryLifetime* self) { state double startTime = now(); state ReadYourWritesTransaction tr(cx); - state Reverse reverse = Reverse::FALSE; - state Snapshot snapshot = Snapshot::FALSE; + state Reverse reverse = Reverse::False; + state Snapshot snapshot = Snapshot::False; loop { try { int op = deterministicRandom()->randomInt(0, 4); diff --git a/fdbserver/workloads/PopulateTPCC.actor.cpp b/fdbserver/workloads/PopulateTPCC.actor.cpp index 40e7a2b283..25a1ccc93f 100644 --- a/fdbserver/workloads/PopulateTPCC.actor.cpp +++ b/fdbserver/workloads/PopulateTPCC.actor.cpp @@ -174,7 +174,7 @@ struct PopulateTPCC : TestWorkload { item.i_data = self->dataString(item.arena); BinaryWriter w(IncludeVersion()); serializer(w, item); - tr.set(item.key(), w.toValue(), AddConflictRange::FALSE); + tr.set(item.key(), w.toValue(), AddConflictRange::False); } wait(tr.commit()); break; @@ -242,11 +242,11 @@ struct PopulateTPCC : TestWorkload { { BinaryWriter w(IncludeVersion()); serializer(w, c); - tr.set(c.key(), w.toValue(), AddConflictRange::FALSE); + tr.set(c.key(), w.toValue(), AddConflictRange::False); } { // Write index - tr.set(c.indexLastKey(), c.key(), AddConflictRange::FALSE); + tr.set(c.indexLastKey(), c.key(), AddConflictRange::False); } { BinaryWriter w(IncludeVersion()); @@ -255,7 +255,7 @@ struct PopulateTPCC : TestWorkload { BinaryWriter kW(Unversioned()); serializer(kW, k); auto key = kW.toValue().withPrefix(LiteralStringRef("History/")); - tr.set(key, w.toValue(), AddConflictRange::FALSE); + tr.set(key, w.toValue(), AddConflictRange::False); } } try { @@ -315,11 +315,11 @@ struct PopulateTPCC : TestWorkload { ol.ol_dist_info = self->aString(ol.arena, 24, 24); BinaryWriter w(IncludeVersion()); serializer(w, ol); - tr.set(ol.key(), w.toValue(), AddConflictRange::FALSE); + tr.set(ol.key(), w.toValue(), AddConflictRange::False); } BinaryWriter w(IncludeVersion()); serializer(w, o); - tr.set(o.key(), w.toValue(), AddConflictRange::FALSE); + tr.set(o.key(), w.toValue(), AddConflictRange::False); } try { wait(tr.commit()); @@ -346,7 +346,7 @@ struct PopulateTPCC : TestWorkload { no.no_w_id = w_id; BinaryWriter w(IncludeVersion()); serializer(w, no); - tr.set(no.key(), w.toValue(), AddConflictRange::FALSE); + tr.set(no.key(), w.toValue(), AddConflictRange::False); } try { wait(tr.commit()); @@ -381,7 +381,7 @@ struct PopulateTPCC : TestWorkload { d.d_next_o_id = 3000; BinaryWriter w(IncludeVersion()); serializer(w, d); - tr.set(d.key(), w.toValue(), AddConflictRange::FALSE); + tr.set(d.key(), w.toValue(), AddConflictRange::False); try { wait(tr.commit()); wait(populateCustomers(self, cx, w_id, d_id)); @@ -426,7 +426,7 @@ struct PopulateTPCC : TestWorkload { s.s_data = self->dataString(s.arena); BinaryWriter w(IncludeVersion()); serializer(w, s); - tr.set(s.key(), w.toValue(), AddConflictRange::FALSE); + tr.set(s.key(), w.toValue(), AddConflictRange::False); } try { wait(tr.commit()); @@ -458,7 +458,7 @@ struct PopulateTPCC : TestWorkload { w.w_ytd = 300000; BinaryWriter writer(IncludeVersion()); serializer(writer, w); - tr.set(w.key(), writer.toValue(), AddConflictRange::FALSE); + tr.set(w.key(), writer.toValue(), AddConflictRange::False); wait(tr.commit()); break; } catch (Error& e) { diff --git a/fdbserver/workloads/QueuePush.actor.cpp b/fdbserver/workloads/QueuePush.actor.cpp index 8e8b9f564d..cae37474f1 100644 --- a/fdbserver/workloads/QueuePush.actor.cpp +++ b/fdbserver/workloads/QueuePush.actor.cpp @@ -115,12 +115,12 @@ struct QueuePushWorkload : TestWorkload { state Key lastKey; if (self->forward) { - Key _lastKey = wait(tr.getKey(lastLessThan(self->endingKey), Snapshot::TRUE)); + Key _lastKey = wait(tr.getKey(lastLessThan(self->endingKey), Snapshot::True)); lastKey = _lastKey; if (lastKey == StringRef()) lastKey = self->startingKey; } else { - Key _lastKey = wait(tr.getKey(firstGreaterThan(self->startingKey), Snapshot::TRUE)); + Key _lastKey = wait(tr.getKey(firstGreaterThan(self->startingKey), Snapshot::True)); lastKey = _lastKey; if (!normalKeys.contains(lastKey)) lastKey = self->endingKey; diff --git a/fdbserver/workloads/RandomSelector.actor.cpp b/fdbserver/workloads/RandomSelector.actor.cpp index 8c16aa516c..d4c6bd00fe 100644 --- a/fdbserver/workloads/RandomSelector.actor.cpp +++ b/fdbserver/workloads/RandomSelector.actor.cpp @@ -103,7 +103,7 @@ struct RandomSelectorWorkload : TestWorkload { state int offsetB; state int randomLimit; state int randomByteLimit; - state Reverse reverse = Reverse::FALSE; + state Reverse reverse = Reverse::False; state Error error; clientID = format("%08d", self->clientId); @@ -447,7 +447,7 @@ struct RandomSelectorWorkload : TestWorkload { wait(trRYOW.getRange(KeySelectorRef(StringRef(clientID + "b/" + myKeyA), onEqualA, offsetA), KeySelectorRef(StringRef(clientID + "b/" + myKeyB), onEqualB, offsetB), randomLimit, - Snapshot::FALSE, + Snapshot::False, reverse)); getRangeTest1 = getRangeTest; @@ -457,7 +457,7 @@ struct RandomSelectorWorkload : TestWorkload { tr.getRange(KeySelectorRef(StringRef(clientID + "d/" + myKeyA), onEqualA, offsetA), KeySelectorRef(StringRef(clientID + "d/" + myKeyB), onEqualB, offsetB), randomLimit, - Snapshot::FALSE, + Snapshot::False, reverse)); bool fail = false; diff --git a/fdbserver/workloads/RestoreBackup.actor.cpp b/fdbserver/workloads/RestoreBackup.actor.cpp index 3240f7ed66..17a4355d36 100644 --- a/fdbserver/workloads/RestoreBackup.actor.cpp +++ b/fdbserver/workloads/RestoreBackup.actor.cpp @@ -114,9 +114,9 @@ struct RestoreBackupWorkload final : TestWorkload { cx, self->tag, Key(self->backupContainer->getURL()), - WaitForComplete::TRUE, + WaitForComplete::True, ::invalidVersion, - Verbose::TRUE))); + Verbose::True))); return Void(); } diff --git a/fdbserver/workloads/RyowCorrectness.actor.cpp b/fdbserver/workloads/RyowCorrectness.actor.cpp index 817422dbf1..56433f6dd4 100644 --- a/fdbserver/workloads/RyowCorrectness.actor.cpp +++ b/fdbserver/workloads/RyowCorrectness.actor.cpp @@ -49,7 +49,7 @@ struct Operation { Value value; int limit; - Reverse reverse{ Reverse::FALSE }; + Reverse reverse{ Reverse::False }; }; // A workload which executes random sequences of operations on RYOW transactions and confirms the results diff --git a/fdbserver/workloads/SelectorCorrectness.actor.cpp b/fdbserver/workloads/SelectorCorrectness.actor.cpp index a848f50bb9..f4b729cbb1 100644 --- a/fdbserver/workloads/SelectorCorrectness.actor.cpp +++ b/fdbserver/workloads/SelectorCorrectness.actor.cpp @@ -109,7 +109,7 @@ struct SelectorCorrectnessWorkload : TestWorkload { state int offsetA; state int offsetB; state Standalone maxKey; - state Reverse reverse = Reverse::FALSE; + state Reverse reverse = Reverse::False; maxKey = Standalone(format("%010d", self->maxKeySpace + 1)); @@ -180,7 +180,7 @@ struct SelectorCorrectnessWorkload : TestWorkload { wait(trRYOW.getRange(KeySelectorRef(StringRef(myKeyA), onEqualA, offsetA), KeySelectorRef(StringRef(myKeyB), onEqualB, offsetB), 2 * (self->maxKeySpace + self->maxOffset), - Snapshot::FALSE, + Snapshot::False, reverse)); int trueSize = 0; @@ -208,7 +208,7 @@ struct SelectorCorrectnessWorkload : TestWorkload { wait(tr.getRange(KeySelectorRef(StringRef(myKeyA), onEqualA, offsetA), KeySelectorRef(StringRef(myKeyB), onEqualB, offsetB), 2 * (self->maxKeySpace + self->maxOffset), - Snapshot::FALSE, + Snapshot::False, reverse)); int trueSize = 0; diff --git a/fdbserver/workloads/Serializability.actor.cpp b/fdbserver/workloads/Serializability.actor.cpp index d774fdcf10..442f4ef41d 100644 --- a/fdbserver/workloads/Serializability.actor.cpp +++ b/fdbserver/workloads/Serializability.actor.cpp @@ -40,18 +40,18 @@ struct SerializabilityWorkload : TestWorkload { KeySelector begin; KeySelector end; int limit; - Snapshot snapshot{ Snapshot::FALSE }; - Reverse reverse{ Reverse::FALSE }; + Snapshot snapshot{ Snapshot::False }; + Reverse reverse{ Reverse::False }; }; struct GetKeyOperation { KeySelector key; - Snapshot snapshot{ Snapshot::FALSE }; + Snapshot snapshot{ Snapshot::False }; }; struct GetOperation { Key key; - Snapshot snapshot{ Snapshot::FALSE }; + Snapshot snapshot{ Snapshot::False }; }; struct TransactionOperation { diff --git a/fdbserver/workloads/SnapTest.actor.cpp b/fdbserver/workloads/SnapTest.actor.cpp index ce779d2c76..caefb96dcd 100644 --- a/fdbserver/workloads/SnapTest.actor.cpp +++ b/fdbserver/workloads/SnapTest.actor.cpp @@ -183,7 +183,7 @@ public: // workload functions Key key1Ref(Key1); std::string Val1 = std::to_string(id); Value val1Ref(Val1); - tr.set(key1Ref, val1Ref, AddConflictRange::FALSE); + tr.set(key1Ref, val1Ref, AddConflictRange::False); } wait(tr.commit()); break; diff --git a/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp b/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp index 151313d249..7c6c2006a6 100644 --- a/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp +++ b/fdbserver/workloads/SpecialKeySpaceCorrectness.actor.cpp @@ -126,14 +126,14 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload { ACTOR Future getRangeCallActor(Database cx, SpecialKeySpaceCorrectnessWorkload* self) { state double lastTime = now(); - state Reverse reverse = Reverse::FALSE; + state Reverse reverse = Reverse::False; loop { wait(poisson(&lastTime, 1.0 / self->transactionsPerSecond)); reverse.set(deterministicRandom()->coinflip()); state GetRangeLimits limit = self->randomLimits(); state KeySelector begin = self->randomKeySelector(); state KeySelector end = self->randomKeySelector(); - auto correctResultFuture = self->ryw->getRange(begin, end, limit, Snapshot::FALSE, reverse); + auto correctResultFuture = self->ryw->getRange(begin, end, limit, Snapshot::False, reverse); ASSERT(correctResultFuture.isReady()); auto correctResult = correctResultFuture.getValue(); auto testResultFuture = cx->specialKeySpace->getRange(self->ryw.getPtr(), begin, end, limit, reverse); @@ -174,7 +174,7 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload { self->ryw->clear(rkr); } // use the same key selectors again to test consistency of ryw - auto correctRywResultFuture = self->ryw->getRange(begin, end, limit, Snapshot::FALSE, reverse); + auto correctRywResultFuture = self->ryw->getRange(begin, end, limit, Snapshot::False, reverse); ASSERT(correctRywResultFuture.isReady()); auto correctRywResult = correctRywResultFuture.getValue(); auto testRywResultFuture = cx->specialKeySpace->getRange(self->ryw.getPtr(), begin, end, limit, reverse); @@ -550,11 +550,11 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload { } Reverse reverse{ deterministicRandom()->coinflip() }; - auto correctResultFuture = referenceTx->getRange(begin, end, limit, Snapshot::FALSE, reverse); + auto correctResultFuture = referenceTx->getRange(begin, end, limit, Snapshot::False, reverse); ASSERT(correctResultFuture.isReady()); begin.setKey(begin.getKey().withPrefix(prefix, begin.arena())); end.setKey(end.getKey().withPrefix(prefix, begin.arena())); - auto testResultFuture = tx->getRange(begin, end, limit, Snapshot::FALSE, reverse); + auto testResultFuture = tx->getRange(begin, end, limit, Snapshot::False, reverse); ASSERT(testResultFuture.isReady()); auto correct_iter = correctResultFuture.get().begin(); auto test_iter = testResultFuture.get().begin(); diff --git a/fdbserver/workloads/SubmitBackup.actor.cpp b/fdbserver/workloads/SubmitBackup.actor.cpp index 8a45431b44..0240bd21dc 100644 --- a/fdbserver/workloads/SubmitBackup.actor.cpp +++ b/fdbserver/workloads/SubmitBackup.actor.cpp @@ -62,7 +62,7 @@ struct SubmitBackupWorkload final : TestWorkload { self->tag.toString(), backupRanges, self->stopWhenDone, - UsePartitionedLog::FALSE, + UsePartitionedLog::False, self->incremental)); } catch (Error& e) { TraceEvent("BackupSubmitError").error(e); diff --git a/fdbserver/workloads/TPCC.actor.cpp b/fdbserver/workloads/TPCC.actor.cpp index 80ad1adcdd..32c8e3c9d3 100644 --- a/fdbserver/workloads/TPCC.actor.cpp +++ b/fdbserver/workloads/TPCC.actor.cpp @@ -468,7 +468,7 @@ struct TPCC : TestWorkload { order.o_w_id = customer.c_w_id; order.o_d_id = customer.c_d_id; order.o_c_id = customer.c_id; - RangeResult range = wait(tr.getRange(order.keyRange(1), 1, Snapshot::FALSE, Reverse::TRUE)); + RangeResult range = wait(tr.getRange(order.keyRange(1), 1, Snapshot::False, Reverse::True)); ASSERT(range.size() > 0); { BinaryReader r(range[0].value, IncludeVersion()); diff --git a/fdbserver/workloads/Unreadable.actor.cpp b/fdbserver/workloads/Unreadable.actor.cpp index 420d5e1e38..93fad4e605 100644 --- a/fdbserver/workloads/Unreadable.actor.cpp +++ b/fdbserver/workloads/Unreadable.actor.cpp @@ -296,8 +296,8 @@ struct UnreadableWorkload : TestWorkload { ACTOR Future _start(Database cx, UnreadableWorkload* self) { state int testCount = 0; - state Reverse reverse = Reverse::FALSE; - state Snapshot snapshot = Snapshot::FALSE; + state Reverse reverse = Reverse::False; + state Snapshot snapshot = Snapshot::False; for (; testCount < 100; testCount++) { //TraceEvent("RYWT_Start").detail("TestCount", testCount); state ReadYourWritesTransaction tr(cx); diff --git a/fdbserver/workloads/WriteBandwidth.actor.cpp b/fdbserver/workloads/WriteBandwidth.actor.cpp index a5f9f13661..6d04b3cf2d 100644 --- a/fdbserver/workloads/WriteBandwidth.actor.cpp +++ b/fdbserver/workloads/WriteBandwidth.actor.cpp @@ -122,7 +122,7 @@ struct WriteBandwidthWorkload : KVWorkload { keyAfter(self->keyForIndex(startIdx + self->keysPerTransaction - 1, false)))); for (int i = 0; i < self->keysPerTransaction; i++) - tr.set(self->keyForIndex(startIdx + i, false), self->randomValue(), AddConflictRange::FALSE); + tr.set(self->keyForIndex(startIdx + i, false), self->randomValue(), AddConflictRange::False); start = now(); wait(tr.commit()); diff --git a/flow/BooleanParam.h b/flow/BooleanParam.h index b0b7ffee54..d06faea79a 100644 --- a/flow/BooleanParam.h +++ b/flow/BooleanParam.h @@ -29,7 +29,7 @@ public: \ explicit constexpr ParamName(bool value) : value(value) {} \ constexpr operator bool() const { return value; } \ - static ParamName const TRUE, FALSE; \ + static ParamName const True, False; \ constexpr void set(bool value) { this->value = value; } \ }; \ template <> \ @@ -38,8 +38,8 @@ } #define FDB_DEFINE_BOOLEAN_PARAM(ParamName) \ - ParamName const ParamName::TRUE = ParamName(true); \ - ParamName const ParamName::FALSE = ParamName(false) + ParamName const ParamName::True = ParamName(true); \ + ParamName const ParamName::False = ParamName(false) #define FDB_BOOLEAN_PARAM(ParamName) \ FDB_DECLARE_BOOLEAN_PARAM(ParamName); \ diff --git a/flow/Knobs.cpp b/flow/Knobs.cpp index a466377842..7a8f1e24f7 100644 --- a/flow/Knobs.cpp +++ b/flow/Knobs.cpp @@ -31,7 +31,7 @@ FlowKnobs::FlowKnobs(Randomize randomize, IsSimulated isSimulated) { initialize(randomize, isSimulated); } -FlowKnobs bootstrapGlobalFlowKnobs(Randomize::FALSE, IsSimulated::FALSE); +FlowKnobs bootstrapGlobalFlowKnobs(Randomize::False, IsSimulated::False); FlowKnobs const* FLOW_KNOBS = &bootstrapGlobalFlowKnobs; #define init(knob, value) initKnob(knob, value, #knob)