Replace g_random and g_nondeterministic_random with functions deterministicRandom() and nondeterministicRandom() that return thread_local random number generators. Delete g_debug_random and trace_random. Allow only deterministicRandom() to be seeded, and require it to be seeded from each thread on which it is used.

This commit is contained in:
A.J. Beamon 2019-05-10 14:01:52 -07:00
parent a8d6593258
commit 5f55f3f613
160 changed files with 1173 additions and 1156 deletions

View File

@ -66,7 +66,7 @@ namespace FDB {
}
loop {
state int64_t candidate = g_random->randomInt(start, start + window);
state int64_t candidate = deterministicRandom()->randomInt(start, start + window);
// if thread safety is needed, this should be locked {
state Future<FDBStandalone<RangeResultRef>> latestCounter = tr->getRange(counters.range(), 1, true, true);

View File

@ -83,9 +83,7 @@ void fdb_flow_test() {
int randomSeed = platform::getRandomSeed();
g_random = new DeterministicRandom(randomSeed);
g_nondeterministic_random = new DeterministicRandom(platform::getRandomSeed());
g_debug_random = new DeterministicRandom(platform::getRandomSeed());
setThreadLocalDeterministicRandomSeed(randomSeed);
g_network = newNet2( false );

View File

@ -1030,7 +1030,7 @@ struct TuplePackFunc : InstructionFunc {
for (; i < items1.size(); ++i) {
Standalone<StringRef> str = wait(items1[i].value);
Tuple itemTuple = Tuple::unpack(str);
if(g_random->coinflip()) {
if(deterministicRandom()->coinflip()) {
Tuple::ElementType type = itemTuple.getType(0);
if(type == Tuple::NULL_TYPE) {
tuple.appendNull();
@ -1119,7 +1119,7 @@ struct TupleRangeFunc : InstructionFunc {
for (; i < items1.size(); ++i) {
Standalone<StringRef> str = wait(items1[i].value);
Tuple itemTuple = Tuple::unpack(str);
if(g_random->coinflip()) {
if(deterministicRandom()->coinflip()) {
Tuple::ElementType type = itemTuple.getType(0);
if(type == Tuple::NULL_TYPE) {
tuple.appendNull();
@ -1809,8 +1809,7 @@ int main( int argc, char** argv ) {
try {
platformInit();
registerCrashHandler();
g_random = new DeterministicRandom(1);
g_nondeterministic_random = new DeterministicRandom(platform::getRandomSeed());
setThreadLocalDeterministicRandomSeed(1);
// Get arguments
if (argc < 3) {

View File

@ -1450,7 +1450,7 @@ ACTOR Future<Void> updateAgentPollRate(Database src, std::string rootKey, std::s
}
ACTOR Future<Void> statusUpdateActor(Database statusUpdateDest, std::string name, enumProgramExe exe, double *pollDelay, Database taskDest = Database(),
std::string id = g_nondeterministic_random->randomUniqueID().toString()) {
std::string id = nondeterministicRandom()->randomUniqueID().toString()) {
state std::string metaKey = layerStatusMetaPrefixRange.begin.toString() + "json/" + name;
state std::string rootKey = backupStatusPrefixRange.begin.toString() + name + "/json";
state std::string instanceKey = rootKey + "/" + "agent-" + id;
@ -1491,7 +1491,7 @@ ACTOR Future<Void> statusUpdateActor(Database statusUpdateDest, std::string name
}
}
wait(delay(CLIENT_KNOBS->BACKUP_STATUS_DELAY * ( ( 1.0 - CLIENT_KNOBS->BACKUP_STATUS_JITTER ) + 2 * g_random->random01() * CLIENT_KNOBS->BACKUP_STATUS_JITTER )));
wait(delay(CLIENT_KNOBS->BACKUP_STATUS_DELAY * ( ( 1.0 - CLIENT_KNOBS->BACKUP_STATUS_JITTER ) + 2 * deterministicRandom()->random01() * CLIENT_KNOBS->BACKUP_STATUS_JITTER )));
// Now that status was written at least once by this process (and hopefully others), start the poll rate control updater if it wasn't started yet
if(!pollRateUpdater.isValid() && pollDelay != nullptr)
@ -1506,7 +1506,7 @@ ACTOR Future<Void> statusUpdateActor(Database statusUpdateDest, std::string name
ACTOR Future<Void> runDBAgent(Database src, Database dest) {
state double pollDelay = 1.0 / CLIENT_KNOBS->BACKUP_AGGREGATE_POLL_RATE;
std::string id = g_nondeterministic_random->randomUniqueID().toString();
std::string id = nondeterministicRandom()->randomUniqueID().toString();
state Future<Void> status = statusUpdateActor(src, "dr_backup", EXE_DR_AGENT, &pollDelay, dest, id);
state Future<Void> status_other = statusUpdateActor(dest, "dr_backup_dest", EXE_DR_AGENT, &pollDelay, dest, id);

View File

@ -2540,7 +2540,7 @@ ACTOR Future<int> cli(CLIOptions opt, LineNoise* plinenoise) {
warn = checkStatus( timeWarning(5.0, "\nWARNING: Long delay (Ctrl-C to interrupt)\n"), ccf );
try {
state UID randomID = g_random->randomUniqueID();
state UID randomID = deterministicRandom()->randomUniqueID();
TraceEvent(SevInfo, "CLICommandLog", randomID).detail("Command", line);
bool malformed, partial;

View File

@ -39,7 +39,7 @@ ACTOR Future<Void> sendStuff(int id, Reference<IRateControl> t, int bytes) {
state double ts = timer();
state int total = 0;
while(total < bytes) {
state int r = std::min<int>(g_random->randomInt(0,1000), bytes - total);
state int r = std::min<int>(deterministicRandom()->randomInt(0,1000), bytes - total);
wait(t->getAllowance(r));
total += r;
}

View File

@ -429,7 +429,7 @@ ACTOR Future<Void> readCommitted(Database cx, PromiseStream<RangeResultWithVersi
values.resize(values.arena(), values.size() / 2);
values.more = true;
// Half of the time wait for this tr to expire so that the next read is at a different version
if(g_random->random01() < 0.5)
if(deterministicRandom()->random01() < 0.5)
wait(delay(6.0));
}
@ -488,7 +488,7 @@ ACTOR Future<Void> readCommitted(Database cx, PromiseStream<RCGroup> results, Fu
rangevalue.resize(rangevalue.arena(), rangevalue.size() / 2);
rangevalue.more = true;
// Half of the time wait for this tr to expire so that the next read is at a different version
if(g_random->random01() < 0.5)
if(deterministicRandom()->random01() < 0.5)
wait(delay(6.0));
}

View File

@ -334,14 +334,14 @@ public:
}
Future<Reference<IBackupFile>> writeLogFile(Version beginVersion, Version endVersion, int blockSize) {
return writeFile(logVersionFolderString(beginVersion) + format("log,%lld,%lld,%s,%d", beginVersion, endVersion, g_random->randomUniqueID().toString().c_str(), blockSize));
return writeFile(logVersionFolderString(beginVersion) + format("log,%lld,%lld,%s,%d", beginVersion, endVersion, deterministicRandom()->randomUniqueID().toString().c_str(), blockSize));
}
Future<Reference<IBackupFile>> writeRangeFile(Version snapshotBeginVersion, int snapshotFileCount, Version fileVersion, int blockSize) {
std::string fileName = format("range,%" PRId64 ",%s,%d", fileVersion, g_random->randomUniqueID().toString().c_str(), blockSize);
std::string fileName = format("range,%" PRId64 ",%s,%d", fileVersion, deterministicRandom()->randomUniqueID().toString().c_str(), blockSize);
// In order to test backward compatibility in simulation, sometimes write to the old path format
if(g_network->isSimulated() && g_random->coinflip()) {
if(g_network->isSimulated() && deterministicRandom()->coinflip()) {
return writeFile(old_rangeVersionFolderString(fileVersion) + fileName);
}
@ -1224,7 +1224,7 @@ public:
if(g_network->isSimulated()) {
if(!fileExists(fullPath))
throw file_not_found();
std::string uniquePath = fullPath + "." + g_random->randomUniqueID().toString() + ".lnk";
std::string uniquePath = fullPath + "." + deterministicRandom()->randomUniqueID().toString() + ".lnk";
unlink(uniquePath.c_str());
ASSERT(symlink(basename(path).c_str(), uniquePath.c_str()) == 0);
fullPath = uniquePath = uniquePath;
@ -1243,16 +1243,16 @@ public:
blockSize = atoi(path.substr(lastComma + 1).c_str());
}
if(blockSize <= 0) {
blockSize = g_random->randomInt(1e4, 1e6);
blockSize = deterministicRandom()->randomInt(1e4, 1e6);
}
if(g_random->random01() < .01) {
blockSize /= g_random->randomInt(1, 3);
if(deterministicRandom()->random01() < .01) {
blockSize /= deterministicRandom()->randomInt(1, 3);
}
return map(f, [=](Reference<IAsyncFile> fr) {
int readAhead = g_random->randomInt(0, 3);
int reads = g_random->randomInt(1, 3);
int cacheSize = g_random->randomInt(0, 3);
int readAhead = deterministicRandom()->randomInt(0, 3);
int reads = deterministicRandom()->randomInt(1, 3);
int cacheSize = deterministicRandom()->randomInt(0, 3);
return Reference<IAsyncFile>(new AsyncFileReadAheadCache(fr, blockSize, readAhead, reads, cacheSize));
});
}
@ -1295,7 +1295,7 @@ public:
int flags = IAsyncFile::OPEN_NO_AIO | IAsyncFile::OPEN_CREATE | IAsyncFile::OPEN_ATOMIC_WRITE_AND_CREATE | IAsyncFile::OPEN_READWRITE;
std::string fullPath = joinPath(m_path, path);
platform::createDirectory(parentDirectory(fullPath));
std::string temp = fullPath + "." + g_random->randomUniqueID().toString() + ".temp";
std::string temp = fullPath + "." + deterministicRandom()->randomUniqueID().toString() + ".temp";
Future<Reference<IAsyncFile>> f = IAsyncFileSystem::filesystem()->open(temp, flags, 0644);
return map(f, [=](Reference<IAsyncFile> f) {
return Reference<IBackupFile>(new BackupFile(path, f, fullPath));
@ -1730,7 +1730,7 @@ ACTOR Future<Void> writeAndVerifyFile(Reference<IBackupContainer> c, Reference<I
if(size > 0) {
content = makeString(size);
for(int i = 0; i < content.size(); ++i)
mutateString(content)[i] = (uint8_t)g_random->randomInt(0, 256);
mutateString(content)[i] = (uint8_t)deterministicRandom()->randomInt(0, 256);
wait(f->append(content.begin(), content.size()));
}
@ -1749,7 +1749,7 @@ ACTOR Future<Void> writeAndVerifyFile(Reference<IBackupContainer> c, Reference<I
// Randomly advance version by up to 1 second of versions
Version nextVersion(Version v) {
int64_t increment = g_random->randomInt64(1, CLIENT_KNOBS->CORE_VERSIONSPERSECOND);
int64_t increment = deterministicRandom()->randomInt64(1, CLIENT_KNOBS->CORE_VERSIONSPERSECOND);
return v + increment;
}
@ -1773,20 +1773,20 @@ ACTOR Future<Void> testBackupContainer(std::string url) {
state std::map<Version, int64_t> snapshotSizes;
state int nRangeFiles = 0;
state std::map<Version, std::string> logs;
state Version v = g_random->randomInt64(0, std::numeric_limits<Version>::max() / 2);
state Version v = deterministicRandom()->randomInt64(0, std::numeric_limits<Version>::max() / 2);
// List of sizes to use to test edge cases on underlying file implementations
state std::vector<int> fileSizes = {0, 10000000, 5000005};
loop {
state Version logStart = v;
state int kvfiles = g_random->randomInt(0, 3);
state int kvfiles = deterministicRandom()->randomInt(0, 3);
while(kvfiles > 0) {
if(snapshots.empty()) {
snapshots[v] = {};
snapshotSizes[v] = 0;
if(g_random->coinflip()) {
if(deterministicRandom()->coinflip()) {
v = nextVersion(v);
}
}
@ -1799,7 +1799,7 @@ ACTOR Future<Void> testBackupContainer(std::string url) {
snapshotSizes.rbegin()->second += size;
writes.push_back(writeAndVerifyFile(c, range, size));
if(g_random->random01() < .2) {
if(deterministicRandom()->random01() < .2) {
writes.push_back(c->writeKeyspaceSnapshotFile(snapshots.rbegin()->second, snapshotSizes.rbegin()->second));
snapshots[v] = {};
snapshotSizes[v] = 0;
@ -1809,7 +1809,7 @@ ACTOR Future<Void> testBackupContainer(std::string url) {
--kvfiles;
}
if(logStart == v || g_random->coinflip()) {
if(logStart == v || deterministicRandom()->coinflip()) {
v = nextVersion(v);
}
state Reference<IBackupFile> log = wait(c->writeLogFile(logStart, v, 10));
@ -1818,7 +1818,7 @@ ACTOR Future<Void> testBackupContainer(std::string url) {
writes.push_back(writeAndVerifyFile(c, log, size));
// Randomly stop after a snapshot has finished and all manually seeded file sizes have been used.
if(fileSizes.empty() && !snapshots.empty() && snapshots.rbegin()->second.empty() && g_random->random01() < .2) {
if(fileSizes.empty() && !snapshots.empty() && snapshots.rbegin()->second.empty() && deterministicRandom()->random01() < .2) {
snapshots.erase(snapshots.rbegin()->first);
break;
}
@ -1918,7 +1918,7 @@ TEST_CASE("/backup/containers_list") {
TEST_CASE("/backup/time") {
// test formatTime()
for(int i = 0; i < 1000; ++i) {
int64_t ts = g_random->randomInt64(0, std::numeric_limits<int32_t>::max());
int64_t ts = deterministicRandom()->randomInt64(0, std::numeric_limits<int32_t>::max());
ASSERT(BackupAgentBase::parseTime(BackupAgentBase::formatTime(ts)) == ts);
}

View File

@ -735,7 +735,7 @@ namespace dbBackup {
Optional<Value> stopValue = wait(fStopValue);
state Version stopVersionData = stopValue.present() ? BinaryReader::fromStringRef<Version>(stopValue.get(), Unversioned()) : -1;
if(endVersion - beginVersion > g_random->randomInt64(0, CLIENT_KNOBS->BACKUP_VERSION_DELAY)) {
if(endVersion - beginVersion > deterministicRandom()->randomInt64(0, CLIENT_KNOBS->BACKUP_VERSION_DELAY)) {
TraceEvent("DBA_CopyLogs").detail("BeginVersion", beginVersion).detail("ApplyVersion", applyVersion).detail("EndVersion", endVersion).detail("StopVersionData", stopVersionData).detail("LogUID", task->params[BackupAgentBase::keyConfigLogUid]);
}
@ -939,7 +939,7 @@ namespace dbBackup {
tr->set(task->params[BackupAgentBase::keyConfigLogUid].withPrefix(applyMutationsEndRange.begin), BinaryWriter::toValue(beginVersion, Unversioned()));
Optional<Value> stopWhenDone = wait(fStopWhenDone);
if(endVersion - beginVersion > g_random->randomInt64(0, CLIENT_KNOBS->BACKUP_VERSION_DELAY)) {
if(endVersion - beginVersion > deterministicRandom()->randomInt64(0, CLIENT_KNOBS->BACKUP_VERSION_DELAY)) {
TraceEvent("DBA_CopyDiffLogs").detail("BeginVersion", beginVersion).detail("EndVersion", endVersion).detail("LogUID", task->params[BackupAgentBase::keyConfigLogUid]);
}
@ -1478,7 +1478,7 @@ namespace dbBackup {
if (existingDestUidValue.present()) {
destUidValue = existingDestUidValue.get();
} else {
destUidValue = BinaryWriter::toValue(g_random->randomUniqueID(), Unversioned());
destUidValue = BinaryWriter::toValue(deterministicRandom()->randomUniqueID(), Unversioned());
srcTr->set(destUidLookupPath, destUidValue);
}
}
@ -1830,7 +1830,7 @@ public:
}
ACTOR static Future<Void> submitBackup(DatabaseBackupAgent* backupAgent, Reference<ReadYourWritesTransaction> tr, Key tagName, Standalone<VectorRef<KeyRangeRef>> backupRanges, bool stopWhenDone, Key addPrefix, Key removePrefix, bool lockDB, bool databasesInSync) {
state UID logUid = g_random->randomUniqueID();
state UID logUid = deterministicRandom()->randomUniqueID();
state Key logUidValue = BinaryWriter::toValue(logUid, Unversioned());
state UID logUidCurrent = wait(backupAgent->getLogUid(tr, tagName));
@ -1960,7 +1960,7 @@ public:
checkAtomicSwitchOverConfig(srcStatus, destStatus, tagName);
}
state UID logUid = g_random->randomUniqueID();
state UID logUid = deterministicRandom()->randomUniqueID();
state Key logUidValue = BinaryWriter::toValue(logUid, Unversioned());
state UID logUidCurrent = wait(drAgent.getLogUid(backupAgent->taskBucket->src, tagName));

View File

@ -143,7 +143,7 @@ ACTOR Future<Void> failureMonitorClientLoop(
double slowThreshold = .200 + waitfor + FLOW_KNOBS->MAX_BUGGIFIED_DELAY;
double warnAlwaysThreshold = CLIENT_KNOBS->FAILURE_MIN_DELAY/2;
if (elapsed > slowThreshold && g_random->random01() < elapsed / warnAlwaysThreshold) {
if (elapsed > slowThreshold && deterministicRandom()->random01() < elapsed / warnAlwaysThreshold) {
TraceEvent(elapsed > warnAlwaysThreshold ? SevWarnAlways : SevWarn, "FailureMonitorClientSlow").detail("Elapsed", elapsed).detail("Expected", waitfor);
}

View File

@ -1171,7 +1171,7 @@ namespace fileBackup {
// Start writing a new file after verifying this task should keep running as of a new read version (which must be >= outVersion)
outVersion = values.second;
// block size must be at least large enough for 3 max size keys and 2 max size values + overhead so 250k conservatively.
state int blockSize = BUGGIFY ? g_random->randomInt(250e3, 4e6) : CLIENT_KNOBS->BACKUP_RANGEFILE_BLOCK_SIZE;
state int blockSize = BUGGIFY ? deterministicRandom()->randomInt(250e3, 4e6) : CLIENT_KNOBS->BACKUP_RANGEFILE_BLOCK_SIZE;
state Version snapshotBeginVersion;
state int64_t snapshotRangeFileCount;
@ -1600,7 +1600,7 @@ namespace fileBackup {
state std::vector<KeyRange> rangesToAdd;
// Limit number of tasks added per transaction
int taskBatchSize = BUGGIFY ? g_random->randomInt(1, countShardsToDispatch + 1) : CLIENT_KNOBS->BACKUP_DISPATCH_ADDTASK_SIZE;
int taskBatchSize = BUGGIFY ? deterministicRandom()->randomInt(1, countShardsToDispatch + 1) : CLIENT_KNOBS->BACKUP_DISPATCH_ADDTASK_SIZE;
int added = 0;
while(countShardsToDispatch > 0 && added < taskBatchSize && shardMap.size() > 0) {
@ -1696,7 +1696,7 @@ namespace fileBackup {
Version scheduledVersion = invalidVersion;
// If the next dispatch version is in the future, choose a random version at which to start the new task.
if(nextDispatchVersion > recentReadVersion)
scheduledVersion = recentReadVersion + g_random->random01() * (nextDispatchVersion - recentReadVersion);
scheduledVersion = recentReadVersion + deterministicRandom()->random01() * (nextDispatchVersion - recentReadVersion);
// Range tasks during the initial snapshot should run at a higher priority
int priority = latestSnapshotEndVersion.present() ? 0 : 1;
@ -1862,7 +1862,7 @@ namespace fileBackup {
}
// Block size must be at least large enough for 1 max size key, 1 max size value, and overhead, so conservatively 125k.
state int blockSize = BUGGIFY ? g_random->randomInt(125e3, 4e6) : CLIENT_KNOBS->BACKUP_LOGFILE_BLOCK_SIZE;
state int blockSize = BUGGIFY ? deterministicRandom()->randomInt(125e3, 4e6) : CLIENT_KNOBS->BACKUP_LOGFILE_BLOCK_SIZE;
state Reference<IBackupFile> outFile = wait(bc->writeLogFile(beginVersion, endVersion, blockSize));
state LogFileWriter logFile(outFile, blockSize);
@ -2631,7 +2631,7 @@ namespace fileBackup {
state int start = 0;
state int end = data.size();
state int dataSizeLimit = BUGGIFY ? g_random->randomInt(256 * 1024, 10e6) : CLIENT_KNOBS->RESTORE_WRITE_TX_SIZE;
state int dataSizeLimit = BUGGIFY ? deterministicRandom()->randomInt(256 * 1024, 10e6) : CLIENT_KNOBS->RESTORE_WRITE_TX_SIZE;
tr->reset();
loop{
@ -2818,7 +2818,7 @@ namespace fileBackup {
state int start = 0;
state int end = data.size();
state int dataSizeLimit = BUGGIFY ? g_random->randomInt(256 * 1024, 10e6) : CLIENT_KNOBS->RESTORE_WRITE_TX_SIZE;
state int dataSizeLimit = BUGGIFY ? deterministicRandom()->randomInt(256 * 1024, 10e6) : CLIENT_KNOBS->RESTORE_WRITE_TX_SIZE;
tr->reset();
loop {
@ -3578,7 +3578,7 @@ public:
prevConfig.clear(tr);
}
state BackupConfig config(g_random->randomUniqueID());
state BackupConfig config(deterministicRandom()->randomUniqueID());
state UID uid = config.getUid();
// This check will ensure that current backupUid is later than the last backup Uid
@ -3631,7 +3631,7 @@ public:
if (existingDestUidValue.present()) {
destUidValue = existingDestUidValue.get();
} else {
destUidValue = BinaryWriter::toValue(g_random->randomUniqueID(), Unversioned());
destUidValue = BinaryWriter::toValue(deterministicRandom()->randomUniqueID(), Unversioned());
tr->set(destUidLookupPath, destUidValue);
}
}
@ -4288,7 +4288,7 @@ public:
//Lock src, record commit version
state Transaction tr(cx);
state Version commitVersion;
state UID randomUid = g_random->randomUniqueID();
state UID randomUid = deterministicRandom()->randomUniqueID();
loop {
try {
// We must get a commit version so add a conflict range that won't likely cause conflicts
@ -4368,7 +4368,7 @@ const int BackupAgentBase::logHeaderSize = 12;
const int FileBackupAgent::dataFooterSize = 20;
Future<Version> FileBackupAgent::restore(Database cx, Optional<Database> cxOrig, Key tagName, Key url, Standalone<VectorRef<KeyRangeRef>> ranges, bool waitForComplete, Version targetVersion, bool verbose, Key addPrefix, Key removePrefix, bool lockDB) {
return FileBackupAgentImpl::restore(this, cx, cxOrig, tagName, url, ranges, waitForComplete, targetVersion, verbose, addPrefix, removePrefix, lockDB, g_random->randomUniqueID());
return FileBackupAgentImpl::restore(this, cx, cxOrig, tagName, url, ranges, waitForComplete, targetVersion, verbose, addPrefix, removePrefix, lockDB, deterministicRandom()->randomUniqueID());
}
Future<Version> FileBackupAgent::atomicRestore(Database cx, Key tagName, Standalone<VectorRef<KeyRangeRef>> ranges, Key addPrefix, Key removePrefix) {

View File

@ -323,7 +323,7 @@ namespace HTTP {
try {
state std::string requestID;
if(!requestIDHeader.empty()) {
requestID = g_random->randomUniqueID().toString();
requestID = deterministicRandom()->randomUniqueID().toString();
requestID = requestID.insert(20, "-");
requestID = requestID.insert(16, "-");
requestID = requestID.insert(12, "-");

View File

@ -44,10 +44,10 @@ ClientKnobs::ClientKnobs(bool randomize) {
// wrong_shard_server sometimes comes from the only nonfailed server, so we need to avoid a fast spin
init( WRONG_SHARD_SERVER_DELAY, .01 ); if( randomize && BUGGIFY ) WRONG_SHARD_SERVER_DELAY = g_random->random01(); // FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY; // SOMEDAY: This delay can limit performance of retrieving data when the cache is mostly wrong (e.g. dumping the database after a test)
init( FUTURE_VERSION_RETRY_DELAY, .01 ); if( randomize && BUGGIFY ) FUTURE_VERSION_RETRY_DELAY = g_random->random01();// FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY;
init( WRONG_SHARD_SERVER_DELAY, .01 ); if( randomize && BUGGIFY ) WRONG_SHARD_SERVER_DELAY = deterministicRandom()->random01(); // FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY; // SOMEDAY: This delay can limit performance of retrieving data when the cache is mostly wrong (e.g. dumping the database after a test)
init( FUTURE_VERSION_RETRY_DELAY, .01 ); if( randomize && BUGGIFY ) FUTURE_VERSION_RETRY_DELAY = deterministicRandom()->random01();// FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY;
init( REPLY_BYTE_LIMIT, 80000 );
init( DEFAULT_BACKOFF, .01 ); if( randomize && BUGGIFY ) DEFAULT_BACKOFF = g_random->random01();
init( DEFAULT_BACKOFF, .01 ); if( randomize && BUGGIFY ) DEFAULT_BACKOFF = deterministicRandom()->random01();
init( DEFAULT_MAX_BACKOFF, 1.0 );
init( BACKOFF_GROWTH_RATE, 2.0 );
init( RESOURCE_CONSTRAINED_MAX_BACKOFF, 30.0 );
@ -185,8 +185,8 @@ ClientKnobs::ClientKnobs(bool randomize) {
init(CSI_SAMPLING_PROBABILITY, -1.0);
init(CSI_SIZE_LIMIT, std::numeric_limits<int64_t>::max());
if (randomize && BUGGIFY) {
CSI_SAMPLING_PROBABILITY = g_random->random01() / 10; // rand range 0 - 0.1
CSI_SIZE_LIMIT = g_random->randomInt(1024 * 1024, 100 * 1024 * 1024); // 1 MB - 100 MB
CSI_SAMPLING_PROBABILITY = deterministicRandom()->random01() / 10; // rand range 0 - 0.1
CSI_SIZE_LIMIT = deterministicRandom()->randomInt(1024 * 1024, 100 * 1024 * 1024); // 1 MB - 100 MB
}
init(CSI_STATUS_DELAY, 10.0 );
@ -194,5 +194,5 @@ ClientKnobs::ClientKnobs(bool randomize) {
init( CONSISTENCY_CHECK_ONE_ROUND_TARGET_COMPLETION_TIME, 7 * 24 * 60 * 60 ); // 7 days
// TLS related
init( CHECK_CONNECTED_COORDINATOR_NUM_DELAY, 1.0 ); if( randomize && BUGGIFY ) CHECK_CONNECTED_COORDINATOR_NUM_DELAY = g_random->random01() * 60.0; // In seconds
init( CHECK_CONNECTED_COORDINATOR_NUM_DELAY, 1.0 ); if( randomize && BUGGIFY ) CHECK_CONNECTED_COORDINATOR_NUM_DELAY = deterministicRandom()->random01() * 60.0; // In seconds
}

View File

@ -298,14 +298,14 @@ ACTOR Future<ConfigurationResult::Type> changeConfig( Database cx, std::map<std:
std::string initKey = configKeysPrefix.toString() + "initialized";
state bool creating = m.count( initKey ) != 0;
if (creating) {
m[initIdKey.toString()] = g_random->randomUniqueID().toString();
m[initIdKey.toString()] = deterministicRandom()->randomUniqueID().toString();
if (!isCompleteConfiguration(m)) {
return ConfigurationResult::INCOMPLETE_CONFIGURATION;
}
}
state Future<Void> tooLong = delay(4.5);
state Key versionKey = BinaryWriter::toValue(g_random->randomUniqueID(),Unversioned());
state Key versionKey = BinaryWriter::toValue(deterministicRandom()->randomUniqueID(),Unversioned());
state bool oldReplicationUsesDcId = false;
loop {
try {
@ -763,7 +763,7 @@ ConfigureAutoResult parseConfig( StatusObject const& status ) {
ACTOR Future<ConfigurationResult::Type> autoConfig( Database cx, ConfigureAutoResult conf ) {
state Transaction tr(cx);
state Key versionKey = BinaryWriter::toValue(g_random->randomUniqueID(),Unversioned());
state Key versionKey = BinaryWriter::toValue(deterministicRandom()->randomUniqueID(),Unversioned());
if(!conf.address_class.size())
return ConfigurationResult::INCOMPLETE_CONFIGURATION; //FIXME: correct return type
@ -790,7 +790,7 @@ ACTOR Future<ConfigurationResult::Type> autoConfig( Database cx, ConfigureAutoRe
}
if(conf.address_class.size())
tr.set(processClassChangeKey, g_random->randomUniqueID().toString());
tr.set(processClassChangeKey, deterministicRandom()->randomUniqueID().toString());
if(conf.auto_logs != conf.old_logs)
tr.set(configKeysPrefix.toString() + "auto_logs", format("%d", conf.auto_logs));
@ -950,7 +950,7 @@ ACTOR Future<CoordinatorsResult::Type> changeQuorum( Database cx, Reference<IQuo
if ( old.coordinators() == desiredCoordinators && old.clusterKeyName() == newName)
return retries ? CoordinatorsResult::SUCCESS : CoordinatorsResult::SAME_NETWORK_ADDRESSES;
state ClusterConnectionString conn( desiredCoordinators, StringRef( newName + ':' + g_random->randomAlphaNumeric( 32 ) ) );
state ClusterConnectionString conn( desiredCoordinators, StringRef( newName + ':' + deterministicRandom()->randomAlphaNumeric( 32 ) ) );
if(g_network->isSimulated()) {
for(int i = 0; i < (desiredCoordinators.size()/2)+1; i++) {
@ -1124,7 +1124,7 @@ struct AutoQuorumChange : IQuorumChange {
void addDesiredWorkers(vector<NetworkAddress>& chosen, const vector<ProcessData>& workers, int desiredCount, const std::set<AddressExclusion>& excluded) {
vector<ProcessData> remainingWorkers(workers);
g_random->randomShuffle(remainingWorkers);
deterministicRandom()->randomShuffle(remainingWorkers);
std::partition(remainingWorkers.begin(), remainingWorkers.end(), [](const ProcessData& data) { return (data.processClass == ProcessClass::CoordinatorClass); });
@ -1197,8 +1197,8 @@ Reference<IQuorumChange> autoQuorumChange( int desired ) { return Reference<IQuo
ACTOR Future<Void> excludeServers( Database cx, vector<AddressExclusion> servers ) {
state Transaction tr(cx);
state Key versionKey = BinaryWriter::toValue(g_random->randomUniqueID(),Unversioned());
state std::string excludeVersionKey = g_random->randomUniqueID().toString();
state Key versionKey = BinaryWriter::toValue(deterministicRandom()->randomUniqueID(),Unversioned());
state std::string excludeVersionKey = deterministicRandom()->randomUniqueID().toString();
loop {
try {
@ -1227,8 +1227,8 @@ ACTOR Future<Void> excludeServers( Database cx, vector<AddressExclusion> servers
ACTOR Future<Void> includeServers( Database cx, vector<AddressExclusion> servers ) {
state bool includeAll = false;
state Transaction tr(cx);
state Key versionKey = BinaryWriter::toValue(g_random->randomUniqueID(),Unversioned());
state std::string excludeVersionKey = g_random->randomUniqueID().toString();
state Key versionKey = BinaryWriter::toValue(deterministicRandom()->randomUniqueID(),Unversioned());
state std::string excludeVersionKey = deterministicRandom()->randomUniqueID().toString();
loop {
try {
@ -1301,7 +1301,7 @@ ACTOR Future<Void> setClass( Database cx, AddressExclusion server, ProcessClass
}
if(foundChange)
tr.set(processClassChangeKey, g_random->randomUniqueID().toString());
tr.set(processClassChangeKey, deterministicRandom()->randomUniqueID().toString());
wait( tr.commit() );
return Void();
@ -1851,7 +1851,7 @@ TEST_CASE("/ManagementAPI/AutoQuorumChange/checkLocality") {
workers.push_back(data);
}
auto noAssignIndex = g_random->randomInt(0, workers.size());
auto noAssignIndex = deterministicRandom()->randomInt(0, workers.size());
workers[noAssignIndex].processClass._class = ProcessClass::CoordinatorClass;
change.addDesiredWorkers(chosen, workers, 5, excluded);

View File

@ -242,19 +242,19 @@ TEST_CASE("/fdbclient/MonitorLeader/parseConnectionString/fuzz") {
std::string output("");
auto c=connectionString.begin();
while(c!=connectionString.end()) {
if(g_random->random01() < 0.1) // Add whitespace character
output += g_random->randomChoice(LiteralStringRef(" \t\n\r"));
if(g_random->random01() < 0.5) { // Add one of the input characters
if(deterministicRandom()->random01() < 0.1) // Add whitespace character
output += deterministicRandom()->randomChoice(LiteralStringRef(" \t\n\r"));
if(deterministicRandom()->random01() < 0.5) { // Add one of the input characters
output += *c;
++c;
}
if(g_random->random01() < 0.1) { // Add a comment block
if(deterministicRandom()->random01() < 0.1) { // Add a comment block
output += "#";
int charCount = g_random->randomInt(0, 20);
int charCount = deterministicRandom()->randomInt(0, 20);
for(int i = 0; i < charCount; i++) {
output += g_random->randomChoice(LiteralStringRef("asdfzxcv123345:!@#$#$&()<\"\' \t"));
output += deterministicRandom()->randomChoice(LiteralStringRef("asdfzxcv123345:!@#$#$&()<\"\' \t"));
}
output += g_random->randomChoice(LiteralStringRef("\n\r"));
output += deterministicRandom()->randomChoice(LiteralStringRef("\n\r"));
}
}

View File

@ -1364,11 +1364,11 @@ private:
struct FutureInfo {
FutureInfo() {
if(g_random->coinflip()) {
expectedValue = Error(g_random->randomInt(1, 100));
if(deterministicRandom()->coinflip()) {
expectedValue = Error(deterministicRandom()->randomInt(1, 100));
}
else {
expectedValue = g_random->randomInt(0, 100);
expectedValue = deterministicRandom()->randomInt(0, 100);
}
}
@ -1388,14 +1388,14 @@ struct FutureInfo {
FutureInfo createVarOnMainThread(bool canBeNever=true) {
FutureInfo f;
if(g_random->coinflip()) {
if(deterministicRandom()->coinflip()) {
f.future = onMainThread([f, canBeNever]() {
Future<Void> sleep ;
if(canBeNever && g_random->coinflip()) {
if(canBeNever && deterministicRandom()->coinflip()) {
sleep = Never();
}
else {
sleep = delay(0.1 * g_random->random01());
sleep = delay(0.1 * deterministicRandom()->random01());
}
if(f.expectedValue.isError()) {
@ -1417,7 +1417,7 @@ FutureInfo createVarOnMainThread(bool canBeNever=true) {
}
THREAD_FUNC setAbort(void *arg) {
threadSleep(0.1 * g_random->random01());
threadSleep(0.1 * deterministicRandom()->random01());
try {
((ThreadSingleAssignmentVar<Void>*)arg)->send(Void());
((ThreadSingleAssignmentVar<Void>*)arg)->delref();
@ -1430,7 +1430,7 @@ THREAD_FUNC setAbort(void *arg) {
}
THREAD_FUNC releaseMem(void *arg) {
threadSleep(0.1 * g_random->random01());
threadSleep(0.1 * deterministicRandom()->random01());
try {
// Must get for releaseMemory to work
((ThreadSingleAssignmentVar<int>*)arg)->get();
@ -1449,7 +1449,7 @@ THREAD_FUNC releaseMem(void *arg) {
}
THREAD_FUNC destroy(void *arg) {
threadSleep(0.1 * g_random->random01());
threadSleep(0.1 * deterministicRandom()->random01());
try {
((ThreadSingleAssignmentVar<int>*)arg)->cancel();
}
@ -1461,7 +1461,7 @@ THREAD_FUNC destroy(void *arg) {
}
THREAD_FUNC cancel(void *arg) {
threadSleep(0.1 * g_random->random01());
threadSleep(0.1 * deterministicRandom()->random01());
try {
((ThreadSingleAssignmentVar<int>*)arg)->addref();
destroy(arg);
@ -1533,8 +1533,8 @@ THREAD_FUNC runSingleAssignmentVarTest(void *arg) {
auto tfp = tf.future.extractPtr();
if(g_random->coinflip()) {
if(g_random->coinflip()) {
if(deterministicRandom()->coinflip()) {
if(deterministicRandom()->coinflip()) {
threads.push_back(g_network->startThread(releaseMem, tfp));
}
threads.push_back(g_network->startThread(cancel, tfp));
@ -1576,7 +1576,7 @@ struct AbortableTest {
auto newFuture = FutureInfo(abortableFuture(f.future, ThreadFuture<Void>(abort)), f.expectedValue, f.legalErrors);
if(!abort->isReady() && g_random->coinflip()) {
if(!abort->isReady() && deterministicRandom()->coinflip()) {
ASSERT(abort->status == ThreadSingleAssignmentVarBase::Unset);
newFuture.threads.push_back(g_network->startThread(setAbort, abort));
}
@ -1718,7 +1718,7 @@ struct FlatMapTest {
ASSERT(!f.expectedValue.isError() && f.expectedValue.get() == v.get());
}
if(mapFuture.expectedValue.isError() && g_random->coinflip()) {
if(mapFuture.expectedValue.isError() && deterministicRandom()->coinflip()) {
return ErrorOr<ThreadFuture<int>>(mapFuture.expectedValue.getError());
}
else {

View File

@ -56,7 +56,6 @@
#endif
#include "flow/actorcompiler.h" // This must be the last #include.
extern IRandom* trace_random;
extern const char* getHGVersion();
using std::make_pair;
@ -365,9 +364,9 @@ ACTOR static Future<Void> clientStatusUpdateActor(DatabaseContext *cx) {
state std::vector<TrInfoChunk> trChunksQ;
for (auto &entry : cx->clientStatusUpdater.outStatusQ) {
auto &bw = entry.second;
int64_t value_size_limit = BUGGIFY ? g_random->randomInt(1e3, CLIENT_KNOBS->VALUE_SIZE_LIMIT) : CLIENT_KNOBS->VALUE_SIZE_LIMIT;
int64_t value_size_limit = BUGGIFY ? deterministicRandom()->randomInt(1e3, CLIENT_KNOBS->VALUE_SIZE_LIMIT) : CLIENT_KNOBS->VALUE_SIZE_LIMIT;
int num_chunks = (bw.getLength() + value_size_limit - 1) / value_size_limit;
std::string random_id = g_random->randomAlphaNumeric(16);
std::string random_id = deterministicRandom()->randomAlphaNumeric(16);
std::string user_provided_id = entry.first.size() ? entry.first + "/" : "";
for (int i = 0; i < num_chunks; i++) {
TrInfoChunk chunk;
@ -387,7 +386,7 @@ ACTOR static Future<Void> clientStatusUpdateActor(DatabaseContext *cx) {
}
// Commit the chunks splitting into different transactions if needed
state int64_t dataSizeLimit = BUGGIFY ? g_random->randomInt(200e3, 1.5 * CLIENT_KNOBS->TRANSACTION_SIZE_LIMIT) : 0.8 * CLIENT_KNOBS->TRANSACTION_SIZE_LIMIT;
state int64_t dataSizeLimit = BUGGIFY ? deterministicRandom()->randomInt(200e3, 1.5 * CLIENT_KNOBS->TRANSACTION_SIZE_LIMIT) : 0.8 * CLIENT_KNOBS->TRANSACTION_SIZE_LIMIT;
state std::vector<TrInfoChunk>::iterator tracking_iter = trChunksQ.begin();
tr = Transaction(Database(Reference<DatabaseContext>::addRef(cx)));
ASSERT(commitQ.empty() && (txBytes == 0));
@ -432,7 +431,7 @@ ACTOR static Future<Void> clientStatusUpdateActor(DatabaseContext *cx) {
cx->clientStatusUpdater.outStatusQ.clear();
double clientSamplingProbability = std::isinf(cx->clientInfo->get().clientTxnInfoSampleRate) ? CLIENT_KNOBS->CSI_SAMPLING_PROBABILITY : cx->clientInfo->get().clientTxnInfoSampleRate;
int64_t clientTxnInfoSizeLimit = cx->clientInfo->get().clientTxnInfoSizeLimit == -1 ? CLIENT_KNOBS->CSI_SIZE_LIMIT : cx->clientInfo->get().clientTxnInfoSizeLimit;
if (!trChunksQ.empty() && g_random->random01() < clientSamplingProbability)
if (!trChunksQ.empty() && deterministicRandom()->random01() < clientSamplingProbability)
wait(delExcessClntTxnEntriesActor(&tr, clientTxnInfoSizeLimit));
// tr is destructed because it hold a reference to DatabaseContext which creates a cycle mentioned above.
@ -1001,10 +1000,8 @@ void setupNetwork(uint64_t transportId, bool useMetrics) {
if( g_network )
throw network_already_setup();
g_random = new DeterministicRandom( platform::getRandomSeed() );
trace_random = new DeterministicRandom( platform::getRandomSeed() );
g_nondeterministic_random = trace_random;
g_debug_random = trace_random;
setThreadLocalDeterministicRandomSeed(platform::getRandomSeed());
if (!networkOptions.logClientInfo.present())
networkOptions.logClientInfo = true;
@ -1326,10 +1323,10 @@ ACTOR Future<Optional<Value>> getValue( Future<Version> version, Key key, Databa
state uint64_t startTime;
state double startTimeD;
try {
//GetValueReply r = wait( g_random->randomChoice( ssi->get() ).getValue.getReply( GetValueRequest(key,ver) ) );
//GetValueReply r = wait( deterministicRandom()->randomChoice( ssi->get() ).getValue.getReply( GetValueRequest(key,ver) ) );
//return r.value;
if( info.debugID.present() ) {
getValueID = g_nondeterministic_random->randomUniqueID();
getValueID = nondeterministicRandom()->randomUniqueID();
g_traceBatch.addAttach("GetValueAttachID", info.debugID.get().first(), getValueID.get().first());
g_traceBatch.addEvent("GetValueDebug", getValueID.get().first(), "NativeAPI.getValue.Before"); //.detail("TaskID", g_network->getCurrentTask());
@ -1464,7 +1461,7 @@ ACTOR Future< Void > watchValue( Future<Version> version, Key key, Optional<Valu
try {
state Optional<UID> watchValueID = Optional<UID>();
if( info.debugID.present() ) {
watchValueID = g_nondeterministic_random->randomUniqueID();
watchValueID = nondeterministicRandom()->randomUniqueID();
g_traceBatch.addAttach("WatchValueAttachID", info.debugID.get().first(), watchValueID.get().first());
g_traceBatch.addEvent("WatchValueDebug", watchValueID.get().first(), "NativeAPI.watchValue.Before"); //.detail("TaskID", g_network->getCurrentTask());
@ -1868,7 +1865,7 @@ ACTOR Future<Standalone<RangeResultRef>> getRange( Database cx, Reference<Transa
if( BUGGIFY && limits.hasByteLimit() && output.size() > std::max(1, originalLimits.minRows) ) {
output.more = true;
output.resize(output.arena(), g_random->randomInt(std::max(1,originalLimits.minRows),output.size()));
output.resize(output.arena(), deterministicRandom()->randomInt(std::max(1,originalLimits.minRows),output.size()));
getRangeFinished(trLogInfo, startTime, originalBegin, originalEnd, snapshot, conflictRange, reverse, output);
return output;
}
@ -2236,7 +2233,7 @@ void Transaction::addReadConflictRange( KeyRangeRef const& keys ) {
void Transaction::makeSelfConflicting() {
BinaryWriter wr(Unversioned());
wr.serializeBytes(LiteralStringRef("\xFF/SC/"));
wr << g_random->randomUniqueID();
wr << deterministicRandom()->randomUniqueID();
auto r = singleKeyRange( wr.toValue(), tr.arena );
tr.transaction.read_conflict_ranges.push_back( tr.arena, r );
tr.transaction.write_conflict_ranges.push_back( tr.arena, r );
@ -2351,7 +2348,7 @@ void Transaction::addWriteConflictRange( const KeyRangeRef& keys ) {
}
double Transaction::getBackoff(int errCode) {
double b = backoff * g_random->random01();
double b = backoff * deterministicRandom()->random01();
backoff = errCode == error_code_proxy_memory_limit_exceeded ? std::min(backoff * CLIENT_KNOBS->BACKOFF_GROWTH_RATE, CLIENT_KNOBS->RESOURCE_CONSTRAINED_MAX_BACKOFF) :
std::min(backoff * CLIENT_KNOBS->BACKOFF_GROWTH_RATE, options.maxBackoff);
return b;
@ -2454,7 +2451,7 @@ ACTOR void checkWrites( Database cx, Future<Void> committed, Promise<Void> outCo
return;
}
wait( delay( g_random->random01() ) ); // delay between 0 and 1 seconds
wait( delay( deterministicRandom()->random01() ) ); // delay between 0 and 1 seconds
//Future<Optional<Version>> version, Database cx, CommitTransactionRequest req ) {
state KeyRangeMap<MutationBlock> expectedValues;
@ -2572,7 +2569,7 @@ ACTOR static Future<Void> tryCommit( Database cx, Reference<TransactionLogInfo>
startTime = now();
state Optional<UID> commitID = Optional<UID>();
if(info.debugID.present()) {
commitID = g_nondeterministic_random->randomUniqueID();
commitID = nondeterministicRandom()->randomUniqueID();
g_traceBatch.addAttach("CommitAttachID", info.debugID.get().first(), commitID.get().first());
g_traceBatch.addEvent("CommitDebug", commitID.get().first(), "NativeAPI.commit.Before");
}
@ -2701,7 +2698,7 @@ Future<Void> Transaction::commitMutations() {
if( !readVersion.isValid() )
getReadVersion( GetReadVersionRequest::FLAG_CAUSAL_READ_RISKY ); // sets up readVersion field. We had no reads, so no need for (expensive) full causal consistency.
bool isCheckingWrites = options.checkWritesEnabled && g_random->random01() < 0.01;
bool isCheckingWrites = options.checkWritesEnabled && deterministicRandom()->random01() < 0.01;
for(int i=0; i<extraConflictRanges.size(); i++)
if (extraConflictRanges[i].isReady() && extraConflictRanges[i].get().first < extraConflictRanges[i].get().second )
tr.transaction.read_conflict_ranges.push_back( tr.arena, KeyRangeRef(extraConflictRanges[i].get().first, extraConflictRanges[i].get().second) );
@ -2715,7 +2712,7 @@ Future<Void> Transaction::commitMutations() {
}
if ( options.debugDump ) {
UID u = g_nondeterministic_random->randomUniqueID();
UID u = nondeterministicRandom()->randomUniqueID();
TraceEvent("TransactionDump", u);
for(auto i=tr.transaction.mutations.begin(); i!=tr.transaction.mutations.end(); ++i)
TraceEvent("TransactionMutation", u).detail("T", i->type).detail("P1", i->param1).detail("P2", i->param2);
@ -2940,7 +2937,7 @@ ACTOR Future<Void> readVersionBatcher( DatabaseContext *cx, FutureStream< std::p
when(std::pair< Promise<GetReadVersionReply>, Optional<UID> > req = waitNext(versionStream)) {
if (req.second.present()) {
if (!debugID.present())
debugID = g_nondeterministic_random->randomUniqueID();
debugID = nondeterministicRandom()->randomUniqueID();
g_traceBatch.addAttach("TransactionAttachID", req.second.get().first(), debugID.get().first());
}
requests.push_back(req.first);
@ -3246,7 +3243,7 @@ void Transaction::checkDeferredError() { cx->checkDeferredError(); }
Reference<TransactionLogInfo> Transaction::createTrLogInfoProbabilistically(const Database &cx) {
if(!cx->isError()) {
double clientSamplingProbability = std::isinf(cx->clientInfo->get().clientTxnInfoSampleRate) ? CLIENT_KNOBS->CSI_SAMPLING_PROBABILITY : cx->clientInfo->get().clientTxnInfoSampleRate;
if (((networkOptions.logClientInfo.present() && networkOptions.logClientInfo.get()) || BUGGIFY) && g_random->random01() < clientSamplingProbability && (!g_network->isSimulated() || !g_simulator.speedUpSimulation)) {
if (((networkOptions.logClientInfo.present() && networkOptions.logClientInfo.get()) || BUGGIFY) && deterministicRandom()->random01() < clientSamplingProbability && (!g_network->isSimulated() || !g_simulator.speedUpSimulation)) {
return Reference<TransactionLogInfo>(new TransactionLogInfo(TransactionLogInfo::DATABASE));
}
}

View File

@ -569,7 +569,7 @@ TEST_CASE("/fdbclient/WriteMap/random") {
KeyRangeMap<bool> unreadableMap;
for (int i = 0; i < 100; i++) {
int r = g_random->randomInt(0, 10);
int r = deterministicRandom()->randomInt(0, 10);
if (r == 0) {
KeyRangeRef range = RandomTestImpl::getRandomRange(arena);
writes.addConflictRange(range);
@ -586,7 +586,7 @@ TEST_CASE("/fdbclient/WriteMap/random") {
TraceEvent("RWMT_AddUnmodifiedAndUnreadableRange").detail("Range", range);
}
else if (r == 2) {
bool addConflict = g_random->random01() < 0.5;
bool addConflict = deterministicRandom()->random01() < 0.5;
KeyRangeRef range = RandomTestImpl::getRandomRange(arena);
writes.clear(range, addConflict);
setMap.erase(setMap.lower_bound(range.begin), setMap.lower_bound(range.end));
@ -597,7 +597,7 @@ TEST_CASE("/fdbclient/WriteMap/random") {
TraceEvent("RWMT_Clear").detail("Range", range).detail("AddConflict", addConflict);
}
else if (r == 3) {
bool addConflict = g_random->random01() < 0.5;
bool addConflict = deterministicRandom()->random01() < 0.5;
KeyRef key = RandomTestImpl::getRandomKey(arena);
ValueRef value = RandomTestImpl::getRandomValue(arena);
writes.mutate(key, MutationRef::SetVersionstampedValue, value, addConflict);
@ -609,7 +609,7 @@ TEST_CASE("/fdbclient/WriteMap/random") {
TraceEvent("RWMT_SetVersionstampedValue").detail("Key", key).detail("Value", value.size()).detail("AddConflict", addConflict);
}
else if (r == 4) {
bool addConflict = g_random->random01() < 0.5;
bool addConflict = deterministicRandom()->random01() < 0.5;
KeyRef key = RandomTestImpl::getRandomKey(arena);
ValueRef value = RandomTestImpl::getRandomValue(arena);
writes.mutate(key, MutationRef::SetVersionstampedKey, value, addConflict);
@ -621,7 +621,7 @@ TEST_CASE("/fdbclient/WriteMap/random") {
TraceEvent("RWMT_SetVersionstampedKey").detail("Key", key).detail("Value", value.size()).detail("AddConflict", addConflict);
}
else if (r == 5) {
bool addConflict = g_random->random01() < 0.5;
bool addConflict = deterministicRandom()->random01() < 0.5;
KeyRef key = RandomTestImpl::getRandomKey(arena);
ValueRef value = RandomTestImpl::getRandomValue(arena);
writes.mutate(key, MutationRef::And, value, addConflict);
@ -641,7 +641,7 @@ TEST_CASE("/fdbclient/WriteMap/random") {
TraceEvent("RWMT_And").detail("Key", key).detail("Value", value.size()).detail("AddConflict", addConflict);
}
else {
bool addConflict = g_random->random01() < 0.5;
bool addConflict = deterministicRandom()->random01() < 0.5;
KeyRef key = RandomTestImpl::getRandomKey(arena);
ValueRef value = RandomTestImpl::getRandomValue(arena);
writes.mutate(key, MutationRef::SetValue, value, addConflict);

View File

@ -76,14 +76,14 @@ private:
class RandomTestImpl {
public:
static ValueRef getRandomValue(Arena& arena) {
return ValueRef(arena, std::string(g_random->randomInt(0, 1000), 'x'));
return ValueRef(arena, std::string(deterministicRandom()->randomInt(0, 1000), 'x'));
}
static ValueRef getRandomVersionstampValue(Arena& arena) {
int len = g_random->randomInt(10, 98);
int len = deterministicRandom()->randomInt(10, 98);
std::string value = std::string(len, 'x');
int32_t pos = g_random->randomInt(0, len - 9);
if (g_random->random01() < 0.01) {
int32_t pos = deterministicRandom()->randomInt(0, len - 9);
if (deterministicRandom()->random01() < 0.01) {
pos = value.size() - 10;
}
pos = littleEndian32(pos);
@ -92,26 +92,26 @@ public:
}
static ValueRef getRandomVersionstampKey(Arena& arena) {
int idx = g_random->randomInt(0, 100);
int idx = deterministicRandom()->randomInt(0, 100);
std::string key = format("%010d", idx / 3);
if (idx % 3 >= 1)
key += '\x00';
if (idx % 3 >= 2)
key += '\x00';
int32_t pos = key.size() - g_random->randomInt(0, 3);
if (g_random->random01() < 0.01) {
int32_t pos = key.size() - deterministicRandom()->randomInt(0, 3);
if (deterministicRandom()->random01() < 0.01) {
pos = 0;
}
key = key.substr(0, pos);
key += "XXXXXXXXYY";
key += std::string(g_random->randomInt(0, 3), 'z');
key += std::string(deterministicRandom()->randomInt(0, 3), 'z');
pos = littleEndian32(pos);
key += std::string((const char*)&pos, sizeof(int32_t));
return ValueRef(arena, key);
}
static KeyRef getRandomKey(Arena& arena) {
return getKeyForIndex(arena, g_random->randomInt(0, 100));
return getKeyForIndex(arena, deterministicRandom()->randomInt(0, 100));
}
static KeyRef getKeyForIndex(Arena& arena, int idx) {
@ -124,14 +124,14 @@ public:
}
static KeyRangeRef getRandomRange(Arena& arena) {
int startLocation = g_random->randomInt(0, 100);
int endLocation = startLocation + g_random->randomInt(1, 1 + 100 - startLocation);
int startLocation = deterministicRandom()->randomInt(0, 100);
int endLocation = startLocation + deterministicRandom()->randomInt(1, 1 + 100 - startLocation);
return KeyRangeRef(getKeyForIndex(arena, startLocation), getKeyForIndex(arena, endLocation));
}
static KeySelectorRef getRandomKeySelector(Arena& arena) {
return KeySelectorRef(getRandomKey(arena), g_random->random01() < 0.5, g_random->randomInt(-10, 10));
return KeySelectorRef(getRandomKey(arena), deterministicRandom()->random01() < 0.5, deterministicRandom()->randomInt(-10, 10));
}
};

View File

@ -472,7 +472,7 @@ public:
state bool readThroughEnd = false;
state int actualBeginOffset = begin.offset;
state int actualEndOffset = end.offset;
//state UID randomID = g_nondeterministic_random->randomUniqueID();
//state UID randomID = nondeterministicRandom()->randomUniqueID();
resolveKeySelectorFromCache( begin, it, ryw->getMaxReadKey(), &readToBegin, &readThroughEnd, &actualBeginOffset );
resolveKeySelectorFromCache( end, itEnd, ryw->getMaxReadKey(), &readToBegin, &readThroughEnd, &actualEndOffset );
@ -745,7 +745,7 @@ public:
state bool readThroughEnd = false;
state int actualBeginOffset = begin.offset;
state int actualEndOffset = end.offset;
//state UID randomID = g_nondeterministic_random->randomUniqueID();
//state UID randomID = nondeterministicRandom()->randomUniqueID();
resolveKeySelectorFromCache( end, it, ryw->getMaxReadKey(), &readToBegin, &readThroughEnd, &actualEndOffset );
resolveKeySelectorFromCache( begin, itEnd, ryw->getMaxReadKey(), &readToBegin, &readThroughEnd, &actualBeginOffset );

View File

@ -61,7 +61,7 @@ struct StorageServerInterface {
RequestStream<struct WatchValueRequest> watchValue;
explicit StorageServerInterface(UID uid) : uniqueID( uid ) {}
StorageServerInterface() : uniqueID( g_random->randomUniqueID() ) {}
StorageServerInterface() : uniqueID( deterministicRandom()->randomUniqueID() ) {}
NetworkAddress address() const { return getVersion.getEndpoint().getPrimaryAddress(); }
UID id() const { return uniqueID; }
std::string toString() const { return id().shortString(); }

View File

@ -131,7 +131,7 @@ unsigned int Task::getPriority() const {
class TaskBucketImpl {
public:
ACTOR static Future<Optional<Key>> getTaskKey(Reference<ReadYourWritesTransaction> tr, Reference<TaskBucket> taskBucket, int priority = 0) {
Standalone<StringRef> uid = StringRef(g_random->randomUniqueID().toString());
Standalone<StringRef> uid = StringRef(deterministicRandom()->randomUniqueID().toString());
// Get keyspace for the specified priority level
state Subspace space = taskBucket->getAvailableSpace(priority);
@ -161,7 +161,7 @@ public:
// give it some chances for the timed out tasks to get into the task loop in the case of
// many other new tasks get added so that the timed out tasks never get chances to re-run
if (g_random->random01() < CLIENT_KNOBS->TASKBUCKET_CHECK_TIMEOUT_CHANCE) {
if (deterministicRandom()->random01() < CLIENT_KNOBS->TASKBUCKET_CHECK_TIMEOUT_CHANCE) {
bool anyTimeouts = wait(requeueTimedOutTasks(tr, taskBucket));
TEST(anyTimeouts); // Found a task that timed out
}
@ -213,7 +213,7 @@ public:
state Standalone<RangeResultRef> values = wait(tr->getRange(taskAvailableSpace.range(), CLIENT_KNOBS->TOO_MANY));
Version version = wait(tr->getReadVersion());
task->timeoutVersion = version + (uint64_t)(taskBucket->timeout * (CLIENT_KNOBS->TASKBUCKET_TIMEOUT_JITTER_OFFSET + CLIENT_KNOBS->TASKBUCKET_TIMEOUT_JITTER_RANGE * g_random->random01()));
task->timeoutVersion = version + (uint64_t)(taskBucket->timeout * (CLIENT_KNOBS->TASKBUCKET_TIMEOUT_JITTER_OFFSET + CLIENT_KNOBS->TASKBUCKET_TIMEOUT_JITTER_RANGE * deterministicRandom()->random01()));
Subspace timeoutSpace = taskBucket->timeouts.get(task->timeoutVersion).get(taskUID);
@ -225,7 +225,7 @@ public:
// Clear task definition in the available keyspace
tr->clear(taskAvailableSpace.range());
tr->set(taskBucket->active.key(), g_random->randomUniqueID().toString());
tr->set(taskBucket->active.key(), deterministicRandom()->randomUniqueID().toString());
return task;
}
@ -327,7 +327,7 @@ public:
state FlowLock::Releaser releaser;
// Wait until we are half way to the timeout version of this task
wait(delay(0.8 * (BUGGIFY ? (2 * g_random->random01()) : 1.0) * (double)(task->timeoutVersion - (uint64_t)versionNow) / CLIENT_KNOBS->CORE_VERSIONSPERSECOND));
wait(delay(0.8 * (BUGGIFY ? (2 * deterministicRandom()->random01()) : 1.0) * (double)(task->timeoutVersion - (uint64_t)versionNow) / CLIENT_KNOBS->CORE_VERSIONSPERSECOND));
// Take the extendMutex lock until we either succeed or stop trying to extend due to failure
wait(task->extendMutex.take());
@ -464,7 +464,7 @@ public:
// Wait for a task to be done. Also, if we have any slots available then stop waiting after pollDelay at the latest.
Future<Void> w = ready(waitForAny(tasks));
if(!availableSlots.empty())
w = w || delay(*pollDelay * (0.9 + g_random->random01() / 5)); // Jittered by 20 %, so +/- 10%
w = w || delay(*pollDelay * (0.9 + deterministicRandom()->random01() / 5)); // Jittered by 20 %, so +/- 10%
wait(w);
// Check all of the task slots, any that are finished should be replaced with Never() and their slots added back to availableSlots
@ -812,7 +812,7 @@ Future<Void> TaskBucket::changePause(Reference<ReadYourWritesTransaction> tr, bo
Key TaskBucket::addTask(Reference<ReadYourWritesTransaction> tr, Reference<Task> task) {
setOptions(tr);
Key key(g_random->randomUniqueID().toString());
Key key(deterministicRandom()->randomUniqueID().toString());
Subspace taskSpace;
@ -998,7 +998,7 @@ public:
ACTOR static Future<Void> _join(Reference<ReadYourWritesTransaction> tr, Reference<TaskBucket> taskBucket, Reference<TaskFuture> taskFuture, std::vector<Reference<TaskFuture>> vectorFuture) {
std::vector<Future<Void>> onSetFutures;
for (int i = 0; i < vectorFuture.size(); ++i) {
Key key = StringRef(g_random->randomUniqueID().toString());
Key key = StringRef(deterministicRandom()->randomUniqueID().toString());
taskFuture->addBlock(tr, key);
Reference<Task> task(new Task());
task->params[Task::reservedTaskParamKeyType] = LiteralStringRef("UnblockFuture");
@ -1033,7 +1033,7 @@ public:
}
else {
TEST(true); // is_set == false
Subspace callbackSpace = taskFuture->callbacks.get(StringRef(g_random->randomUniqueID().toString()));
Subspace callbackSpace = taskFuture->callbacks.get(StringRef(deterministicRandom()->randomUniqueID().toString()));
for (auto & v : task->params) {
tr->set(callbackSpace.pack(v.key), v.value);
}
@ -1155,7 +1155,7 @@ TaskFuture::TaskFuture(const Reference<FutureBucket> bucket, Key k)
: futureBucket(bucket), key(k)
{
if (k.size() == 0) {
key = g_random->randomUniqueID().toString();
key = deterministicRandom()->randomUniqueID().toString();
}
prefix = futureBucket->prefix.get(key);

View File

@ -58,7 +58,7 @@ namespace PTreeImpl {
Reference<PTree> right(Version at) const { return child(true, at); }
PTree(const T& data, Version ver) : data(data), lastUpdateVersion(ver), updated(false) {
priority = g_random->randomUInt32();
priority = deterministicRandom()->randomUInt32();
}
PTree( uint32_t pri, T const& data, Reference<PTree> const& left, Reference<PTree> const& right, Version ver ) : priority(pri), data(data), lastUpdateVersion(ver), updated(false) {
pointer[0] = left; pointer[1] = right;

View File

@ -60,7 +60,7 @@ struct EvictablePageCache : ReferenceCounted<EvictablePageCache> {
void try_evict() {
if (pages.size() >= (uint64_t)maxPages && !pages.empty()) {
for (int i = 0; i < FLOW_KNOBS->MAX_EVICT_ATTEMPTS; i++) { // If we don't manage to evict anything, just go ahead and exceed the cache limit
int toEvict = g_random->randomInt(0, pages.size());
int toEvict = deterministicRandom()->randomInt(0, pages.size());
if (pages[toEvict]->evict())
break;
}

View File

@ -281,7 +281,7 @@ public:
result = ftruncate(fd, size);
double end = timer_monotonic();
if(g_nondeterministic_random->random01() < end-begin) {
if(nondeterministicRandom()->random01() < end-begin) {
TraceEvent("SlowKAIOTruncate")
.detail("TruncateTime", end - begin)
.detail("TruncateBytes", size - lastFileSize);
@ -402,7 +402,7 @@ public:
ctx.slowAioSubmitMetric->largestTruncate = largestTruncate;
ctx.slowAioSubmitMetric->log();
if(g_nondeterministic_random->random01() < end-begin) {
if(nondeterministicRandom()->random01() < end-begin) {
TraceEvent("SlowKAIOLaunch")
.detail("IOSubmitTime", end-truncateComplete)
.detail("TruncateTime", truncateComplete-begin)
@ -765,13 +765,13 @@ ACTOR Future<Void> runTestOps(Reference<IAsyncFile> f, int numIterations, int fi
for(; iteration < numIterations; ++iteration) {
state std::vector<Future<Void>> futures;
state int numOps = g_random->randomInt(1, 20);
state int numOps = deterministicRandom()->randomInt(1, 20);
for(; numOps > 0; --numOps) {
if(g_random->coinflip()) {
futures.push_back(success(f->read(buf, 4096, g_random->randomInt(0, fileSize)/4096*4096)));
if(deterministicRandom()->coinflip()) {
futures.push_back(success(f->read(buf, 4096, deterministicRandom()->randomInt(0, fileSize)/4096*4096)));
}
else {
futures.push_back(f->write(buf, 4096, g_random->randomInt(0, fileSize)/4096*4096));
futures.push_back(f->write(buf, 4096, deterministicRandom()->randomInt(0, fileSize)/4096*4096));
}
}
state int fIndex = 0;

View File

@ -179,7 +179,7 @@ private:
//This is only designed to work in simulation
ASSERT(g_network->isSimulated());
this->id = g_random->randomUniqueID();
this->id = deterministicRandom()->randomUniqueID();
//TraceEvent("AsyncFileNonDurable_Create", id).detail("Filename", filename);
this->file = file;
@ -188,7 +188,7 @@ private:
maxWriteDelay = 5.0;
hasBeenSynced = false;
killMode = (KillMode)g_random->randomInt(1, 3);
killMode = (KillMode)deterministicRandom()->randomInt(1, 3);
//TraceEvent("AsyncFileNonDurable_CreateEnd", id).detail("Filename", filename).backtrace();
}
@ -414,7 +414,7 @@ private:
state int currentTaskID = g_network->getCurrentTask();
wait( g_simulator.onMachine( currentProcess ) );
state double delayDuration = g_random->random01() * self->maxWriteDelay;
state double delayDuration = deterministicRandom()->random01() * self->maxWriteDelay;
state Standalone<StringRef> dataCopy(StringRef((uint8_t*)data, length));
state Future<bool> startSyncFuture = self->startSyncPromise.getFuture();
@ -427,7 +427,7 @@ private:
std::vector<Future<Void>> priorModifications = self->getModificationsAndInsert(offset, length, true, writeEnded);
if(BUGGIFY_WITH_PROB(0.001))
priorModifications.push_back(delay(g_random->random01() * FLOW_KNOBS->MAX_PRIOR_MODIFICATION_DELAY) || self->killed.getFuture());
priorModifications.push_back(delay(deterministicRandom()->random01() * FLOW_KNOBS->MAX_PRIOR_MODIFICATION_DELAY) || self->killed.getFuture());
else
priorModifications.push_back(waitUntilDiskReady(self->diskParameters, length) || self->killed.getFuture());
@ -466,25 +466,25 @@ private:
vector<Future<Void>> writeFutures;
for(int writeOffset = 0; writeOffset < length; writeOffset += pageLength) {
//choose a random action to perform on this page write (write correctly, corrupt, or don't write)
KillMode pageKillMode = (KillMode)g_random->randomInt(0, self->killMode + 1);
KillMode pageKillMode = (KillMode)deterministicRandom()->randomInt(0, self->killMode + 1);
for(int pageOffset = 0; pageOffset < pageLength; pageOffset += sectorLength) {
//If saving durable, then perform the write correctly. Otherwise, perform the write correcly with a probability of 1/3.
//If corrupting the write, then this sector will be written correctly with a 1/4 chance
if(saveDurable || pageKillMode == NO_CORRUPTION || (pageKillMode == FULL_CORRUPTION && g_random->random01() < 0.25)) {
if(saveDurable || pageKillMode == NO_CORRUPTION || (pageKillMode == FULL_CORRUPTION && deterministicRandom()->random01() < 0.25)) {
//if (!saveDurable) TraceEvent(SevInfo, "AsyncFileNonDurableWrite", self->id).detail("Filename", self->filename).detail("Offset", offset+writeOffset+pageOffset).detail("Length", sectorLength);
writeFutures.push_back(self->file->write(dataCopy.begin() + writeOffset + pageOffset, sectorLength, offset + writeOffset + pageOffset));
}
//If the write is not durable, then the write will either be corrupted or not written at all. If corrupted, there is 1/4 chance that a given
//sector will not be written
else if(pageKillMode == FULL_CORRUPTION && g_random->random01() < 0.66667) {
else if(pageKillMode == FULL_CORRUPTION && deterministicRandom()->random01() < 0.66667) {
//The incorrect part of the write can be the rightmost bytes (side = 0), the leftmost bytes (side = 1), or the entire write (side = 2)
int side = g_random->randomInt(0, 3);
int side = deterministicRandom()->randomInt(0, 3);
//There is a 1/2 chance that a bad write will have garbage written into its bad portion
//The chance is increased to 1 if the entire write is bad
bool garbage = side == 2 || g_random->random01() < 0.5;
bool garbage = side == 2 || deterministicRandom()->random01() < 0.5;
int64_t goodStart = 0;
int64_t goodEnd = sectorLength;
@ -492,11 +492,11 @@ private:
int64_t badEnd = sectorLength;
if(side == 0) {
goodEnd = g_random->randomInt(0, sectorLength);
goodEnd = deterministicRandom()->randomInt(0, sectorLength);
badStart = goodEnd;
}
else if(side == 1) {
badEnd = g_random->randomInt(0, sectorLength);
badEnd = deterministicRandom()->randomInt(0, sectorLength);
goodStart = badEnd;
}
else
@ -506,7 +506,7 @@ private:
if(garbage && badStart != badEnd) {
uint8_t *badData = const_cast<uint8_t*>(&dataCopy.begin()[badStart + writeOffset + pageOffset]);
for(int i = 0; i < badEnd - badStart; i += sizeof(uint32_t)) {
uint32_t val = g_random->randomUInt32();
uint32_t val = deterministicRandom()->randomUInt32();
memcpy(&badData[i], &val, std::min(badEnd - badStart - i, (int64_t)sizeof(uint32_t)));
}
@ -538,7 +538,7 @@ private:
state int currentTaskID = g_network->getCurrentTask();
wait( g_simulator.onMachine( currentProcess ) );
state double delayDuration = g_random->random01() * self->maxWriteDelay;
state double delayDuration = deterministicRandom()->random01() * self->maxWriteDelay;
state Future<bool> startSyncFuture = self->startSyncPromise.getFuture();
try {
@ -549,7 +549,7 @@ private:
std::vector<Future<Void>> priorModifications = self->getModificationsAndInsert(size, -1, true, truncateEnded);
if(BUGGIFY_WITH_PROB(0.001))
priorModifications.push_back(delay(g_random->random01() * FLOW_KNOBS->MAX_PRIOR_MODIFICATION_DELAY) || self->killed.getFuture());
priorModifications.push_back(delay(deterministicRandom()->random01() * FLOW_KNOBS->MAX_PRIOR_MODIFICATION_DELAY) || self->killed.getFuture());
else
priorModifications.push_back(waitUntilDiskReady(self->diskParameters, 0) || self->killed.getFuture());
@ -578,7 +578,7 @@ private:
}
//If performing a durable truncate, then pass it through to the file. Otherwise, pass it through with a 1/2 chance
if(saveDurable || self->killMode == NO_CORRUPTION || g_random->random01() < 0.5)
if(saveDurable || self->killMode == NO_CORRUPTION || deterministicRandom()->random01() < 0.5)
wait(self->file->truncate(size));
else {
TraceEvent("AsyncFileNonDurable_DroppedTruncate", self->id).detail("Size", size);
@ -630,7 +630,7 @@ private:
self->startSyncPromise = Promise<bool>();
//Writes will be durable in a kill with a 10% probability
state bool writeDurable = durable || g_random->random01() < 0.1;
state bool writeDurable = durable || deterministicRandom()->random01() < 0.1;
startSyncPromise.send(writeDurable);
//Wait for outstanding writes to complete
@ -642,7 +642,7 @@ private:
if(!durable) {
//Sometimes sync the file if writes were made durably. Before a file is first synced, it is stored in a temporary file and then renamed to the correct
//location once sync is called. By not calling sync, we simulate a failure to fsync the directory storing the file
if(self->hasBeenSynced && writeDurable && g_random->random01() < 0.5) {
if(self->hasBeenSynced && writeDurable && deterministicRandom()->random01() < 0.5) {
TEST(true); //AsyncFileNonDurable kill was durable and synced
wait(success(errorOr(self->file->sync())));
}

View File

@ -41,8 +41,8 @@ public:
if( populationSize <= sampleSize ) {
samples.push_back( sample );
} else if( g_random->random01() < ( (double)sampleSize / populationSize ) ) {
samples[ g_random->randomInt( 0, sampleSize ) ] = sample;
} else if( deterministicRandom()->random01() < ( (double)sampleSize / populationSize ) ) {
samples[ deterministicRandom()->randomInt( 0, sampleSize ) ] = sample;
}
_max = std::max( _max, sample );

View File

@ -641,7 +641,7 @@ TEST_CASE("/flow/flow/yieldedFuture/random")
ASSERT( numReady()==0 );
int expectYield = g_random->randomInt(0, 4);
int expectYield = deterministicRandom()->randomInt(0, 4);
int expectReady = expectYield;
yn->nextYield = 1 + expectYield;
@ -649,7 +649,7 @@ TEST_CASE("/flow/flow/yieldedFuture/random")
ASSERT( u.isReady() && i.isReady() && j.isReady() && numReady()==expectReady );
while (numReady() != v.size()) {
expectYield = g_random->randomInt(0, 4);
expectYield = deterministicRandom()->randomInt(0, 4);
yn->nextYield = 1 + expectYield;
expectReady += 1 + expectYield;
yn->tick();
@ -1032,11 +1032,11 @@ struct YAMRandom {
YAMRandom() : kmax(3) {}
void randomOp() {
if (g_random->random01() < 0.01)
if (deterministicRandom()->random01() < 0.01)
while (!check_yield());
int k = g_random->randomInt(0, kmax);
int op = g_random->randomInt(0, 7);
int k = deterministicRandom()->randomInt(0, kmax);
int op = deterministicRandom()->randomInt(0, 7);
//printf("%d",op);
if (op == 0) {
onchanges.push_back(yam.onChange(k));
@ -1044,20 +1044,20 @@ struct YAMRandom {
onchanges.push_back( trigger([this](){ this->randomOp(); }, yam.onChange(k)) );
} else if (op == 2) {
if (onchanges.size()) {
int i = g_random->randomInt(0, onchanges.size());
int i = deterministicRandom()->randomInt(0, onchanges.size());
onchanges[i] = onchanges.back();
onchanges.pop_back();
}
} else if (op == 3) {
onchanges.clear();
} else if (op == 4) {
int v = g_random->randomInt(0, 3);
int v = deterministicRandom()->randomInt(0, 3);
yam.set(k, v);
} else if (op == 5) {
yam.trigger(k);
} else if (op == 6) {
int a = g_random->randomInt(0, kmax);
int b = g_random->randomInt(0, kmax);
int a = deterministicRandom()->randomInt(0, kmax);
int b = deterministicRandom()->randomInt(0, kmax);
yam.triggerRange(std::min(a,b), std::max(a,b)+1);
}
}

View File

@ -599,16 +599,16 @@ static void scanPackets( TransportData* transport, uint8_t*& unprocessed_begin,
g_simulator.lastConnectionFailure = g_network->now();
isBuggifyEnabled = true;
TraceEvent(SevInfo, "BitsFlip");
int flipBits = 32 - (int) floor(log2(g_random->randomUInt32()));
int flipBits = 32 - (int) floor(log2(deterministicRandom()->randomUInt32()));
uint32_t firstFlipByteLocation = g_random->randomUInt32() % packetLen;
int firstFlipBitLocation = g_random->randomInt(0, 8);
uint32_t firstFlipByteLocation = deterministicRandom()->randomUInt32() % packetLen;
int firstFlipBitLocation = deterministicRandom()->randomInt(0, 8);
*(p + firstFlipByteLocation) ^= 1 << firstFlipBitLocation;
flipBits--;
for (int i = 0; i < flipBits; i++) {
uint32_t byteLocation = g_random->randomUInt32() % packetLen;
int bitLocation = g_random->randomInt(0, 8);
uint32_t byteLocation = deterministicRandom()->randomUInt32() % packetLen;
int bitLocation = deterministicRandom()->randomInt(0, 8);
if (byteLocation != firstFlipByteLocation || bitLocation != firstFlipBitLocation) {
*(p + byteLocation) ^= 1 << bitLocation;
}
@ -965,7 +965,7 @@ void FlowTransport::removePeerReference( const Endpoint& endpoint, NetworkMessag
}
void FlowTransport::addEndpoint( Endpoint& endpoint, NetworkMessageReceiver* receiver, uint32_t taskID ) {
endpoint.token = g_random->randomUniqueID();
endpoint.token = deterministicRandom()->randomUniqueID();
if (receiver->isStream()) {
endpoint.addresses = self->localAddresses;
endpoint.token = UID( endpoint.token.first() | TOKEN_STREAM_FLAG, endpoint.token.second() );

View File

@ -177,8 +177,8 @@ Future< REPLY_TYPE(Request) > loadBalance(
ASSERT( alternatives->size() );
state int bestAlt = g_random->randomInt(0, alternatives->countBest());
state int nextAlt = g_random->randomInt(0, std::max(alternatives->size() - 1,1));
state int bestAlt = deterministicRandom()->randomInt(0, alternatives->countBest());
state int nextAlt = deterministicRandom()->randomInt(0, std::max(alternatives->size() - 1,1));
if( nextAlt >= bestAlt )
nextAlt++;
@ -287,7 +287,7 @@ Future< REPLY_TYPE(Request) > loadBalance(
double delay = std::max(std::min((now()-g_network->networkMetrics.oldestAlternativesFailure)*FLOW_KNOBS->ALTERNATIVES_FAILURE_DELAY_RATIO, FLOW_KNOBS->ALTERNATIVES_FAILURE_MAX_DELAY), FLOW_KNOBS->ALTERNATIVES_FAILURE_MIN_DELAY);
// Making this SevWarn means a lot of clutter
if(now() - g_network->networkMetrics.newestAlternativesFailure > 1 || g_random->random01() < 0.01) {
if(now() - g_network->networkMetrics.newestAlternativesFailure > 1 || deterministicRandom()->random01() < 0.01) {
TraceEvent("AllAlternativesFailed")
.detail("Interval", FLOW_KNOBS->CACHE_REFRESH_INTERVAL_WHEN_ALL_ALTERNATIVES_FAILED)
.detail("Alternatives", alternatives->description())

View File

@ -63,7 +63,7 @@ public:
MultiInterface( const vector<T>& v, LocalityData const& locality = LocalityData() ) : bestCount(0) {
for(int i=0; i<v.size(); i++)
alternatives.push_back(KVPair<int,T>(LBDistance::DISTANT,v[i]));
g_random->randomShuffle(alternatives);
deterministicRandom()->randomShuffle(alternatives);
if ( LBLocalityData<T>::Present ) {
for(int a=0; a<alternatives.size(); a++)
alternatives[a].k = loadBalanceDistance( locality, LBLocalityData<T>::getLocality( alternatives[a].v ), LBLocalityData<T>::getAddress( alternatives[a].v ) );
@ -108,7 +108,7 @@ template <class T>
class MultiInterface<ReferencedInterface<T>> : public ReferenceCounted<MultiInterface<ReferencedInterface<T>>> {
public:
MultiInterface( const vector<Reference<ReferencedInterface<T>>>& v ) : alternatives(v), bestCount(0) {
g_random->randomShuffle(alternatives);
deterministicRandom()->randomShuffle(alternatives);
if ( LBLocalityData<T>::Present ) {
std::stable_sort( alternatives.begin(), alternatives.end(), ReferencedInterface<T>::sort_by_distance );
}

View File

@ -138,7 +138,7 @@ public:
}
int size() const { return map.size() - 1; } // We always have one range bounded by two entries
Iterator randomRange() {
return Iterator( map.index( g_random->randomInt(0, map.size()-1) ) );
return Iterator( map.index( deterministicRandom()->randomInt(0, map.size()-1) ) );
}
Iterator nthRange(int n) { return Iterator(map.index(n)); }

View File

@ -239,7 +239,7 @@ public:
// Return a random entry
LocalityEntry const& random() const
{ return _entryArray[g_random->randomInt(0, _entryArray.size())]; }
{ return _entryArray[deterministicRandom()->randomInt(0, _entryArray.size())]; }
// Return a given number of random entries that are not within the
// specified exception array
@ -259,7 +259,7 @@ public:
}
while (nItemsLeft > 0)
{
auto itemIndex = g_random->randomInt(0, nItemsLeft);
auto itemIndex = deterministicRandom()->randomInt(0, nItemsLeft);
auto item = _mutableEntryArray[itemIndex];
nItemsLeft --;
@ -299,7 +299,7 @@ public:
Optional<AttribValue> result;
while (checksLeft > 0) {
indexValue = g_random->randomInt(0, checksLeft);
indexValue = deterministicRandom()->randomInt(0, checksLeft);
valueValue = valueArray[indexValue];
validValue = true;
for (int usedLoop=0; usedLoop < usedValues; usedLoop ++) {

View File

@ -374,7 +374,7 @@ bool PolicyAcross::selectReplicas(
if (g_replicationdebug > 6) {
LocalitySet::staticDisplayEntries(fromServers, mutableArray, "mutable");
}
recordIndex = g_random->randomInt(0, checksLeft);
recordIndex = deterministicRandom()->randomInt(0, checksLeft);
auto& entry = mutableArray[recordIndex];
auto value = fromServers->getValueViaGroupKey(entry, groupIndexKey);
if (value.present()) {

View File

@ -629,12 +629,12 @@ Reference<IReplicationPolicy> const randomAcrossPolicy(LocalitySet const& server
Reference<IReplicationPolicy> policy(new PolicyOne());
// Determine the number of keys to used within the policy
usedKeyTotal = g_random->randomInt(1, keyArray.size()+1);
maxKeyTotal = g_random->randomInt(1, 4);
if ((usedKeyTotal > maxKeyTotal) && (g_random->random01() > .1)) {
usedKeyTotal = deterministicRandom()->randomInt(1, keyArray.size()+1);
maxKeyTotal = deterministicRandom()->randomInt(1, 4);
if ((usedKeyTotal > maxKeyTotal) && (deterministicRandom()->random01() > .1)) {
usedKeyTotal = maxKeyTotal;
}
maxValueTotal = g_random->randomInt(1, 10);
maxValueTotal = deterministicRandom()->randomInt(1, 10);
keysUsed = skips = 0;
if (g_replicationdebug > 6) {
@ -647,17 +647,17 @@ Reference<IReplicationPolicy> const randomAcrossPolicy(LocalitySet const& server
if (g_replicationdebug > 2) printf("Policy using%3d of%3lu keys Max values:%3d\n", usedKeyTotal, keyArray.size(), maxValueTotal);
while (keysUsed < usedKeyTotal) {
keyIndex = g_random->randomInt(0, keyArray.size()-keysUsed);
keyIndex = deterministicRandom()->randomInt(0, keyArray.size()-keysUsed);
keyText = keyArray[keyIndex];
lastKeyIndex = keyArray.size() - 1 - keysUsed;
// Do not allow az and sz within a policy, 90% of the time
if ((!keyText.compare("az")) && (g_random->random01() > .1) &&
if ((!keyText.compare("az")) && (deterministicRandom()->random01() > .1) &&
(std::find(keyArray.begin()+lastKeyIndex+1, keyArray.end(), "sz") != keyArray.end()))
{
skips ++;
}
else if ((!keyText.compare("sz")) && (g_random->random01() > .1) &&
else if ((!keyText.compare("sz")) && (deterministicRandom()->random01() > .1) &&
(std::find(keyArray.begin()+lastKeyIndex+1, keyArray.end(), "az") != keyArray.end()))
{
skips ++;
@ -677,8 +677,8 @@ Reference<IReplicationPolicy> const randomAcrossPolicy(LocalitySet const& server
valueSet.insert(serverSet.valueText(keyValue.get()));
}
}
valueTotal = g_random->randomInt(1, valueSet.size()+2);
if ((valueTotal > maxValueTotal) && (g_random->random01() > .25)) valueTotal = maxValueTotal;
valueTotal = deterministicRandom()->randomInt(1, valueSet.size()+2);
if ((valueTotal > maxValueTotal) && (deterministicRandom()->random01() > .25)) valueTotal = maxValueTotal;
policy = Reference<IReplicationPolicy>( new PolicyAcross(valueTotal, keyText, policy) );
if (g_replicationdebug > 1) {
printf(" item%3d: (%3d =>%3d) %-10s =>%4d\n", keysUsed+1, keyIndex, indexKey._id, keyText.c_str(), valueTotal);
@ -742,8 +742,8 @@ int testReplication()
if (debugLevelEnv) g_replicationdebug = atoi(debugLevelEnv);
debugBackup = g_replicationdebug;
testServers = createTestLocalityMap(serverIndexes, g_random->randomInt(1, 5), g_random->randomInt(1, 6), g_random->randomInt(1, 10), g_random->randomInt(1, 10), g_random->randomInt(0, 4), g_random->randomInt(1, 5));
maxAlsoSize = testServers->size() / g_random->randomInt(2, 20);
testServers = createTestLocalityMap(serverIndexes, deterministicRandom()->randomInt(1, 5), deterministicRandom()->randomInt(1, 6), deterministicRandom()->randomInt(1, 10), deterministicRandom()->randomInt(1, 10), deterministicRandom()->randomInt(0, 4), deterministicRandom()->randomInt(1, 5));
maxAlsoSize = testServers->size() / deterministicRandom()->randomInt(2, 20);
if (g_replicationdebug >= 0) printf("Running %d Replication test\n", totalTests);
@ -771,8 +771,8 @@ int testReplication()
g_replicationdebug = debugBackup;
skipTotal = 0;
}
alsoSize = g_random->randomInt(0, testServers->size()+1);
if ((alsoSize > maxAlsoSize) && (g_random->random01() > .2)) {
alsoSize = deterministicRandom()->randomInt(0, testServers->size()+1);
if ((alsoSize > maxAlsoSize) && (deterministicRandom()->random01() > .2)) {
alsoSize = maxAlsoSize;
}
@ -781,13 +781,13 @@ int testReplication()
}
else {
alsoServers = testServers->getEntries();
g_random->randomShuffle(alsoServers);
deterministicRandom()->randomShuffle(alsoServers);
if (alsoSize < testServers->size()) {
alsoServers.resize(alsoSize);
}
}
policyIndex = (policyIndexStatic>=0) ? policyIndexStatic : g_random->randomInt(0, policies.size());
policyIndex = (policyIndexStatic>=0) ? policyIndexStatic : deterministicRandom()->randomInt(0, policies.size());
if (g_replicationdebug > 0) printf(" #%7d: (%3d) ", testCounter, policyIndex);

View File

@ -42,8 +42,8 @@ bool testFuzzActor( Future<int>(*actor)(FutureStream<int> const&, PromiseStream<
for(int trial=0; trial<5; trial++) {
PromiseStream<int> in, out;
Promise<Void> err;
int before = g_random->randomInt(0, 4);
int errorBefore = before + g_random->randomInt(0, 4);
int before = deterministicRandom()->randomInt(0, 4);
int errorBefore = before + deterministicRandom()->randomInt(0, 4);
//printf("\t\tTrial #%d: %d, %d\n", trial, before, errorBefore);
if (errorBefore <= before) err.sendError( operation_failed() );
for(i=0; i<before; i++) {
@ -107,10 +107,10 @@ void memoryTest2() {
char** random = new char*[ Reads * MaxThreads ];
random[0] = block;
for(int i=1; i<Reads*MaxThreads; ) {
char *s = &block[ g_random->randomInt(0, Size) ];
char *s = &block[ deterministicRandom()->randomInt(0, Size) ];
random[i++] = s;
/*for(int j=0; j<10 && i<Reads*MaxThreads; j++,i++) {
random[i] = s + g_random->randomInt(0, 4096);
random[i] = s + deterministicRandom()->randomInt(0, 4096);
if (random[i] >= block+Size) random[i] -= Size;
}*/
}
@ -175,7 +175,7 @@ void memoryTest() {
x[i] = &x[i];
// Sattolo's algorithm
for(int n = N-1; n >= 1; n--) {
int k = g_random->randomInt(0, n); //random.IRandomX(0, n-1);
int k = deterministicRandom()->randomInt(0, n); //random.IRandomX(0, n-1);
std::swap( x[k], x[n] );
}
} else {
@ -201,7 +201,7 @@ void memoryTest() {
void **starts[MT*MaxTraversalsPerThread];
for(int t=0; t<PseudoThreads; t++) {
starts[t] = &x[ N/PseudoThreads * t ];
//starts[t] = &x[ g_random->randomInt(0,N) ];
//starts[t] = &x[ deterministicRandom()->randomInt(0,N) ];
}
for(int T=1; T<=MT; T+=T) {
double start = timer();
@ -315,7 +315,7 @@ void fastAllocTest() {
std::vector<void*> d;
for(int i=0; i<1000000;i++) {
d.push_back( FastAllocator<64>::allocate() );
int r = g_random->randomInt(0,1000000);
int r = deterministicRandom()->randomInt(0,1000000);
if (r < d.size()) {
FastAllocator<64>::release(d[r]);
d[r] = d.back();
@ -523,7 +523,7 @@ void threadSafetyTest2() {
vector<int> counts( streams.size() );
v.clear();
for(int k=0; k<V; k++) {
int i = g_random->randomInt(0, (int)streams.size());
int i = deterministicRandom()->randomInt(0, (int)streams.size());
counts[i]++;
v.push_back( streams[i] );
}
@ -1135,7 +1135,7 @@ extern void net2_test();
void dsltest() {
double startt, endt;
g_random = new DeterministicRandom(40);
setThreadLocalDeterministicRandomSeed(40);
asyncMapTest();
@ -1174,7 +1174,7 @@ void dsltest() {
printf("Actor fuzz tests: %d/%d passed\n", afResults.first, afResults.second);
startt = timer();
for(int i=0; i<1000000; i++)
g_random->random01();
deterministicRandom()->random01();
endt = timer();
printf("Random01: %0.2f M/sec\n", 1.0/(endt-startt));

View File

@ -25,7 +25,7 @@
#include "flow/actorcompiler.h"
ACTOR void simDeliverDuplicate( Standalone<StringRef> data, Endpoint destination ) {
wait( delay( g_random->random01() * FLOW_KNOBS->MAX_DELIVER_DUPLICATE_DELAY ) );
wait( delay( deterministicRandom()->random01() * FLOW_KNOBS->MAX_DELIVER_DUPLICATE_DELAY ) );
FlowTransport::transport().sendUnreliable( SerializeSourceRaw(data), destination );
}

View File

@ -41,7 +41,7 @@ bool simulator_should_inject_fault( const char* context, const char* file, int l
auto p = g_simulator.getCurrentProcess();
if (p->fault_injection_p2 && g_random->random01() < p->fault_injection_p2 && !g_simulator.speedUpSimulation) {
if (p->fault_injection_p2 && deterministicRandom()->random01() < p->fault_injection_p2 && !g_simulator.speedUpSimulation) {
uint32_t h1 = line + (p->fault_injection_r >> 32);
if (h1 < p->fault_injection_p1*std::numeric_limits<uint32_t>::max()) {
@ -158,7 +158,7 @@ private:
std::map<std::pair<IPAddress, IPAddress>, double> clogPairUntil;
std::map<std::pair<IPAddress, IPAddress>, double> clogPairLatency;
double halfLatency() {
double a = g_random->random01();
double a = deterministicRandom()->random01();
const double pFast = 0.999;
if (a <= pFast) {
a = a / pFast;
@ -174,7 +174,7 @@ SimClogging g_clogging;
struct Sim2Conn : IConnection, ReferenceCounted<Sim2Conn> {
Sim2Conn( ISimulator::ProcessInfo* process )
: process(process), dbgid( g_random->randomUniqueID() ), opened(false), closedByCaller(false)
: process(process), dbgid( deterministicRandom()->randomUniqueID() ), opened(false), closedByCaller(false)
{
pipes = sender(this) && receiver(this);
}
@ -187,8 +187,8 @@ struct Sim2Conn : IConnection, ReferenceCounted<Sim2Conn> {
this->peerEndpoint = peerEndpoint;
// Every one-way connection gets a random permanent latency and a random send buffer for the duration of the connection
auto latency = g_clogging.setPairLatencyIfNotSet( peerProcess->address.ip, process->address.ip, FLOW_KNOBS->MAX_CLOGGING_LATENCY*g_random->random01() );
sendBufSize = std::max<double>( g_random->randomInt(0, 5000000), 25e6 * (latency + .002) );
auto latency = g_clogging.setPairLatencyIfNotSet( peerProcess->address.ip, process->address.ip, FLOW_KNOBS->MAX_CLOGGING_LATENCY*deterministicRandom()->random01() );
sendBufSize = std::max<double>( deterministicRandom()->randomInt(0, 5000000), 25e6 * (latency + .002) );
TraceEvent("Sim2Connection").detail("SendBufSize", sendBufSize).detail("Latency", latency);
}
@ -246,7 +246,7 @@ struct Sim2Conn : IConnection, ReferenceCounted<Sim2Conn> {
}
}
ASSERT(toSend);
if (BUGGIFY) toSend = std::min(toSend, g_random->randomInt(0, 1000));
if (BUGGIFY) toSend = std::min(toSend, deterministicRandom()->randomInt(0, 1000));
if (!peer) return toSend;
toSend = std::min( toSend, peer->availableSendBufferForPeer() );
@ -300,7 +300,7 @@ private:
loop {
wait( self->writtenBytes.onChange() ); // takes place on peer!
ASSERT( g_simulator.getCurrentProcess() == self->peerProcess );
wait( delay( .002 * g_random->random01() ) );
wait( delay( .002 * deterministicRandom()->random01() ) );
self->sentBytes.set( self->writtenBytes.get() ); // or possibly just some sometimes...
}
}
@ -311,7 +311,7 @@ private:
while ( self->sentBytes.get() == self->receivedBytes.get() )
wait( self->sentBytes.onChange() );
ASSERT( g_simulator.getCurrentProcess() == self->peerProcess );
state int64_t pos = g_random->random01() < .5 ? self->sentBytes.get() : g_random->randomInt64( self->receivedBytes.get(), self->sentBytes.get()+1 );
state int64_t pos = deterministicRandom()->random01() < .5 ? self->sentBytes.get() : deterministicRandom()->randomInt64( self->receivedBytes.get(), self->sentBytes.get()+1 );
wait( delay( g_clogging.getSendDelay( self->process->address, self->peerProcess->address ) ) );
wait( g_simulator.onProcess( self->process ) );
ASSERT( g_simulator.getCurrentProcess() == self->process );
@ -360,9 +360,9 @@ private:
}
void rollRandomClose() {
if (now() - g_simulator.lastConnectionFailure > g_simulator.connectionFailuresDisableDuration && g_random->random01() < .00001) {
if (now() - g_simulator.lastConnectionFailure > g_simulator.connectionFailuresDisableDuration && deterministicRandom()->random01() < .00001) {
g_simulator.lastConnectionFailure = now();
double a = g_random->random01(), b = g_random->random01();
double a = deterministicRandom()->random01(), b = deterministicRandom()->random01();
TEST(true); // Simulated connection failure
TraceEvent("ConnectionFailure", dbgid).detail("MyAddr", process->address).detail("PeerAddr", peerProcess->address).detail("SendClosed", a > .33).detail("RecvClosed", a < .66).detail("Explicit", b < .3);
if (a < .66 && peer) peer->closeInternal();
@ -436,7 +436,7 @@ public:
wait( g_simulator.onMachine( currentProcess ) );
try {
wait( delay(FLOW_KNOBS->MIN_OPEN_TIME + g_random->random01() * (FLOW_KNOBS->MAX_OPEN_TIME - FLOW_KNOBS->MIN_OPEN_TIME) ) );
wait( delay(FLOW_KNOBS->MIN_OPEN_TIME + deterministicRandom()->random01() * (FLOW_KNOBS->MAX_OPEN_TIME - FLOW_KNOBS->MIN_OPEN_TIME) ) );
std::string open_filename = filename;
if (flags & OPEN_ATOMIC_WRITE_AND_CREATE) {
@ -512,7 +512,7 @@ private:
bool delayOnWrite;
SimpleFile(int h, Reference<DiskParameters> diskParameters, bool delayOnWrite, const std::string& filename, const std::string& actualFilename, int flags)
: h(h), diskParameters(diskParameters), delayOnWrite(delayOnWrite), filename(filename), actualFilename(actualFilename), dbgId(g_random->randomUniqueID()), flags(flags) {}
: h(h), diskParameters(diskParameters), delayOnWrite(delayOnWrite), filename(filename), actualFilename(actualFilename), dbgId(deterministicRandom()->randomUniqueID()), flags(flags) {}
static int flagConversion( int flags ) {
int outFlags = O_BINARY;
@ -528,7 +528,7 @@ private:
ACTOR static Future<int> read_impl( SimpleFile* self, void* data, int length, int64_t offset ) {
ASSERT( ( self->flags & IAsyncFile::OPEN_NO_AIO ) != 0 ||
( (uintptr_t)data % 4096 == 0 && length % 4096 == 0 && offset % 4096 == 0 ) ); // Required by KAIO.
state UID opId = g_random->randomUniqueID();
state UID opId = deterministicRandom()->randomUniqueID();
if (randLog)
fprintf( randLog, "SFR1 %s %s %s %d %" PRId64 "\n", self->dbgId.shortString().c_str(), self->filename.c_str(), opId.shortString().c_str(), length, offset );
@ -560,7 +560,7 @@ private:
}
ACTOR static Future<Void> write_impl( SimpleFile* self, StringRef data, int64_t offset ) {
state UID opId = g_random->randomUniqueID();
state UID opId = deterministicRandom()->randomUniqueID();
if (randLog) {
uint32_t a=0, b=0;
hashlittle2( data.begin(), data.size(), &a, &b );
@ -599,7 +599,7 @@ private:
}
ACTOR static Future<Void> truncate_impl( SimpleFile* self, int64_t size ) {
state UID opId = g_random->randomUniqueID();
state UID opId = deterministicRandom()->randomUniqueID();
if (randLog)
fprintf( randLog, "SFT1 %s %s %s %" PRId64 "\n", self->dbgId.shortString().c_str(), self->filename.c_str(), opId.shortString().c_str(), size );
@ -621,7 +621,7 @@ private:
}
ACTOR static Future<Void> sync_impl( SimpleFile* self ) {
state UID opId = g_random->randomUniqueID();
state UID opId = deterministicRandom()->randomUniqueID();
if (randLog)
fprintf( randLog, "SFC1 %s %s %s\n", self->dbgId.shortString().c_str(), self->filename.c_str(), opId.shortString().c_str());
@ -654,7 +654,7 @@ private:
}
ACTOR static Future<int64_t> size_impl( SimpleFile* self ) {
state UID opId = g_random->randomUniqueID();
state UID opId = deterministicRandom()->randomUniqueID();
if (randLog)
fprintf(randLog, "SFS1 %s %s %s\n", self->dbgId.shortString().c_str(), self->filename.c_str(), opId.shortString().c_str());
@ -707,7 +707,7 @@ private:
ACTOR static void incoming( Reference<Sim2Listener> self, double seconds, Reference<IConnection> conn ) {
wait( g_simulator.onProcess(self->process) );
wait( delay( seconds ) );
if (((Sim2Conn*)conn.getPtr())->isPeerGone() && g_random->random01()<0.5)
if (((Sim2Conn*)conn.getPtr())->isPeerGone() && deterministicRandom()->random01()<0.5)
return;
TraceEvent("Sim2IncomingConn", conn->getDebugID())
.detail("ListenAddress", self->getListenAddress())
@ -740,8 +740,8 @@ public:
seconds = std::max(0.0, seconds);
Future<Void> f;
if(!currentProcess->rebooting && machine == currentProcess && !currentProcess->shutdownSignal.isSet() && FLOW_KNOBS->MAX_BUGGIFIED_DELAY > 0 && g_random->random01() < 0.25) { //FIXME: why doesnt this work when we are changing machines?
seconds += FLOW_KNOBS->MAX_BUGGIFIED_DELAY*pow(g_random->random01(),1000.0);
if(!currentProcess->rebooting && machine == currentProcess && !currentProcess->shutdownSignal.isSet() && FLOW_KNOBS->MAX_BUGGIFIED_DELAY > 0 && deterministicRandom()->random01() < 0.25) { //FIXME: why doesnt this work when we are changing machines?
seconds += FLOW_KNOBS->MAX_BUGGIFIED_DELAY*pow(deterministicRandom()->random01(),1000.0);
}
mutex.enter();
@ -768,7 +768,7 @@ public:
virtual bool check_yield( int taskID ) {
if (yielded) return true;
if (--yield_limit <= 0) {
yield_limit = g_random->randomInt(1, 150); // If yield returns false *too* many times in a row, there could be a stack overflow, since we can't deterministically check stack size as the real network does
yield_limit = deterministicRandom()->randomInt(1, 150); // If yield returns false *too* many times in a row, there could be a stack overflow, since we can't deterministically check stack size as the real network does
return yielded = true;
}
return yielded = BUGGIFY_WITH_PROB(0.01);
@ -794,22 +794,22 @@ public:
if (getCurrentProcess()->address.ip.isV6()) {
IPAddress::IPAddressStore store = getCurrentProcess()->address.ip.toV6();
uint16_t* ipParts = (uint16_t*)store.data();
ipParts[7] += g_random->randomInt(0, 256);
ipParts[7] += deterministicRandom()->randomInt(0, 256);
localIp = IPAddress(store);
} else {
localIp = IPAddress(getCurrentProcess()->address.ip.toV4() + g_random->randomInt(0, 256));
localIp = IPAddress(getCurrentProcess()->address.ip.toV4() + deterministicRandom()->randomInt(0, 256));
}
peerc->connect(myc, NetworkAddress(localIp, g_random->randomInt(40000, 60000)));
peerc->connect(myc, NetworkAddress(localIp, deterministicRandom()->randomInt(40000, 60000)));
((Sim2Listener*)peerp->getListener(toAddr).getPtr())->incomingConnection( 0.5*g_random->random01(), Reference<IConnection>(peerc) );
return onConnect( ::delay(0.5*g_random->random01()), myc );
((Sim2Listener*)peerp->getListener(toAddr).getPtr())->incomingConnection( 0.5*deterministicRandom()->random01(), Reference<IConnection>(peerc) );
return onConnect( ::delay(0.5*deterministicRandom()->random01()), myc );
}
virtual Future<std::vector<NetworkAddress>> resolveTCPEndpoint( std::string host, std::string service) {
throw lookup_failed();
}
ACTOR static Future<Reference<IConnection>> onConnect( Future<Void> ready, Reference<Sim2Conn> conn ) {
wait(ready);
if (conn->isPeerGone() && g_random->random01()<0.5) {
if (conn->isPeerGone() && deterministicRandom()->random01()<0.5) {
conn.clear();
wait(Never());
}
@ -826,7 +826,7 @@ public:
NetworkAddress toAddr, INetworkConnections *self ) {
// We have to be able to connect to processes that don't yet exist, so we do some silly polling
loop {
wait( ::delay( 0.1 * g_random->random01() ) );
wait( ::delay( 0.1 * deterministicRandom()->random01() ) );
if (g_sim2.addressMap.count(toAddr)) {
Reference<IConnection> c = wait( self->connect( toAddr ) );
return c;
@ -880,14 +880,14 @@ public:
}
if(diskSpace.totalSpace == 0) {
diskSpace.totalSpace = 5e9 + g_random->random01() * 100e9; //Total space between 5GB and 105GB
diskSpace.baseFreeSpace = std::min<int64_t>(diskSpace.totalSpace, std::max(5e9, (g_random->random01() * (1 - .075) + .075) * diskSpace.totalSpace) + totalFileSize); //Minimum 5GB or 7.5% total disk space, whichever is higher
diskSpace.totalSpace = 5e9 + deterministicRandom()->random01() * 100e9; //Total space between 5GB and 105GB
diskSpace.baseFreeSpace = std::min<int64_t>(diskSpace.totalSpace, std::max(5e9, (deterministicRandom()->random01() * (1 - .075) + .075) * diskSpace.totalSpace) + totalFileSize); //Minimum 5GB or 7.5% total disk space, whichever is higher
TraceEvent("Sim2DiskSpaceInitialization").detail("TotalSpace", diskSpace.totalSpace).detail("BaseFreeSpace", diskSpace.baseFreeSpace).detail("TotalFileSize", totalFileSize).detail("NumFiles", numFiles);
}
else {
int64_t maxDelta = std::min(5.0, (now() - diskSpace.lastUpdate)) * (BUGGIFY ? 10e6 : 1e6); //External processes modifying the disk
int64_t delta = -maxDelta + g_random->random01() * maxDelta * 2;
int64_t delta = -maxDelta + deterministicRandom()->random01() * maxDelta * 2;
diskSpace.baseFreeSpace = std::min<int64_t>(diskSpace.totalSpace, std::max<int64_t>(diskSpace.baseFreeSpace + delta, totalFileSize));
}
@ -911,16 +911,16 @@ public:
g_simulator.getCurrentProcess()->machine->openFiles.erase(filename);
g_simulator.getCurrentProcess()->machine->deletingFiles.insert(filename);
}
if ( mustBeDurable || g_random->random01() < 0.5 ) {
if ( mustBeDurable || deterministicRandom()->random01() < 0.5 ) {
state ISimulator::ProcessInfo* currentProcess = g_simulator.getCurrentProcess();
state int currentTaskID = g_network->getCurrentTask();
wait( g_simulator.onMachine( currentProcess ) );
try {
wait( ::delay(0.05 * g_random->random01()) );
wait( ::delay(0.05 * deterministicRandom()->random01()) );
if (!currentProcess->rebooting) {
auto f = IAsyncFileSystem::filesystem(self->net2)->deleteFile(filename, false);
ASSERT( f.isReady() );
wait( ::delay(0.05 * g_random->random01()) );
wait( ::delay(0.05 * deterministicRandom()->random01()) );
TEST( true ); // Simulated durable delete
}
wait( g_simulator.onProcess( currentProcess, currentTaskID ) );
@ -1230,9 +1230,9 @@ public:
} else if (kt == InjectFaults) {
TraceEvent(SevWarn, "FaultMachine").detail("Name", machine->name).detail("Address", machine->address).detail("ZoneId", machine->locality.zoneId()).detail("Process", machine->toString()).detail("Rebooting", machine->rebooting).detail("Protected", protectedAddresses.count(machine->address)).backtrace();
should_inject_fault = simulator_should_inject_fault;
machine->fault_injection_r = g_random->randomUniqueID().first();
machine->fault_injection_r = deterministicRandom()->randomUniqueID().first();
machine->fault_injection_p1 = 0.1;
machine->fault_injection_p2 = g_random->random01();
machine->fault_injection_p2 = deterministicRandom()->random01();
} else {
ASSERT( false );
}
@ -1259,7 +1259,7 @@ public:
}
}
if( processes.size() )
doReboot( g_random->randomChoice( processes ), RebootProcess );
doReboot( deterministicRandom()->randomChoice( processes ), RebootProcess );
}
}
virtual void killProcess( ProcessInfo* machine, KillType kt ) {
@ -1475,7 +1475,7 @@ public:
KillType ktResult, ktMin = kt;
for (auto& datacenterMachine : datacenterMachines) {
if(g_random->random01() < 0.99) {
if(deterministicRandom()->random01() < 0.99) {
killMachine(datacenterMachine.first, kt, true, &ktResult);
if (ktResult != kt) {
TraceEvent(SevWarn, "KillDCFail")
@ -1512,7 +1512,7 @@ public:
}
virtual void clogInterface(const IPAddress& ip, double seconds, ClogMode mode = ClogDefault) {
if (mode == ClogDefault) {
double a = g_random->random01();
double a = deterministicRandom()->random01();
if ( a < 0.3 ) mode = ClogSend;
else if (a < 0.6 ) mode = ClogReceive;
else mode = ClogAll;
@ -1610,7 +1610,7 @@ public:
ASSERT( this->currentProcess == t.machine );
/*auto elapsed = getCPUTicks() - before;
currentProcess->cpuTicks += elapsed;
if (g_random->random01() < 0.01){
if (deterministicRandom()->random01() < 0.01){
TraceEvent("TaskDuration").detail("CpuTicks", currentProcess->cpuTicks);
currentProcess->cpuTicks = 0;
}*/
@ -1624,7 +1624,7 @@ public:
//}
if (randLog)
fprintf( randLog, "T %f %d %s %" PRId64 "\n", this->time, int(g_random->peek() % 10000), t.machine ? t.machine->name : "none", t.stable);
fprintf( randLog, "T %f %d %s %" PRId64 "\n", this->time, int(deterministicRandom()->peek() % 10000), t.machine ? t.machine->name : "none", t.stable);
}
}
@ -1676,7 +1676,7 @@ public:
void startNewSimulator() {
ASSERT( !g_network );
g_network = g_pSimulator = new Sim2();
g_simulator.connectionFailuresDisableDuration = g_random->random01() < 0.5 ? 0 : 1e6;
g_simulator.connectionFailuresDisableDuration = deterministicRandom()->random01() < 0.5 ? 0 : 1e6;
}
ACTOR void doReboot( ISimulator::ProcessInfo *p, ISimulator::KillType kt ) {
@ -1718,9 +1718,9 @@ Future<Void> waitUntilDiskReady( Reference<DiskParameters> diskParameters, int64
double randomLatency;
if(sync) {
randomLatency = .005 + g_random->random01() * (BUGGIFY ? 1.0 : .010);
randomLatency = .005 + deterministicRandom()->random01() * (BUGGIFY ? 1.0 : .010);
} else
randomLatency = 10 * g_random->random01() / diskParameters->iops;
randomLatency = 10 * deterministicRandom()->random01() / diskParameters->iops;
return delayUntil( diskParameters->nextOperation + randomLatency );
}
@ -1793,7 +1793,7 @@ Future< Void > Sim2FileSystem::deleteFile( std::string filename, bool mustBeDura
Future< std::time_t > Sim2FileSystem::lastWriteTime( std::string filename ) {
// TODO: update this map upon file writes.
static std::map<std::string, double> fileWrites;
if (BUGGIFY && g_random->random01() < 0.01) {
if (BUGGIFY && deterministicRandom()->random01() < 0.01) {
fileWrites[filename] = now();
}
return fileWrites[filename];

View File

@ -113,21 +113,21 @@ public:
void setDistributor(const DataDistributorInterface& interf) {
ServerDBInfo newInfo = serverInfo->get();
newInfo.id = g_random->randomUniqueID();
newInfo.id = deterministicRandom()->randomUniqueID();
newInfo.distributor = interf;
serverInfo->set( newInfo );
}
void setRatekeeper(const RatekeeperInterface& interf) {
ServerDBInfo newInfo = serverInfo->get();
newInfo.id = g_random->randomUniqueID();
newInfo.id = deterministicRandom()->randomUniqueID();
newInfo.ratekeeper = interf;
serverInfo->set( newInfo );
}
void clearInterf(ProcessClass::ClassType t) {
ServerDBInfo newInfo = serverInfo->get();
newInfo.id = g_random->randomUniqueID();
newInfo.id = deterministicRandom()->randomUniqueID();
if (t == ProcessClass::DataDistributorClass) {
newInfo.distributor = Optional<DataDistributorInterface>();
} else if (t == ProcessClass::RatekeeperClass) {
@ -457,7 +457,7 @@ public:
for( auto& it : fitness_workers ) {
for( int j=0; j < 2; j++ ) {
auto& w = j==0 ? it.second.first : it.second.second;
g_random->randomShuffle(w);
deterministicRandom()->randomShuffle(w);
for( int i=0; i < w.size(); i++ ) {
id_used[w[i].interf.locality.processId()]++;
return WorkerFitnessInfo(w[i], it.first.first, it.first.second);
@ -490,7 +490,7 @@ public:
for( auto& it : fitness_workers ) {
for( int j=0; j < 2; j++ ) {
auto& w = j==0 ? it.second.first : it.second.second;
g_random->randomShuffle(w);
deterministicRandom()->randomShuffle(w);
for( int i=0; i < w.size(); i++ ) {
results.push_back(w[i]);
id_used[w[i].interf.locality.processId()]++;
@ -811,7 +811,7 @@ public:
bestFitness = fitness;
numEquivalent = 1;
bestDC = dcId;
} else if( fitness == bestFitness && g_random->random01() < 1.0/++numEquivalent ) {
} else if( fitness == bestFitness && deterministicRandom()->random01() < 1.0/++numEquivalent ) {
bestDC = dcId;
}
}
@ -1166,7 +1166,7 @@ public:
versionDifferenceUpdated(false), recruitingDistributor(false), recruitRatekeeper(false)
{
auto serverInfo = db.serverInfo->get();
serverInfo.id = g_random->randomUniqueID();
serverInfo.id = deterministicRandom()->randomUniqueID();
serverInfo.masterLifetime.ccID = id;
serverInfo.clusterInterface = ccInterface;
serverInfo.myLocality = locality;
@ -1227,7 +1227,7 @@ ACTOR Future<Void> clusterWatchDatabase( ClusterControllerData* cluster, Cluster
auto dbInfo = ServerDBInfo();
dbInfo.master = iMaster;
dbInfo.id = g_random->randomUniqueID();
dbInfo.id = deterministicRandom()->randomUniqueID();
dbInfo.masterLifetime = db->serverInfo->get().masterLifetime;
++dbInfo.masterLifetime;
dbInfo.clusterInterface = db->serverInfo->get().clusterInterface;
@ -1274,7 +1274,7 @@ void setIssues(ProcessIssuesMap& issueMap, NetworkAddress const& addr, VectorRef
if (issues.size()) {
auto& e = issueMap[addr];
e.first = issues;
e.second = g_random->randomUniqueID();
e.second = deterministicRandom()->randomUniqueID();
issueID = e.second;
} else {
issueMap.erase(addr);
@ -1803,7 +1803,7 @@ void clusterRegisterMaster( ClusterControllerData* self, RegisterMasterRequest c
if (db->clientInfo->get().proxies != req.proxies) {
isChanged = true;
ClientDBInfo clientInfo;
clientInfo.id = g_random->randomUniqueID();
clientInfo.id = deterministicRandom()->randomUniqueID();
clientInfo.proxies = req.proxies;
clientInfo.clientTxnInfoSampleRate = db->clientInfo->get().clientTxnInfoSampleRate;
clientInfo.clientTxnInfoSizeLimit = db->clientInfo->get().clientTxnInfoSizeLimit;
@ -1827,7 +1827,7 @@ void clusterRegisterMaster( ClusterControllerData* self, RegisterMasterRequest c
}
if( isChanged ) {
dbInfo.id = g_random->randomUniqueID();
dbInfo.id = deterministicRandom()->randomUniqueID();
self->db.serverInfo->set( dbInfo );
}
@ -1985,7 +1985,7 @@ ACTOR Future<Void> timeKeeper(ClusterControllerData *self) {
// FIXME: replace or augment this with logging on the proxy which tracks
// how long it is taking to hear responses from each other component.
UID debugID = g_random->randomUniqueID();
UID debugID = deterministicRandom()->randomUniqueID();
TraceEvent("TimeKeeperCommit", debugID);
tr->debugTransaction(debugID);
}
@ -2202,7 +2202,7 @@ ACTOR Future<Void> monitorServerInfoConfig(ClusterControllerData::DBInfo* db) {
ServerDBInfo serverInfo = db->serverInfo->get();
if(config != serverInfo.latencyBandConfig) {
TraceEvent("LatencyBandConfigChanged").detail("Present", config.present());
serverInfo.id = g_random->randomUniqueID();
serverInfo.id = deterministicRandom()->randomUniqueID();
serverInfo.latencyBandConfig = config;
db->serverInfo->set(serverInfo);
}
@ -2234,7 +2234,7 @@ ACTOR Future<Void> monitorClientTxnInfoConfigs(ClusterControllerData::DBInfo* db
double sampleRate = rateVal.present() ? BinaryReader::fromStringRef<double>(rateVal.get(), Unversioned()) : std::numeric_limits<double>::infinity();
int64_t sizeLimit = limitVal.present() ? BinaryReader::fromStringRef<int64_t>(limitVal.get(), Unversioned()) : -1;
if (sampleRate != clientInfo.clientTxnInfoSampleRate || sizeLimit != clientInfo.clientTxnInfoSampleRate) {
clientInfo.id = g_random->randomUniqueID();
clientInfo.id = deterministicRandom()->randomUniqueID();
clientInfo.clientTxnInfoSampleRate = sampleRate;
clientInfo.clientTxnInfoSizeLimit = sizeLimit;
db->clientInfo->set(clientInfo);
@ -2490,7 +2490,7 @@ ACTOR Future<DataDistributorInterface> startDataDistributor( ClusterControllerDa
worker = self->id_worker[self->masterProcessId.get()].details;
}
InitializeDataDistributorRequest req(g_random->randomUniqueID());
InitializeDataDistributorRequest req(deterministicRandom()->randomUniqueID());
TraceEvent("CC_DataDistributorRecruit", self->id).detail("Addr", worker.interf.address());
ErrorOr<DataDistributorInterface> distributor = wait( worker.interf.dataDistributor.getReplyUnlessFailedFor(req, SERVER_KNOBS->WAIT_FOR_DISTRIBUTOR_JOIN_DELAY, 0) );
@ -2546,7 +2546,7 @@ ACTOR Future<Void> startRatekeeper(ClusterControllerData *self) {
std::map<Optional<Standalone<StringRef>>, int> id_used = self->getUsedIds();
WorkerFitnessInfo rkWorker = self->getWorkerForRoleInDatacenter(self->clusterControllerDcId, ProcessClass::Ratekeeper, ProcessClass::NeverAssign, self->db.config, id_used);
InitializeRatekeeperRequest req(g_random->randomUniqueID());
InitializeRatekeeperRequest req(deterministicRandom()->randomUniqueID());
state WorkerDetails worker = rkWorker.worker;
if (self->onMasterIsBetter(worker, ProcessClass::Ratekeeper)) {
worker = self->id_worker[self->masterProcessId.get()].details;

View File

@ -524,20 +524,20 @@ void compactMapTests(std::vector<std::string> testData, std::vector<std::string>
copies.push_back((CompactPreOrderTree*) new uint8_t[compactTreeBytes]);
memcpy(copies.back(), t, compactTreeBytes);
}
g_random->randomShuffle(copies);
deterministicRandom()->randomShuffle(copies);
std::vector<PrefixTree*> prefixTreeCopies;
for (int i = 0; i < 2*sampleQueries.size(); i++) {
prefixTreeCopies.push_back((PrefixTree *) new uint8_t[prefixTreeBytes]);
memcpy(prefixTreeCopies.back(), pt, prefixTreeBytes);
}
g_random->randomShuffle(prefixTreeCopies);
deterministicRandom()->randomShuffle(prefixTreeCopies);
std::vector<std::vector<std::string>> array_copies;
for (int i = 0; i < sampleQueries.size(); i++) {
array_copies.push_back(testData);
}
g_random->randomShuffle(array_copies);
deterministicRandom()->randomShuffle(array_copies);
printf("shuffled\n");
@ -608,7 +608,7 @@ std::vector<std::string> sampleDocuments(int N) {
std::string oa = "\x01""amount\x00\x00";
std::string dbl = "\x02\x00\x00\x00\x00\x00\x00\x00\x00\x00";
for (int i = 0; i < N; i++) {
std::string id = BinaryWriter::toValue(g_random->randomUniqueID(), Unversioned()).substr(12).toString();
std::string id = BinaryWriter::toValue(deterministicRandom()->randomUniqueID(), Unversioned()).substr(12).toString();
testData.push_back(p + id + n);
testData.push_back(p + id + a);
for (int j = 0; j < 5; j++) {
@ -696,10 +696,10 @@ void ingestBenchmark() {
std::set<StringRef> testmap;
for(int i = 0; i < 1000000; ++i) {
keys_generated.push_back(StringRef(arena, format("........%02X......%02X.....%02X........%02X",
g_random->randomInt(0, 100),
g_random->randomInt(0, 100),
g_random->randomInt(0, 100),
g_random->randomInt(0, 100)
deterministicRandom()->randomInt(0, 100),
deterministicRandom()->randomInt(0, 100),
deterministicRandom()->randomInt(0, 100),
deterministicRandom()->randomInt(0, 100)
)));
}
@ -726,7 +726,7 @@ void ingestBenchmark() {
pages.resize(pageCount);
for(auto &key : keys_generated) {
int p = g_random->randomInt(0, pageCount);
int p = deterministicRandom()->randomInt(0, pageCount);
Page *&pPage = pages[p];
if(pPage == nullptr)
pPage = new Page();
@ -783,7 +783,7 @@ int main() {
printf("Left pointer is %s\n", CompactPreOrderTree::Node::ENABLE_LEFT_PTR ? "ON" : "OFF");
printf("Fancy build is %s\n", CompactPreOrderTree::ENABLE_FANCY_BUILD ? "ON" : "OFF");
g_random = new DeterministicRandom(1);
setThreadLocalDeterministicRandomSeed(1);
//ingestBenchmark();
@ -804,27 +804,27 @@ int main() {
printf("\n16 byte hexadecimal random keys\n");
std::vector<std::string> testData;
for (int i = 0; i < 200; i++) {
testData.push_back(g_random->randomUniqueID().shortString());
testData.push_back(deterministicRandom()->randomUniqueID().shortString());
}
std::vector<std::string> sampleQueries;
for (int i = 0; i < 10000; i++) {
sampleQueries.push_back(g_random->randomUniqueID().shortString().substr(0, g_random->randomInt(0, 16)));
sampleQueries.push_back(deterministicRandom()->randomUniqueID().shortString().substr(0, deterministicRandom()->randomInt(0, 16)));
}
compactMapTests(testData, sampleQueries);
printf("\nRaw index keys\n");
testData.clear(); sampleQueries.clear();
for (int i = 0; i < 100; i++) {
testData.push_back(format("%d Main Street #%d, New York NY 12345, United States of America|", 1234 * (i / 100), (i/10) % 10 + 1000) + g_random->randomUniqueID().shortString());
testData.push_back(format("%d Main Street #%d, New York NY 12345, United States of America|", 1234 * (i / 100), (i/10) % 10 + 1000) + deterministicRandom()->randomUniqueID().shortString());
}
for (int i = 0; i < 10000; i++)
sampleQueries.push_back(format("%d Main Street", g_random->randomInt(1000, 10000)));
sampleQueries.push_back(format("%d Main Street", deterministicRandom()->randomInt(1000, 10000)));
compactMapTests(testData, sampleQueries, "graph_addresses.dot");
printf("\nb+tree separators for index keys\n");
testData.clear();
for (int i = 0; i < 100000; i++) {
testData.push_back(format("%d Main Street #%d, New York NY 12345, United States of America|", 12 * (i / 100), (i/10) % 10 + 1000) + g_random->randomUniqueID().shortString());
testData.push_back(format("%d Main Street #%d, New York NY 12345, United States of America|", 12 * (i / 100), (i/10) % 10 + 1000) + deterministicRandom()->randomUniqueID().shortString());
}
testData = sampleBPlusTreeSeparators(testData, 0);
compactMapTests(testData, sampleQueries);
@ -834,7 +834,7 @@ int main() {
sampleQueries.clear();
std::string p = "pre";
for (int i = 0; i < 10000; i++)
sampleQueries.push_back(p + BinaryWriter::toValue(g_random->randomUniqueID(), Unversioned()).substr(12).toString());
sampleQueries.push_back(p + BinaryWriter::toValue(deterministicRandom()->randomUniqueID(), Unversioned()).substr(12).toString());
compactMapTests(testData, sampleQueries);
printf("\nb+tree split keys for documents\n");

View File

@ -27,19 +27,19 @@
ACTOR Future<GenerationRegReadReply> waitAndSendRead( RequestStream<GenerationRegReadRequest> to, GenerationRegReadRequest req ) {
if( SERVER_KNOBS->BUGGIFY_ALL_COORDINATION || BUGGIFY )
wait( delay( SERVER_KNOBS->BUGGIFIED_EVENTUAL_CONSISTENCY*g_random->random01() ) );
wait( delay( SERVER_KNOBS->BUGGIFIED_EVENTUAL_CONSISTENCY*deterministicRandom()->random01() ) );
state GenerationRegReadReply reply = wait( retryBrokenPromise( to, req ) );
if( SERVER_KNOBS->BUGGIFY_ALL_COORDINATION || BUGGIFY )
wait( delay( SERVER_KNOBS->BUGGIFIED_EVENTUAL_CONSISTENCY*g_random->random01() ) );
wait( delay( SERVER_KNOBS->BUGGIFIED_EVENTUAL_CONSISTENCY*deterministicRandom()->random01() ) );
return reply;
}
ACTOR Future<UniqueGeneration> waitAndSendWrite(RequestStream<GenerationRegWriteRequest> to, GenerationRegWriteRequest req) {
if( SERVER_KNOBS->BUGGIFY_ALL_COORDINATION || BUGGIFY )
wait( delay( SERVER_KNOBS->BUGGIFIED_EVENTUAL_CONSISTENCY*g_random->random01() ) );
wait( delay( SERVER_KNOBS->BUGGIFIED_EVENTUAL_CONSISTENCY*deterministicRandom()->random01() ) );
state UniqueGeneration reply = wait( retryBrokenPromise( to, req ) );
if( SERVER_KNOBS->BUGGIFY_ALL_COORDINATION || BUGGIFY )
wait( delay( SERVER_KNOBS->BUGGIFIED_EVENTUAL_CONSISTENCY*g_random->random01() ) );
wait( delay( SERVER_KNOBS->BUGGIFIED_EVENTUAL_CONSISTENCY*deterministicRandom()->random01() ) );
return reply;
}
@ -82,7 +82,7 @@ struct CoordinatedStateImpl {
self->stage = 1;
GenerationRegReadReply rep = wait( self->replicatedRead( self, GenerationRegReadRequest( self->coordinators.clusterKey, UniqueGeneration() ) ) );
self->conflictGen = std::max( self->conflictGen, std::max(rep.gen.generation, rep.rgen.generation) ) + 1;
self->gen = UniqueGeneration( self->conflictGen, g_random->randomUniqueID() );
self->gen = UniqueGeneration( self->conflictGen, deterministicRandom()->randomUniqueID() );
}
{

View File

@ -164,11 +164,11 @@ ACTOR Future<Void> localGenerationReg( GenerationRegInterface interf, OnDemandSt
TEST_CASE("/fdbserver/Coordination/localGenerationReg/simple") {
state GenerationRegInterface reg;
state OnDemandStore store("simfdb/unittests/", //< FIXME
g_random->randomUniqueID());
deterministicRandom()->randomUniqueID());
state Future<Void> actor = localGenerationReg(reg, &store);
state Key the_key(g_random->randomAlphaNumeric( g_random->randomInt(0, 10)));
state Key the_key(deterministicRandom()->randomAlphaNumeric( deterministicRandom()->randomInt(0, 10)));
state UniqueGeneration firstGen(0, g_random->randomUniqueID());
state UniqueGeneration firstGen(0, deterministicRandom()->randomUniqueID());
{
GenerationRegReadReply r = wait(reg.read.getReply(GenerationRegReadRequest(the_key, firstGen)));
@ -450,7 +450,7 @@ ACTOR Future<Void> leaderServer(LeaderElectionRegInterface interf, OnDemandStore
}
ACTOR Future<Void> coordinationServer(std::string dataFolder) {
state UID myID = g_random->randomUniqueID();
state UID myID = deterministicRandom()->randomUniqueID();
state LeaderElectionRegInterface myLeaderInterface( g_network );
state GenerationRegInterface myInterface( g_network );
state OnDemandStore store( dataFolder, myID );

View File

@ -244,7 +244,7 @@ public:
int64_t inFlightBytes = includeInFlight ? getDataInFlightToTeam() / servers.size() : 0;
double freeSpaceMultiplier = SERVER_KNOBS->FREE_SPACE_RATIO_CUTOFF / ( std::max( std::min( SERVER_KNOBS->FREE_SPACE_RATIO_CUTOFF, minFreeSpaceRatio ), 0.000001 ) );
if(freeSpaceMultiplier > 1 && g_random->random01() < 0.001)
if(freeSpaceMultiplier > 1 && deterministicRandom()->random01() < 0.001)
TraceEvent(SevWarn, "DiskNearCapacity").detail("FreeSpaceRatio", minFreeSpaceRatio);
return (physicalBytes + (inflightPenalty*inFlightBytes)) * freeSpaceMultiplier;
@ -784,7 +784,7 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
if( !req.wantsTrueBest ) {
while( similarTeams.size() && randomTeams.size() < SERVER_KNOBS->BEST_TEAM_OPTION_COUNT ) {
int randomTeam = g_random->randomInt( 0, similarTeams.size() );
int randomTeam = deterministicRandom()->randomInt( 0, similarTeams.size() );
randomTeams.push_back( std::make_pair( SOME_SHARED, similarTeams[randomTeam] ) );
swapAndPop( &similarTeams, randomTeam );
}
@ -806,7 +806,7 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
else {
int nTries = 0;
while( randomTeams.size() < SERVER_KNOBS->BEST_TEAM_OPTION_COUNT && nTries < SERVER_KNOBS->BEST_TEAM_MAX_TEAM_TRIES ) {
Reference<IDataDistributionTeam> dest = g_random->randomChoice(self->teams);
Reference<IDataDistributionTeam> dest = deterministicRandom()->randomChoice(self->teams);
bool ok = dest->isHealthy() && (!req.preferLowerUtilization || dest->hasHealthyFreeSpace());
for(int i=0; ok && i<randomTeams.size(); i++)
@ -1359,7 +1359,7 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
// We will use forcedAttribute to call existing function to form a team
if (leastUsedMachines.size()) {
// Randomly choose 1 least used machine
Reference<TCMachineInfo> tcMachineInfo = g_random->randomChoice(leastUsedMachines);
Reference<TCMachineInfo> tcMachineInfo = deterministicRandom()->randomChoice(leastUsedMachines);
ASSERT(!tcMachineInfo->serversOnMachine.empty());
LocalityEntry process = tcMachineInfo->localityEntry;
forcedAttributes.push_back(process);
@ -1508,7 +1508,7 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
}
}
return g_random->randomChoice(leastUsedServers);
return deterministicRandom()->randomChoice(leastUsedServers);
}
// Randomly choose one machine team that has chosenServer and has the correct size
@ -1522,7 +1522,7 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
}
}
if (!healthyMachineTeamsForChosenServer.empty()) {
return g_random->randomChoice(healthyMachineTeamsForChosenServer);
return deterministicRandom()->randomChoice(healthyMachineTeamsForChosenServer);
}
}
@ -1713,7 +1713,7 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
healthyProcesses.push_back(it);
}
}
serverID = g_random->randomChoice(healthyProcesses)->id;
serverID = deterministicRandom()->randomChoice(healthyProcesses)->id;
}
serverTeam.push_back(serverID);
}
@ -2553,7 +2553,7 @@ ACTOR Future<Void> teamTracker(DDTeamCollection* self, Reference<TCTeamInfo> tea
rs.priority = maxPriority;
self->output.send(rs);
if(g_random->random01() < 0.01) {
if(deterministicRandom()->random01() < 0.01) {
TraceEvent("SendRelocateToDDQx100", self->distributorId)
.detail("Team", team->getDesc())
.detail("KeyBegin", rs.keys.begin)
@ -2754,7 +2754,7 @@ ACTOR Future<Void> serverMetricsPolling( TCServerInfo *server) {
state double lastUpdate = now();
loop {
wait( updateServerMetrics( server ) );
wait( delayUntil( lastUpdate + SERVER_KNOBS->STORAGE_METRICS_POLLING_DELAY + SERVER_KNOBS->STORAGE_METRICS_RANDOM_DELAY * g_random->random01(), TaskDataDistributionLaunch ) );
wait( delayUntil( lastUpdate + SERVER_KNOBS->STORAGE_METRICS_POLLING_DELAY + SERVER_KNOBS->STORAGE_METRICS_RANDOM_DELAY * deterministicRandom()->random01(), TaskDataDistributionLaunch ) );
lastUpdate = now();
}
}
@ -3141,11 +3141,11 @@ ACTOR Future<Void> initializeStorage( DDTeamCollection* self, RecruitStorageRepl
// SOMEDAY: Cluster controller waits for availability, retry quickly if a server's Locality changes
self->recruitingStream.set(self->recruitingStream.get()+1);
state UID interfaceId = g_random->randomUniqueID();
state UID interfaceId = deterministicRandom()->randomUniqueID();
InitializeStorageRequest isr;
isr.storeType = self->configuration.storageServerStoreType;
isr.seedTag = invalidTag;
isr.reqId = g_random->randomUniqueID();
isr.reqId = deterministicRandom()->randomUniqueID();
isr.interfaceId = interfaceId;
TraceEvent("DDRecruiting").detail("State", "Sending request to worker").detail("WorkerID", candidateWorker.worker.id())

View File

@ -46,7 +46,7 @@ struct RelocateData {
TraceInterval interval;
RelocateData() : startTime(-1), priority(-1), workFactor(0), wantsNewServers(false), interval("QueuedRelocation") {}
RelocateData( RelocateShard const& rs ) : keys(rs.keys), priority(rs.priority), startTime(now()), randomId(g_random->randomUniqueID()), workFactor(0),
RelocateData( RelocateShard const& rs ) : keys(rs.keys), priority(rs.priority), startTime(now()), randomId(deterministicRandom()->randomUniqueID()), workFactor(0),
wantsNewServers(
rs.priority == PRIORITY_REBALANCE_SHARD ||
rs.priority == PRIORITY_REBALANCE_OVERUTILIZED_TEAM ||
@ -828,7 +828,7 @@ struct DDQueueData {
//logRelocation( rd, "LaunchedRelocation" );
}
if( now() - startTime > .001 && g_random->random01()<0.001 )
if( now() - startTime > .001 && deterministicRandom()->random01()<0.001 )
TraceEvent(SevWarnAlways, "LaunchingQueueSlowx1000").detail("Elapsed", now() - startTime );
/*if( startedHere > 0 ) {
@ -950,7 +950,7 @@ ACTOR Future<Void> dataDistributionRelocator( DDQueueData *self, RelocateData rd
// We randomly choose a server in bestTeams[i] as the shard's destination and
// move the shard to the randomly chosen server (in the remote DC), which will later
// propogate its data to the servers in the same team. This saves data movement bandwidth across DC
int idx = g_random->randomInt(0, serverIds.size());
int idx = deterministicRandom()->randomInt(0, serverIds.size());
destIds.push_back(serverIds[idx]);
healthyIds.push_back(serverIds[idx]);
for(int j = 0; j < serverIds.size(); j++) {
@ -1092,7 +1092,7 @@ ACTOR Future<bool> rebalanceTeams( DDQueueData* self, int priority, Reference<ID
if( !shards.size() )
return false;
state KeyRange moveShard = g_random->randomChoice( shards );
state KeyRange moveShard = deterministicRandom()->randomChoice( shards );
StorageMetrics metrics = wait( brokenPromiseToNever( self->getShardMetrics.getReply(GetMetricsRequest(moveShard)) ) );
int64_t sourceBytes = sourceTeam->getLoadBytes(false);

View File

@ -327,7 +327,7 @@ ACTOR Future<Void> shardSplitter(
//}
int numShards = splitKeys.size() - 1;
if( g_random->random01() < 0.01 ) {
if( deterministicRandom()->random01() < 0.01 ) {
TraceEvent("RelocateShardStartSplitx100", self->distributorId)
.detail("Begin", keys.begin)
.detail("End", keys.end)
@ -339,7 +339,7 @@ ACTOR Future<Void> shardSplitter(
}
if( numShards > 1 ) {
int skipRange = g_random->randomInt(0, numShards);
int skipRange = deterministicRandom()->randomInt(0, numShards);
// The queue can't deal with RelocateShard requests which split an existing shard into three pieces, so
// we have to send the unskipped ranges in this order (nibbling in from the edges of the old range)
for( int i = 0; i < skipRange; i++ )

View File

@ -164,9 +164,9 @@ public:
readyToPush(Void()), fileSizeWarningLimit(fileSizeWarningLimit), lastCommit(Void()), isFirstCommit(true)
{
if (BUGGIFY)
fileExtensionBytes = 1<<10 * g_random->randomSkewedUInt32( 1, 40<<10 );
fileExtensionBytes = 1<<10 * deterministicRandom()->randomSkewedUInt32( 1, 40<<10 );
if (BUGGIFY)
fileShrinkBytes = _PAGE_SIZE * g_random->randomSkewedUInt32( 1, 10<<10 );
fileShrinkBytes = _PAGE_SIZE * deterministicRandom()->randomSkewedUInt32( 1, 10<<10 );
files[0].dbgFilename = filename(0);
files[1].dbgFilename = filename(1);
// We issue reads into firstPages, so it needs to be 4k aligned.
@ -666,7 +666,7 @@ public:
}
// Read up to 1MB into readingBuffer
int len = std::min<int64_t>( (files[readingFile].size/sizeof(Page) - readingPage)*sizeof(Page), BUGGIFY_WITH_PROB(1.0) ? sizeof(Page)*g_random->randomInt(1,4) : (1<<20) );
int len = std::min<int64_t>( (files[readingFile].size/sizeof(Page) - readingPage)*sizeof(Page), BUGGIFY_WITH_PROB(1.0) ? sizeof(Page)*deterministicRandom()->randomInt(1,4) : (1<<20) );
readingBuffer.clear();
readingBuffer.alignReserve( sizeof(Page), len );
void* p = readingBuffer.append(len);
@ -687,7 +687,7 @@ public:
if (!self->readingBuffer.size()) {
state Future<Void> f = Void();
//if (BUGGIFY) f = delay( g_random->random01() * 0.1 );
//if (BUGGIFY) f = delay( deterministicRandom()->random01() * 0.1 );
int read = wait( self->fillReadingBuffer() );
ASSERT( read == self->readingBuffer.size() );

View File

@ -222,7 +222,7 @@ struct SQLiteDB : NonCopyable {
}
void checkError( const char* context, int rc ) {
//if (g_random->random01() < .001) rc = SQLITE_INTERRUPT;
//if (deterministicRandom()->random01() < .001) rc = SQLITE_INTERRUPT;
if (rc) {
// Our exceptions don't propagate through sqlite, so we don't know for sure if the error that caused this was
// an injected fault. Assume that if fault injection is happening, this is an injected fault.
@ -1292,7 +1292,7 @@ void SQLiteDB::open(bool writable) {
// Either we died partway through creating this DB, or died partway through deleting it, or someone is monkeying with our files
// Create a new blank DB by backing up the WAL file (just in case it is important) and then hitting the next case
walFile = file_not_found();
renameFile( walpath, walpath + "-old-" + g_random->randomUniqueID().toString() );
renameFile( walpath, walpath + "-old-" + deterministicRandom()->randomUniqueID().toString() );
ASSERT_WE_THINK(false); //< This code should not be hit in FoundationDB at the moment, because worker looks for databases to open by listing .fdb files, not .fdb-wal files
//TEST(true); // Replace a partially constructed or destructed DB
}
@ -1329,7 +1329,7 @@ void SQLiteDB::open(bool writable) {
if( !g_network->isSimulated() ) {
chunkSize = 4096 * SERVER_KNOBS->SQLITE_CHUNK_SIZE_PAGES;
} else if( BUGGIFY ) {
chunkSize = 4096 * g_random->randomInt(0, 100);
chunkSize = 4096 * deterministicRandom()->randomInt(0, 100);
} else {
chunkSize = 4096 * SERVER_KNOBS->SQLITE_CHUNK_SIZE_PAGES_SIM;
}
@ -1676,7 +1676,7 @@ private:
cursor = new Cursor(conn, true);
checkFreePages();
++writesComplete;
if (t3-a.issuedTime > 10.0*g_random->random01())
if (t3-a.issuedTime > 10.0*deterministicRandom()->random01())
TraceEvent("KVCommit10sSample", dbgid).detail("Queued", t1-a.issuedTime).detail("Commit", t2-t1).detail("Checkpoint", t3-t2);
diskBytesUsed = waitForAndGet( conn.dbFile->size() ) + waitForAndGet( conn.walFile->size() );
@ -1751,7 +1751,7 @@ private:
break;
}
if(canDelete && (!canVacuum || g_random->random01() < lazyDeleteBatchProbability)) {
if(canDelete && (!canVacuum || deterministicRandom()->random01() < lazyDeleteBatchProbability)) {
TEST(canVacuum); // SQLite lazy deletion when vacuuming is active
TEST(!canVacuum); // SQLite lazy deletion when vacuuming is inactive

View File

@ -62,7 +62,7 @@ ServerKnobs::ServerKnobs(bool randomize, ClientKnobs* clientKnobs) {
init( MAX_MESSAGE_SIZE, std::max<int>(LOG_SYSTEM_PUSHED_DATA_BLOCK_SIZE, 1e5 + 2e4 + 1) + 8 ); // VALUE_SIZE_LIMIT + SYSTEM_KEY_SIZE_LIMIT + 9 bytes (4 bytes for length, 4 bytes for sequence number, and 1 byte for mutation type)
init( TLOG_MESSAGE_BLOCK_BYTES, 10e6 );
init( TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR, double(TLOG_MESSAGE_BLOCK_BYTES) / (TLOG_MESSAGE_BLOCK_BYTES - MAX_MESSAGE_SIZE) ); //1.0121466709838096006362758832473
init( PEEK_TRACKER_EXPIRATION_TIME, 600 ); if( randomize && BUGGIFY ) PEEK_TRACKER_EXPIRATION_TIME = g_random->coinflip() ? 0.1 : 60;
init( PEEK_TRACKER_EXPIRATION_TIME, 600 ); if( randomize && BUGGIFY ) PEEK_TRACKER_EXPIRATION_TIME = deterministicRandom()->coinflip() ? 0.1 : 60;
init( PARALLEL_GET_MORE_REQUESTS, 32 ); if( randomize && BUGGIFY ) PARALLEL_GET_MORE_REQUESTS = 2;
init( MULTI_CURSOR_PRE_FETCH_LIMIT, 10 );
init( MAX_QUEUE_COMMIT_BYTES, 15e6 ); if( randomize && BUGGIFY ) MAX_QUEUE_COMMIT_BYTES = 5000;
@ -144,7 +144,7 @@ ServerKnobs::ServerKnobs(bool randomize, ClientKnobs* clientKnobs) {
If this value is too small relative to SHARD_MIN_BYTES_PER_KSEC immediate merging work will be generated.
*/
init( STORAGE_METRIC_TIMEOUT, 600.0 ); if( randomize && BUGGIFY ) STORAGE_METRIC_TIMEOUT = g_random->coinflip() ? 10.0 : 60.0;
init( STORAGE_METRIC_TIMEOUT, 600.0 ); if( randomize && BUGGIFY ) STORAGE_METRIC_TIMEOUT = deterministicRandom()->coinflip() ? 10.0 : 60.0;
init( METRIC_DELAY, 0.1 ); if( randomize && BUGGIFY ) METRIC_DELAY = 1.0;
init( ALL_DATA_REMOVED_DELAY, 1.0 );
init( INITIAL_FAILURE_REACTION_DELAY, 30.0 ); if( randomize && BUGGIFY ) INITIAL_FAILURE_REACTION_DELAY = 0.0;
@ -178,8 +178,8 @@ ServerKnobs::ServerKnobs(bool randomize, ClientKnobs* clientKnobs) {
init( DD_ZERO_HEALTHY_TEAM_DELAY, 1.0 );
// TeamRemover
TR_FLAG_DISABLE_TEAM_REMOVER = false; if( randomize && BUGGIFY ) TR_FLAG_DISABLE_TEAM_REMOVER = g_random->random01() < 0.1 ? true : false; // false by default. disable the consistency check when it's true
init( TR_REMOVE_MACHINE_TEAM_DELAY, 60.0 ); if( randomize && BUGGIFY ) TR_REMOVE_MACHINE_TEAM_DELAY = g_random->random01() * 60.0;
TR_FLAG_DISABLE_TEAM_REMOVER = false; if( randomize && BUGGIFY ) TR_FLAG_DISABLE_TEAM_REMOVER = deterministicRandom()->random01() < 0.1 ? true : false; // false by default. disable the consistency check when it's true
init( TR_REMOVE_MACHINE_TEAM_DELAY, 60.0 ); if( randomize && BUGGIFY ) TR_REMOVE_MACHINE_TEAM_DELAY = deterministicRandom()->random01() * 60.0;
// Redwood Storage Engine
init( PREFIX_TREE_IMMEDIATE_KEY_SIZE_LIMIT, 30 );
@ -224,17 +224,17 @@ ServerKnobs::ServerKnobs(bool randomize, ClientKnobs* clientKnobs) {
init( SQLITE_FRAGMENT_MIN_SAVINGS, 0.20 );
// KeyValueStoreSqlite spring cleaning
init( SPRING_CLEANING_NO_ACTION_INTERVAL, 1.0 ); if( randomize && BUGGIFY ) SPRING_CLEANING_NO_ACTION_INTERVAL = g_random->coinflip() ? 0.1 : g_random->random01() * 5;
init( SPRING_CLEANING_LAZY_DELETE_INTERVAL, 0.1 ); if( randomize && BUGGIFY ) SPRING_CLEANING_LAZY_DELETE_INTERVAL = g_random->coinflip() ? 1.0 : g_random->random01() * 5;
init( SPRING_CLEANING_VACUUM_INTERVAL, 1.0 ); if( randomize && BUGGIFY ) SPRING_CLEANING_VACUUM_INTERVAL = g_random->coinflip() ? 0.1 : g_random->random01() * 5;
init( SPRING_CLEANING_LAZY_DELETE_TIME_ESTIMATE, .010 ); if( randomize && BUGGIFY ) SPRING_CLEANING_LAZY_DELETE_TIME_ESTIMATE = g_random->random01() * 5;
init( SPRING_CLEANING_VACUUM_TIME_ESTIMATE, .010 ); if( randomize && BUGGIFY ) SPRING_CLEANING_VACUUM_TIME_ESTIMATE = g_random->random01() * 5;
init( SPRING_CLEANING_VACUUMS_PER_LAZY_DELETE_PAGE, 0.0 ); if( randomize && BUGGIFY ) SPRING_CLEANING_VACUUMS_PER_LAZY_DELETE_PAGE = g_random->coinflip() ? 1e9 : g_random->random01() * 5;
init( SPRING_CLEANING_MIN_LAZY_DELETE_PAGES, 0 ); if( randomize && BUGGIFY ) SPRING_CLEANING_MIN_LAZY_DELETE_PAGES = g_random->randomInt(1, 100);
init( SPRING_CLEANING_MAX_LAZY_DELETE_PAGES, 1e9 ); if( randomize && BUGGIFY ) SPRING_CLEANING_MAX_LAZY_DELETE_PAGES = g_random->coinflip() ? 0 : g_random->randomInt(1, 1e4);
init( SPRING_CLEANING_LAZY_DELETE_BATCH_SIZE, 100 ); if( randomize && BUGGIFY ) SPRING_CLEANING_LAZY_DELETE_BATCH_SIZE = g_random->randomInt(1, 1000);
init( SPRING_CLEANING_MIN_VACUUM_PAGES, 1 ); if( randomize && BUGGIFY ) SPRING_CLEANING_MIN_VACUUM_PAGES = g_random->randomInt(0, 100);
init( SPRING_CLEANING_MAX_VACUUM_PAGES, 1e9 ); if( randomize && BUGGIFY ) SPRING_CLEANING_MAX_VACUUM_PAGES = g_random->coinflip() ? 0 : g_random->randomInt(1, 1e4);
init( SPRING_CLEANING_NO_ACTION_INTERVAL, 1.0 ); if( randomize && BUGGIFY ) SPRING_CLEANING_NO_ACTION_INTERVAL = deterministicRandom()->coinflip() ? 0.1 : deterministicRandom()->random01() * 5;
init( SPRING_CLEANING_LAZY_DELETE_INTERVAL, 0.1 ); if( randomize && BUGGIFY ) SPRING_CLEANING_LAZY_DELETE_INTERVAL = deterministicRandom()->coinflip() ? 1.0 : deterministicRandom()->random01() * 5;
init( SPRING_CLEANING_VACUUM_INTERVAL, 1.0 ); if( randomize && BUGGIFY ) SPRING_CLEANING_VACUUM_INTERVAL = deterministicRandom()->coinflip() ? 0.1 : deterministicRandom()->random01() * 5;
init( SPRING_CLEANING_LAZY_DELETE_TIME_ESTIMATE, .010 ); if( randomize && BUGGIFY ) SPRING_CLEANING_LAZY_DELETE_TIME_ESTIMATE = deterministicRandom()->random01() * 5;
init( SPRING_CLEANING_VACUUM_TIME_ESTIMATE, .010 ); if( randomize && BUGGIFY ) SPRING_CLEANING_VACUUM_TIME_ESTIMATE = deterministicRandom()->random01() * 5;
init( SPRING_CLEANING_VACUUMS_PER_LAZY_DELETE_PAGE, 0.0 ); if( randomize && BUGGIFY ) SPRING_CLEANING_VACUUMS_PER_LAZY_DELETE_PAGE = deterministicRandom()->coinflip() ? 1e9 : deterministicRandom()->random01() * 5;
init( SPRING_CLEANING_MIN_LAZY_DELETE_PAGES, 0 ); if( randomize && BUGGIFY ) SPRING_CLEANING_MIN_LAZY_DELETE_PAGES = deterministicRandom()->randomInt(1, 100);
init( SPRING_CLEANING_MAX_LAZY_DELETE_PAGES, 1e9 ); if( randomize && BUGGIFY ) SPRING_CLEANING_MAX_LAZY_DELETE_PAGES = deterministicRandom()->coinflip() ? 0 : deterministicRandom()->randomInt(1, 1e4);
init( SPRING_CLEANING_LAZY_DELETE_BATCH_SIZE, 100 ); if( randomize && BUGGIFY ) SPRING_CLEANING_LAZY_DELETE_BATCH_SIZE = deterministicRandom()->randomInt(1, 1000);
init( SPRING_CLEANING_MIN_VACUUM_PAGES, 1 ); if( randomize && BUGGIFY ) SPRING_CLEANING_MIN_VACUUM_PAGES = deterministicRandom()->randomInt(0, 100);
init( SPRING_CLEANING_MAX_VACUUM_PAGES, 1e9 ); if( randomize && BUGGIFY ) SPRING_CLEANING_MAX_VACUUM_PAGES = deterministicRandom()->coinflip() ? 0 : deterministicRandom()->randomInt(1, 1e4);
// KeyValueStoreMemory
init( REPLACE_CONTENTS_BYTES, 1e5 ); if( randomize && BUGGIFY ) REPLACE_CONTENTS_BYTES = 1e3;
@ -265,7 +265,7 @@ ServerKnobs::ServerKnobs(bool randomize, ClientKnobs* clientKnobs) {
init( COMMIT_TRANSACTION_BATCH_INTERVAL_LATENCY_FRACTION, 0.1 );
init( COMMIT_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA, 0.1 );
init( COMMIT_TRANSACTION_BATCH_COUNT_MAX, 32768 ); if( randomize && BUGGIFY ) COMMIT_TRANSACTION_BATCH_COUNT_MAX = 1000; // Do NOT increase this number beyond 32768, as CommitIds only budget 2 bytes for storing transaction id within each batch
init( COMMIT_BATCHES_MEM_BYTES_HARD_LIMIT, 8LL << 30 ); if (randomize && BUGGIFY) COMMIT_BATCHES_MEM_BYTES_HARD_LIMIT = g_random->randomInt64(100LL << 20, 8LL << 30);
init( COMMIT_BATCHES_MEM_BYTES_HARD_LIMIT, 8LL << 30 ); if (randomize && BUGGIFY) COMMIT_BATCHES_MEM_BYTES_HARD_LIMIT = deterministicRandom()->randomInt64(100LL << 20, 8LL << 30);
init( COMMIT_BATCHES_MEM_FRACTION_OF_TOTAL, 0.5 );
init( COMMIT_BATCHES_MEM_TO_TOTAL_MEM_SCALE_FACTOR, 10.0 );
@ -277,7 +277,7 @@ ServerKnobs::ServerKnobs(bool randomize, ClientKnobs* clientKnobs) {
init( TRANSACTION_BUDGET_TIME, 0.050 ); if( randomize && BUGGIFY ) TRANSACTION_BUDGET_TIME = 0.0;
init( RESOLVER_COALESCE_TIME, 1.0 );
init( BUGGIFIED_ROW_LIMIT, APPLY_MUTATION_BYTES ); if( randomize && BUGGIFY ) BUGGIFIED_ROW_LIMIT = g_random->randomInt(3, 30);
init( BUGGIFIED_ROW_LIMIT, APPLY_MUTATION_BYTES ); if( randomize && BUGGIFY ) BUGGIFIED_ROW_LIMIT = deterministicRandom()->randomInt(3, 30);
init( PROXY_SPIN_DELAY, 0.01 );
init( UPDATE_REMOTE_LOG_VERSION_INTERVAL, 2.0 );
init( MAX_TXS_POP_VERSION_HISTORY, 1e5 );
@ -333,7 +333,7 @@ ServerKnobs::ServerKnobs(bool randomize, ClientKnobs* clientKnobs) {
init( EXPECTED_LOG_ROUTER_FITNESS, ProcessClass::UnsetFit );
init( EXPECTED_PROXY_FITNESS, ProcessClass::UnsetFit );
init( EXPECTED_RESOLVER_FITNESS, ProcessClass::UnsetFit );
init( RECRUITMENT_TIMEOUT, 600 ); if( randomize && BUGGIFY ) RECRUITMENT_TIMEOUT = g_random->coinflip() ? 60.0 : 1.0;
init( RECRUITMENT_TIMEOUT, 600 ); if( randomize && BUGGIFY ) RECRUITMENT_TIMEOUT = deterministicRandom()->coinflip() ? 60.0 : 1.0;
init( POLICY_RATING_TESTS, 200 ); if( randomize && BUGGIFY ) POLICY_RATING_TESTS = 20;
init( POLICY_GENERATIONS, 100 ); if( randomize && BUGGIFY ) POLICY_GENERATIONS = 10;

View File

@ -48,7 +48,7 @@ ACTOR Future<Void> submitCandidacy( Key key, LeaderElectionRegInterface coord, L
ACTOR template <class T> Future<Void> buggifyDelayedAsyncVar( Reference<AsyncVar<T>> in, Reference<AsyncVar<T>> out ) {
try {
loop {
wait( delay( SERVER_KNOBS->BUGGIFIED_EVENTUAL_CONSISTENCY * g_random->random01() ) );
wait( delay( SERVER_KNOBS->BUGGIFIED_EVENTUAL_CONSISTENCY * deterministicRandom()->random01() ) );
out->set( in->get() );
wait( in->onChange() );
}
@ -99,7 +99,7 @@ ACTOR Future<Void> tryBecomeLeaderInternal( ServerCoordinators coordinators, Val
while (!iAmLeader) {
state Future<Void> badCandidateTimeout;
myInfo.changeID = g_random->randomUniqueID();
myInfo.changeID = deterministicRandom()->randomUniqueID();
prevChangeID = myInfo.changeID;
myInfo.updateChangeID( asyncPriorityInfo->get() );
@ -214,7 +214,7 @@ ACTOR Future<Void> tryBecomeLeaderInternal( ServerCoordinators coordinators, Val
wait( rate );
}
if (SERVER_KNOBS->BUGGIFY_ALL_COORDINATION || BUGGIFY) wait( delay( SERVER_KNOBS->BUGGIFIED_EVENTUAL_CONSISTENCY * g_random->random01() ) );
if (SERVER_KNOBS->BUGGIFY_ALL_COORDINATION || BUGGIFY) wait( delay( SERVER_KNOBS->BUGGIFIED_EVENTUAL_CONSISTENCY * deterministicRandom()->random01() ) );
return Void(); // We are no longer leader
}

View File

@ -25,14 +25,14 @@
#include "flow/actorcompiler.h" // has to be last include
ILogSystem::ServerPeekCursor::ServerPeekCursor( Reference<AsyncVar<OptionalInterface<TLogInterface>>> const& interf, Tag tag, Version begin, Version end, bool returnIfBlocked, bool parallelGetMore )
: interf(interf), tag(tag), messageVersion(begin), end(end), hasMsg(false), rd(results.arena, results.messages, Unversioned()), randomID(g_random->randomUniqueID()), poppedVersion(0), returnIfBlocked(returnIfBlocked), sequence(0), parallelGetMore(parallelGetMore) {
: interf(interf), tag(tag), messageVersion(begin), end(end), hasMsg(false), rd(results.arena, results.messages, Unversioned()), randomID(deterministicRandom()->randomUniqueID()), poppedVersion(0), returnIfBlocked(returnIfBlocked), sequence(0), parallelGetMore(parallelGetMore) {
this->results.maxKnownVersion = 0;
this->results.minKnownCommittedVersion = 0;
//TraceEvent("SPC_Starting", randomID).detail("Tag", tag.toString()).detail("Begin", begin).detail("End", end).backtrace();
}
ILogSystem::ServerPeekCursor::ServerPeekCursor( TLogPeekReply const& results, LogMessageVersion const& messageVersion, LogMessageVersion const& end, int32_t messageLength, int32_t rawLength, bool hasMsg, Version poppedVersion, Tag tag )
: results(results), tag(tag), rd(results.arena, results.messages, Unversioned()), messageVersion(messageVersion), end(end), messageLength(messageLength), rawLength(rawLength), hasMsg(hasMsg), randomID(g_random->randomUniqueID()), poppedVersion(poppedVersion), returnIfBlocked(false), sequence(0), parallelGetMore(false)
: results(results), tag(tag), rd(results.arena, results.messages, Unversioned()), messageVersion(messageVersion), end(end), messageLength(messageLength), rawLength(rawLength), hasMsg(hasMsg), randomID(deterministicRandom()->randomUniqueID()), poppedVersion(poppedVersion), returnIfBlocked(false), sequence(0), parallelGetMore(false)
{
//TraceEvent("SPC_Clone", randomID);
this->results.maxKnownVersion = 0;
@ -170,7 +170,7 @@ ACTOR Future<Void> serverPeekParallelGetMore( ILogSystem::ServerPeekCursor* self
}
when( wait( self->interfaceChanged ) ) {
self->interfaceChanged = self->interf->onChange();
self->randomID = g_random->randomUniqueID();
self->randomID = deterministicRandom()->randomUniqueID();
self->sequence = 0;
self->futureResults.clear();
}
@ -182,7 +182,7 @@ ACTOR Future<Void> serverPeekParallelGetMore( ILogSystem::ServerPeekCursor* self
} else if(e.code() == error_code_timed_out) {
TraceEvent("PeekCursorTimedOut", self->randomID);
self->interfaceChanged = self->interf->onChange();
self->randomID = g_random->randomUniqueID();
self->randomID = deterministicRandom()->randomUniqueID();
self->sequence = 0;
self->futureResults.clear();
} else {
@ -268,13 +268,13 @@ Version ILogSystem::ServerPeekCursor::popped() { return poppedVersion; }
ILogSystem::MergedPeekCursor::MergedPeekCursor( vector< Reference<ILogSystem::IPeekCursor> > const& serverCursors, Version begin )
: serverCursors(serverCursors), bestServer(-1), readQuorum(serverCursors.size()), tag(invalidTag), currentCursor(0), hasNextMessage(false),
messageVersion(begin), randomID(g_random->randomUniqueID()), tLogReplicationFactor(0) {
messageVersion(begin), randomID(deterministicRandom()->randomUniqueID()), tLogReplicationFactor(0) {
sortedVersions.resize(serverCursors.size());
}
ILogSystem::MergedPeekCursor::MergedPeekCursor( std::vector<Reference<AsyncVar<OptionalInterface<TLogInterface>>>> const& logServers, int bestServer, int readQuorum, Tag tag, Version begin, Version end,
bool parallelGetMore, std::vector< LocalityData > const& tLogLocalities, Reference<IReplicationPolicy> const tLogPolicy, int tLogReplicationFactor )
: bestServer(bestServer), readQuorum(readQuorum), tag(tag), currentCursor(0), hasNextMessage(false), messageVersion(begin), randomID(g_random->randomUniqueID()), tLogReplicationFactor(tLogReplicationFactor) {
: bestServer(bestServer), readQuorum(readQuorum), tag(tag), currentCursor(0), hasNextMessage(false), messageVersion(begin), randomID(deterministicRandom()->randomUniqueID()), tLogReplicationFactor(tLogReplicationFactor) {
if(tLogPolicy) {
logSet = Reference<LogSet>( new LogSet() );
logSet->tLogPolicy = tLogPolicy;
@ -293,7 +293,7 @@ ILogSystem::MergedPeekCursor::MergedPeekCursor( std::vector<Reference<AsyncVar<O
ILogSystem::MergedPeekCursor::MergedPeekCursor( vector< Reference<ILogSystem::IPeekCursor> > const& serverCursors, LogMessageVersion const& messageVersion, int bestServer, int readQuorum, Optional<LogMessageVersion> nextVersion, Reference<LogSet> logSet, int tLogReplicationFactor )
: serverCursors(serverCursors), bestServer(bestServer), readQuorum(readQuorum), currentCursor(0), hasNextMessage(false), messageVersion(messageVersion), nextVersion(nextVersion), logSet(logSet),
randomID(g_random->randomUniqueID()), tLogReplicationFactor(tLogReplicationFactor) {
randomID(deterministicRandom()->randomUniqueID()), tLogReplicationFactor(tLogReplicationFactor) {
sortedVersions.resize(serverCursors.size());
calcHasMessage();
}
@ -497,7 +497,7 @@ Version ILogSystem::MergedPeekCursor::popped() {
}
ILogSystem::SetPeekCursor::SetPeekCursor( std::vector<Reference<LogSet>> const& logSets, int bestSet, int bestServer, Tag tag, Version begin, Version end, bool parallelGetMore )
: logSets(logSets), bestSet(bestSet), bestServer(bestServer), tag(tag), currentCursor(0), currentSet(bestSet), hasNextMessage(false), messageVersion(begin), useBestSet(true), randomID(g_random->randomUniqueID()) {
: logSets(logSets), bestSet(bestSet), bestServer(bestServer), tag(tag), currentCursor(0), currentSet(bestSet), hasNextMessage(false), messageVersion(begin), useBestSet(true), randomID(deterministicRandom()->randomUniqueID()) {
serverCursors.resize(logSets.size());
int maxServers = 0;
for( int i = 0; i < logSets.size(); i++ ) {
@ -512,7 +512,7 @@ ILogSystem::SetPeekCursor::SetPeekCursor( std::vector<Reference<LogSet>> const&
ILogSystem::SetPeekCursor::SetPeekCursor( std::vector<Reference<LogSet>> const& logSets, std::vector< std::vector< Reference<IPeekCursor> > > const& serverCursors, LogMessageVersion const& messageVersion, int bestSet, int bestServer,
Optional<LogMessageVersion> nextVersion, bool useBestSet ) : logSets(logSets), serverCursors(serverCursors), messageVersion(messageVersion), bestSet(bestSet), bestServer(bestServer), nextVersion(nextVersion), currentSet(bestSet), currentCursor(0),
hasNextMessage(false), useBestSet(useBestSet), randomID(g_random->randomUniqueID()) {
hasNextMessage(false), useBestSet(useBestSet), randomID(deterministicRandom()->randomUniqueID()) {
int maxServers = 0;
for( int i = 0; i < logSets.size(); i++ ) {
maxServers = std::max<int>(maxServers, serverCursors[i].size());

View File

@ -431,13 +431,13 @@ ACTOR Future<Void> commitBatch(
for (int t = 0; t<trs.size(); t++) {
if (trs[t].debugID.present()) {
if (!debugID.present())
debugID = g_nondeterministic_random->randomUniqueID();
debugID = nondeterministicRandom()->randomUniqueID();
g_traceBatch.addAttach("CommitAttachID", trs[t].debugID.get().first(), debugID.get().first());
}
}
if(localBatchNumber == 2 && !debugID.present() && self->firstProxy && !g_network->isSimulated()) {
debugID = g_random->randomUniqueID();
debugID = deterministicRandom()->randomUniqueID();
TraceEvent("SecondCommitBatch", self->dbgid).detail("DebugID", debugID.get());
}
@ -1143,7 +1143,7 @@ ACTOR static Future<Void> transactionStarter(
}
if (req.debugID.present()) {
if (!debugID.present()) debugID = g_nondeterministic_random->randomUniqueID();
if (!debugID.present()) debugID = nondeterministicRandom()->randomUniqueID();
g_traceBatch.addAttach("TransactionAttachID", req.debugID.get().first(), debugID.get().first());
}

View File

@ -35,7 +35,7 @@ ACTOR Future<MoveKeysLock> takeMoveKeysLock( Database cx, UID masterId ) {
state MoveKeysLock lock;
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
if( !g_network->isSimulated() ) {
UID id(g_random->randomUniqueID());
UID id(deterministicRandom()->randomUniqueID());
TraceEvent("TakeMoveKeysLockTransaction", masterId)
.detail("TransactionUID", id);
tr.debugTransaction( id );
@ -48,7 +48,7 @@ ACTOR Future<MoveKeysLock> takeMoveKeysLock( Database cx, UID masterId ) {
Optional<Value> readVal = wait( tr.get( moveKeysLockWriteKey ) );
lock.prevWrite = readVal.present() ? BinaryReader::fromStringRef<UID>(readVal.get(), Unversioned()) : UID();
}
lock.myOwner = g_random->randomUniqueID();
lock.myOwner = deterministicRandom()->randomUniqueID();
return lock;
} catch (Error &e){
wait(tr.onError(e));
@ -74,7 +74,7 @@ ACTOR Future<Void> checkMoveKeysLock( Transaction* tr, MoveKeysLock lock, bool i
if(isWrite) {
BinaryWriter wrMyOwner(Unversioned()); wrMyOwner << lock.myOwner;
tr->set( moveKeysLockOwnerKey, wrMyOwner.toValue() );
BinaryWriter wrLastWrite(Unversioned()); wrLastWrite << g_random->randomUniqueID();
BinaryWriter wrLastWrite(Unversioned()); wrLastWrite << deterministicRandom()->randomUniqueID();
tr->set( moveKeysLockWriteKey, wrLastWrite.toValue() );
}
@ -82,7 +82,7 @@ ACTOR Future<Void> checkMoveKeysLock( Transaction* tr, MoveKeysLock lock, bool i
} else if (currentOwner == lock.myOwner) {
if(isWrite) {
// Touch the lock, preventing overlapping attempts to take it
BinaryWriter wrLastWrite(Unversioned()); wrLastWrite << g_random->randomUniqueID();
BinaryWriter wrLastWrite(Unversioned()); wrLastWrite << deterministicRandom()->randomUniqueID();
tr->set( moveKeysLockWriteKey, wrLastWrite.toValue() );
// Make this transaction self-conflicting so the database will not execute it twice with the same write key
tr->makeSelfConflicting();
@ -727,7 +727,7 @@ ACTOR Future<std::pair<Version, Tag>> addStorageServer( Database cx, StorageServ
tr.set( tagLocalityListKeyFor(server.locality.dcId()), tagLocalityListValue(locality) );
}
int skipTags = g_random->randomInt(0, maxSkipTags);
int skipTags = deterministicRandom()->randomInt(0, maxSkipTags);
state uint16_t tagId = 0;
std::vector<uint16_t> usedTags;

View File

@ -304,7 +304,7 @@ namespace oldTLog_4_6 {
bool terminated;
TLogData(UID dbgid, IKeyValueStore* persistentData, IDiskQueue * persistentQueue, Reference<AsyncVar<ServerDBInfo>> const& dbInfo)
: dbgid(dbgid), instanceID(g_random->randomUniqueID().first()),
: dbgid(dbgid), instanceID(deterministicRandom()->randomUniqueID().first()),
persistentData(persistentData), rawPersistentQueue(persistentQueue), persistentQueue(new TLogQueue(persistentQueue, dbgid)),
dbInfo(dbInfo), queueCommitBegin(0), queueCommitEnd(0), prevVersion(0),
diskQueueCommitBytes(0), largeDiskQueueCommitBytes(false),
@ -1078,7 +1078,7 @@ namespace oldTLog_4_6 {
{
TraceEvent("TLogDisplaced", tli.id()).detail("Reason", "DBInfoDoesNotContain").detail("RecoveryCount", recoveryCount).detail("InfRecoveryCount", inf.recoveryCount).detail("RecoveryState", (int)inf.recoveryState)
.detail("LogSysConf", describe(inf.logSystemConfig.tLogs)).detail("PriorLogs", describe(inf.priorCommittedLogServers)).detail("OldLogGens", inf.logSystemConfig.oldTLogs.size());
if (BUGGIFY) wait( delay( SERVER_KNOBS->BUGGIFY_WORKER_REMOVED_MAX_LAG * g_random->random01() ) );
if (BUGGIFY) wait( delay( SERVER_KNOBS->BUGGIFY_WORKER_REMOVED_MAX_LAG * deterministicRandom()->random01() ) );
throw worker_removed();
}
@ -1158,7 +1158,7 @@ namespace oldTLog_4_6 {
}
when (TLogConfirmRunningRequest req = waitNext(tli.confirmRunning.getFuture())){
if (req.debugID.present() ) {
UID tlogDebugID = g_nondeterministic_random->randomUniqueID();
UID tlogDebugID = nondeterministicRandom()->randomUniqueID();
g_traceBatch.addAttach("TransactionAttachID", req.debugID.get().first(), tlogDebugID.first());
g_traceBatch.addEvent("TransactionDebug", tlogDebugID.first(), "TLogServer.TLogConfirmRunningRequest");
}

View File

@ -277,7 +277,7 @@ struct TLogData : NonCopyable {
Reference<AsyncVar<bool>> degraded;
TLogData(UID dbgid, IKeyValueStore* persistentData, IDiskQueue * persistentQueue, Reference<AsyncVar<ServerDBInfo>> dbInfo, Reference<AsyncVar<bool>> degraded)
: dbgid(dbgid), instanceID(g_random->randomUniqueID().first()),
: dbgid(dbgid), instanceID(deterministicRandom()->randomUniqueID().first()),
persistentData(persistentData), rawPersistentQueue(persistentQueue), persistentQueue(new TLogQueue(persistentQueue, dbgid)),
dbInfo(dbInfo), degraded(degraded), queueCommitBegin(0), queueCommitEnd(0),
diskQueueCommitBytes(0), largeDiskQueueCommitBytes(false), bytesInput(0), bytesDurable(0), overheadBytesInput(0), overheadBytesDurable(0),
@ -1225,7 +1225,7 @@ ACTOR Future<Void> tLogCommit(
state Optional<UID> tlogDebugID;
if(req.debugID.present())
{
tlogDebugID = g_nondeterministic_random->randomUniqueID();
tlogDebugID = nondeterministicRandom()->randomUniqueID();
g_traceBatch.addAttach("CommitAttachID", req.debugID.get().first(), tlogDebugID.get().first());
g_traceBatch.addEvent("CommitDebug", tlogDebugID.get().first(), "TLog.tLogCommit.BeforeWaitForVersion");
}
@ -1358,7 +1358,7 @@ ACTOR Future<Void> rejoinMasters( TLogData* self, TLogInterface tli, DBRecoveryC
{
TraceEvent("TLogDisplaced", tli.id()).detail("Reason", "DBInfoDoesNotContain").detail("RecoveryCount", recoveryCount).detail("InfRecoveryCount", inf.recoveryCount).detail("RecoveryState", (int)inf.recoveryState)
.detail("LogSysConf", describe(inf.logSystemConfig.tLogs)).detail("PriorLogs", describe(inf.priorCommittedLogServers)).detail("OldLogGens", inf.logSystemConfig.oldTLogs.size());
if (BUGGIFY) wait( delay( SERVER_KNOBS->BUGGIFY_WORKER_REMOVED_MAX_LAG * g_random->random01() ) );
if (BUGGIFY) wait( delay( SERVER_KNOBS->BUGGIFY_WORKER_REMOVED_MAX_LAG * deterministicRandom()->random01() ) );
throw worker_removed();
}
@ -1491,7 +1491,7 @@ ACTOR Future<Void> serveTLogInterface( TLogData* self, TLogInterface tli, Refere
}
when (TLogConfirmRunningRequest req = waitNext(tli.confirmRunning.getFuture())){
if (req.debugID.present() ) {
UID tlogDebugID = g_nondeterministic_random->randomUniqueID();
UID tlogDebugID = nondeterministicRandom()->randomUniqueID();
g_traceBatch.addAttach("TransactionAttachID", req.debugID.get().first(), tlogDebugID.first());
g_traceBatch.addEvent("TransactionDebug", tlogDebugID.first(), "TLogServer.TLogConfirmRunningRequest");
}
@ -2198,12 +2198,12 @@ TEST_CASE("/fdbserver/tlogserver/VersionMessagesOverheadFactor" ) {
DequeAllocator<TestType> allocator;
std::deque<TestType, DequeAllocator<TestType>> d(allocator);
int numElements = g_random->randomInt(pow(10, i-1), pow(10, i));
int numElements = deterministicRandom()->randomInt(pow(10, i-1), pow(10, i));
for(int k = 0; k < numElements; ++k) {
d.push_back(TestType());
}
int removedElements = 0;//g_random->randomInt(0, numElements); // FIXME: the overhead factor does not accurately account for removal!
int removedElements = 0;//deterministicRandom()->randomInt(0, numElements); // FIXME: the overhead factor does not accurately account for removal!
for(int k = 0; k < removedElements; ++k) {
d.pop_front();
}

View File

@ -399,7 +399,7 @@ ACTOR Future<Void> reconfigureAfter(Database cx, double time, Reference<AsyncVar
ACTOR Future<Void> waitForQuietDatabase( Database cx, Reference<AsyncVar<ServerDBInfo>> dbInfo, std::string phase, int64_t dataInFlightGate = 2e6,
int64_t maxTLogQueueGate = 5e6, int64_t maxStorageServerQueueGate = 5e6, int64_t maxDataDistributionQueueSize = 0 ) {
state Future<Void> reconfig = reconfigureAfter(cx, 100 + (g_random->random01()*100), dbInfo, "QuietDatabase");
state Future<Void> reconfig = reconfigureAfter(cx, 100 + (deterministicRandom()->random01()*100), dbInfo, "QuietDatabase");
TraceEvent(("QuietDatabase" + phase + "Begin").c_str());

View File

@ -393,7 +393,7 @@ void updateRate(RatekeeperData* self, RatekeeperLimits* limits) {
double inputRate = ss.smoothInputBytes.smoothRate();
//inputRate = std::max( inputRate, actualTps / SERVER_KNOBS->MAX_TRANSACTIONS_PER_BYTE );
/*if( g_random->random01() < 0.1 ) {
/*if( deterministicRandom()->random01() < 0.1 ) {
std::string name = "RatekeeperUpdateRate" + limits.context;
TraceEvent(name, ss.id)
.detail("MinFreeSpace", minFreeSpace)
@ -584,7 +584,7 @@ void updateRate(RatekeeperData* self, RatekeeperLimits* limits) {
limits->tpsLimitMetric = std::min(limits->tpsLimit, 1e6);
limits->reasonMetric = limitReason;
if (g_random->random01() < 0.1) {
if (deterministicRandom()->random01() < 0.1) {
std::string name = "RkUpdate" + limits->context;
TraceEvent(name.c_str())
.detail("TPSLimit", limits->tpsLimit)

View File

@ -79,7 +79,7 @@ ACTOR Future<Void> resolveBatch(
state ProxyRequestsInfo &proxyInfo = self->proxyInfoMap[proxyAddress];
if(req.debugID.present()) {
debugID = g_nondeterministic_random->randomUniqueID();
debugID = nondeterministicRandom()->randomUniqueID();
g_traceBatch.addAttach("CommitAttachID", req.debugID.get().first(), debugID.get().first());
g_traceBatch.addEvent("CommitDebug",debugID.get().first(),"Resolver.resolveBatch.Before");
}

View File

@ -36,7 +36,7 @@ struct ResolverInterface {
RequestStream<ReplyPromise<Void>> waitFailure;
ResolverInterface() : uniqueID( g_random->randomUniqueID() ) {}
ResolverInterface() : uniqueID( deterministicRandom()->randomUniqueID() ) {}
UID id() const { return uniqueID; }
std::string toString() const { return id().shortString(); }
bool operator == ( ResolverInterface const& r ) const { return id() == r.id(); }

View File

@ -198,11 +198,11 @@ ACTOR Future<ISimulator::KillType> simulatedFDBDRebooter(Reference<ClusterConnec
std::string baseFolder, ClusterConnectionString connStr,
bool useSeedFile, bool runBackupAgents) {
state ISimulator::ProcessInfo *simProcess = g_simulator.getCurrentProcess();
state UID randomId = g_nondeterministic_random->randomUniqueID();
state UID randomId = nondeterministicRandom()->randomUniqueID();
state int cycles = 0;
loop {
auto waitTime = SERVER_KNOBS->MIN_REBOOT_TIME + (SERVER_KNOBS->MAX_REBOOT_TIME - SERVER_KNOBS->MIN_REBOOT_TIME) * g_random->random01();
auto waitTime = SERVER_KNOBS->MIN_REBOOT_TIME + (SERVER_KNOBS->MAX_REBOOT_TIME - SERVER_KNOBS->MIN_REBOOT_TIME) * deterministicRandom()->random01();
cycles ++;
TraceEvent("SimulatedFDBDPreWait").detail("Cycles", cycles).detail("RandomId", randomId)
.detail("Address", NetworkAddress(ip, port, true, false))
@ -319,8 +319,8 @@ ACTOR Future<ISimulator::KillType> simulatedFDBDRebooter(Reference<ClusterConnec
.detail("Address", process->address)
.detail("ZoneId", localities.zoneId())
.detail("KillType", shutdownResult);
*coordFolder = joinPath(baseFolder, g_random->randomUniqueID().toString());
*dataFolder = joinPath(baseFolder, g_random->randomUniqueID().toString());
*coordFolder = joinPath(baseFolder, deterministicRandom()->randomUniqueID().toString());
*dataFolder = joinPath(baseFolder, deterministicRandom()->randomUniqueID().toString());
platform::createDirectory( *dataFolder );
if(!useSeedFile) {
@ -360,7 +360,7 @@ ACTOR Future<Void> simulatedMachine(ClusterConnectionString connStr, std::vector
state int bootCount = 0;
state std::vector<std::string> myFolders;
state std::vector<std::string> coordFolders;
state UID randomId = g_nondeterministic_random->randomUniqueID();
state UID randomId = nondeterministicRandom()->randomUniqueID();
state int listenPerProcess = (sslEnabled && !sslOnly) ? 2 : 1;
try {
@ -370,20 +370,20 @@ ACTOR Future<Void> simulatedMachine(ClusterConnectionString connStr, std::vector
for (int i = 0; i < ips.size(); i++) {
if (restarting) {
myFolders.push_back( ini.GetValue(printable(localities.machineId()).c_str(), format("%d", i*listenPerProcess).c_str(), joinPath(baseFolder, g_random->randomUniqueID().toString()).c_str()) );
myFolders.push_back( ini.GetValue(printable(localities.machineId()).c_str(), format("%d", i*listenPerProcess).c_str(), joinPath(baseFolder, deterministicRandom()->randomUniqueID().toString()).c_str()) );
if(i == 0) {
std::string coordinationFolder = ini.GetValue(printable(localities.machineId()).c_str(), "coordinationFolder", "");
if(!coordinationFolder.size())
coordinationFolder = ini.GetValue(printable(localities.machineId()).c_str(), format("c%d", i*listenPerProcess).c_str(), joinPath(baseFolder, g_random->randomUniqueID().toString()).c_str());
coordinationFolder = ini.GetValue(printable(localities.machineId()).c_str(), format("c%d", i*listenPerProcess).c_str(), joinPath(baseFolder, deterministicRandom()->randomUniqueID().toString()).c_str());
coordFolders.push_back(coordinationFolder);
} else {
coordFolders.push_back( ini.GetValue(printable(localities.machineId()).c_str(), format("c%d", i*listenPerProcess).c_str(), joinPath(baseFolder, g_random->randomUniqueID().toString()).c_str()) );
coordFolders.push_back( ini.GetValue(printable(localities.machineId()).c_str(), format("c%d", i*listenPerProcess).c_str(), joinPath(baseFolder, deterministicRandom()->randomUniqueID().toString()).c_str()) );
}
}
else {
coordFolders.push_back( joinPath(baseFolder, g_random->randomUniqueID().toString()) );
std::string thisFolder = g_random->randomUniqueID().toString();
coordFolders.push_back( joinPath(baseFolder, deterministicRandom()->randomUniqueID().toString()) );
std::string thisFolder = deterministicRandom()->randomUniqueID().toString();
myFolders.push_back( joinPath(baseFolder, thisFolder ) );
platform::createDirectory( myFolders[i] );
@ -528,7 +528,7 @@ ACTOR Future<Void> simulatedMachine(ClusterConnectionString connStr, std::vector
if( swap )
availableFolders[localities.dcId()].push_back( myFolders );
auto rebootTime = g_random->random01() * MACHINE_REBOOT_TIME;
auto rebootTime = deterministicRandom()->random01() * MACHINE_REBOOT_TIME;
TraceEvent("SimulatedMachineShutdown", randomId)
.detail("Swap", swap)
@ -542,7 +542,7 @@ ACTOR Future<Void> simulatedMachine(ClusterConnectionString connStr, std::vector
if( swap ) {
auto& avail = availableFolders[localities.dcId()];
int i = g_random->randomInt(0, avail.size());
int i = deterministicRandom()->randomInt(0, avail.size());
if( i != avail.size() - 1 )
std::swap( avail[i], avail.back() );
auto toRebootFrom = avail.back();
@ -564,8 +564,8 @@ ACTOR Future<Void> simulatedMachine(ClusterConnectionString connStr, std::vector
}
} else if( killType == ISimulator::RebootAndDelete ) {
for( int i = 0; i < ips.size(); i++ ) {
coordFolders[i] = joinPath(baseFolder, g_random->randomUniqueID().toString());
myFolders[i] = joinPath(baseFolder, g_random->randomUniqueID().toString());
coordFolders[i] = joinPath(baseFolder, deterministicRandom()->randomUniqueID().toString());
myFolders[i] = joinPath(baseFolder, deterministicRandom()->randomUniqueID().toString());
platform::createDirectory( myFolders[i] );
if(!useSeedFile) {
@ -628,7 +628,7 @@ ACTOR Future<Void> restartSimulatedSystem(
*pConnString = conn;
*pTesterCount = testerCount;
bool usingSSL = conn.toString().find(":tls") != std::string::npos || listenersPerProcess > 1;
int useSeedForMachine = g_random->randomInt(0, machineCount);
int useSeedForMachine = deterministicRandom()->randomInt(0, machineCount);
std::vector<std::string> dcIds;
for( int i = 0; i < machineCount; i++) {
Optional<Standalone<StringRef>> dcUID;
@ -785,12 +785,12 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
// if we do not remove the surplus server and machine teams, the simulation test will report error.
// This is needed to make sure the number of server (and machine) teams is no larger than the desired number.
bool generateMachineTeamTestConfig = BUGGIFY_WITH_PROB(0.1) ? true : false;
bool generateFearless = simple ? false : (minimumRegions > 1 || g_random->random01() < 0.5);
datacenters = simple ? 1 : ( generateFearless ? ( minimumReplication > 0 || g_random->random01() < 0.5 ? 4 : 6 ) : g_random->randomInt( 1, 4 ) );
if (g_random->random01() < 0.25) db.desiredTLogCount = g_random->randomInt(1,7);
if (g_random->random01() < 0.25) db.masterProxyCount = g_random->randomInt(1,7);
if (g_random->random01() < 0.25) db.resolverCount = g_random->randomInt(1,7);
if (g_random->random01() < 0.5) {
bool generateFearless = simple ? false : (minimumRegions > 1 || deterministicRandom()->random01() < 0.5);
datacenters = simple ? 1 : ( generateFearless ? ( minimumReplication > 0 || deterministicRandom()->random01() < 0.5 ? 4 : 6 ) : deterministicRandom()->randomInt( 1, 4 ) );
if (deterministicRandom()->random01() < 0.25) db.desiredTLogCount = deterministicRandom()->randomInt(1,7);
if (deterministicRandom()->random01() < 0.25) db.masterProxyCount = deterministicRandom()->randomInt(1,7);
if (deterministicRandom()->random01() < 0.25) db.resolverCount = deterministicRandom()->randomInt(1,7);
if (deterministicRandom()->random01() < 0.5) {
set_config("ssd");
} else {
set_config("memory");
@ -801,14 +801,14 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
db.resolverCount = 1;
}
int replication_type = simple ? 1 : ( std::max(minimumReplication, datacenters > 4 ? g_random->randomInt(1,3) : std::min(g_random->randomInt(0,6), 3)) );
int replication_type = simple ? 1 : ( std::max(minimumReplication, datacenters > 4 ? deterministicRandom()->randomInt(1,3) : std::min(deterministicRandom()->randomInt(0,6), 3)) );
switch (replication_type) {
case 0: {
TEST( true ); // Simulated cluster using custom redundancy mode
int storage_servers = g_random->randomInt(1, generateFearless ? 4 : 5);
int storage_servers = deterministicRandom()->randomInt(1, generateFearless ? 4 : 5);
//FIXME: log replicas must be more than storage replicas because otherwise better master exists will not recognize it needs to change dcs
int replication_factor = g_random->randomInt(storage_servers, generateFearless ? 4 : 5);
int anti_quorum = g_random->randomInt(0, (replication_factor/2) + 1); //The anti quorum cannot be more than half of the replication factor, or the log system will continue to accept commits when a recovery is impossible
int replication_factor = deterministicRandom()->randomInt(storage_servers, generateFearless ? 4 : 5);
int anti_quorum = deterministicRandom()->randomInt(0, (replication_factor/2) + 1); //The anti quorum cannot be more than half of the replication factor, or the log system will continue to accept commits when a recovery is impossible
// Go through buildConfiguration, as it sets tLogPolicy/storagePolicy.
set_config(format("storage_replicas:=%d log_replicas:=%d log_anti_quorum:=%d "
"replica_datacenters:=1 min_replica_datacenters:=1",
@ -843,11 +843,11 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
ASSERT(false); // Programmer forgot to adjust cases.
}
if (g_random->random01() < 0.5) {
if (g_random->random01() < 0.5) {
if (deterministicRandom()->random01() < 0.5) {
if (deterministicRandom()->random01() < 0.5) {
set_config("log_spill:=1"); // VALUE
}
int logVersion = g_random->randomInt( 0, 3 );
int logVersion = deterministicRandom()->randomInt( 0, 3 );
switch (logVersion) {
case 0:
break;
@ -863,7 +863,7 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
set_config("log_spill:=2"); // REFERENCE
}
if(generateFearless || (datacenters == 2 && g_random->random01() < 0.5)) {
if(generateFearless || (datacenters == 2 && deterministicRandom()->random01() < 0.5)) {
//The kill region workload relies on the fact that all "0", "2", and "4" are all of the possible primary dcids.
StatusObject primaryObj;
StatusObject primaryDcObj;
@ -883,7 +883,7 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
if(generateFearless) {
if(datacenters > 4) {
//FIXME: we cannot use one satellite replication with more than one satellite per region because canKillProcesses does not respect usable_dcs
int satellite_replication_type = g_random->randomInt(0,3);
int satellite_replication_type = deterministicRandom()->randomInt(0,3);
switch (satellite_replication_type) {
case 0: {
TEST( true ); // Simulated cluster using no satellite redundancy mode
@ -905,7 +905,7 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
ASSERT(false); // Programmer forgot to adjust cases.
}
} else {
int satellite_replication_type = g_random->randomInt(0,5);
int satellite_replication_type = deterministicRandom()->randomInt(0,5);
switch (satellite_replication_type) {
case 0: {
//FIXME: implement
@ -939,14 +939,14 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
}
}
if (g_random->random01() < 0.25) {
int logs = g_random->randomInt(1,7);
if (deterministicRandom()->random01() < 0.25) {
int logs = deterministicRandom()->randomInt(1,7);
primaryObj["satellite_logs"] = logs;
remoteObj["satellite_logs"] = logs;
}
//We cannot run with a remote DC when MAX_READ_TRANSACTION_LIFE_VERSIONS is too small, because the log routers will not be able to keep up.
if (minimumRegions <= 1 && (g_random->random01() < 0.25 || SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS < SERVER_KNOBS->VERSIONS_PER_SECOND)) {
if (minimumRegions <= 1 && (deterministicRandom()->random01() < 0.25 || SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS < SERVER_KNOBS->VERSIONS_PER_SECOND)) {
TEST( true ); // Simulated cluster using one region
needsRemote = false;
} else {
@ -954,7 +954,7 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
db.usableRegions = 2;
}
int remote_replication_type = g_random->randomInt(0, datacenters > 4 ? 4 : 5);
int remote_replication_type = deterministicRandom()->randomInt(0, datacenters > 4 ? 4 : 5);
switch (remote_replication_type) {
case 0: {
//FIXME: implement
@ -984,10 +984,10 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
ASSERT(false); // Programmer forgot to adjust cases.
}
if (g_random->random01() < 0.25) db.desiredLogRouterCount = g_random->randomInt(1,7);
if (g_random->random01() < 0.25) db.remoteDesiredTLogCount = g_random->randomInt(1,7);
if (deterministicRandom()->random01() < 0.25) db.desiredLogRouterCount = deterministicRandom()->randomInt(1,7);
if (deterministicRandom()->random01() < 0.25) db.remoteDesiredTLogCount = deterministicRandom()->randomInt(1,7);
bool useNormalDCsAsSatellites = datacenters > 4 && minimumRegions < 2 && g_random->random01() < 0.3;
bool useNormalDCsAsSatellites = datacenters > 4 && minimumRegions < 2 && deterministicRandom()->random01() < 0.3;
StatusObject primarySatelliteObj;
primarySatelliteObj["id"] = useNormalDCsAsSatellites ? "1" : "2";
primarySatelliteObj["priority"] = 1;
@ -1023,7 +1023,7 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
StatusArray regionArr;
regionArr.push_back(primaryObj);
if(needsRemote || g_random->random01() < 0.5) {
if(needsRemote || deterministicRandom()->random01() < 0.5) {
regionArr.push_back(remoteObj);
}
@ -1052,18 +1052,18 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
} else {
//datacenters+2 so that the configure database workload can configure into three_data_hall
machine_count = std::max(datacenters+2, ((db.minDatacentersRequired() > 0) ? datacenters : 1) * std::max(3, db.minMachinesRequiredPerDatacenter()));
machine_count = g_random->randomInt( machine_count, std::max(machine_count+1, extraDB ? 6 : 10) );
machine_count = deterministicRandom()->randomInt( machine_count, std::max(machine_count+1, extraDB ? 6 : 10) );
if (generateMachineTeamTestConfig) {
// When DESIRED_TEAMS_PER_SERVER is set to 1, the desired machine team number is 5
// while the max possible machine team number is 10.
// If machine_count > 5, we can still test the effectivenss of machine teams
// Note: machine_count may be much larger than 5 because we may have a big replication factor
machine_count = std::max(machine_count, g_random->randomInt(5, extraDB ? 6 : 10));
machine_count = std::max(machine_count, deterministicRandom()->randomInt(5, extraDB ? 6 : 10));
}
}
//because we protect a majority of coordinators from being killed, it is better to run with low numbers of coordinators to prevent too many processes from being protected
coordinators = ( minimumRegions <= 1 && BUGGIFY ) ? g_random->randomInt(1, std::max(machine_count,2)) : 1;
coordinators = ( minimumRegions <= 1 && BUGGIFY ) ? deterministicRandom()->randomInt(1, std::max(machine_count,2)) : 1;
if(minimumReplication > 1 && datacenters == 3) {
//low latency tests in 3 data hall mode need 2 other data centers with 2 machines each to avoid waiting for logs to recover.
@ -1074,7 +1074,7 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
if(generateFearless) {
processes_per_machine = 1;
} else {
processes_per_machine = g_random->randomInt(1, (extraDB ? 14 : 28)/machine_count + 2 );
processes_per_machine = deterministicRandom()->randomInt(1, (extraDB ? 14 : 28)/machine_count + 2 );
}
}
@ -1148,17 +1148,17 @@ void setupSimulatedSystem(vector<Future<Void>>* systemActors, std::string baseFo
const int processesPerMachine = simconfig.processes_per_machine;
// half the time, when we have more than 4 machines that are not the first in their dataCenter, assign classes
bool assignClasses = machineCount - dataCenters > 4 && g_random->random01() < 0.5;
bool assignClasses = machineCount - dataCenters > 4 && deterministicRandom()->random01() < 0.5;
// Use SSL 5% of the time
bool sslEnabled = g_random->random01() < 0.10 && tlsOptions->enabled();
bool sslOnly = sslEnabled && g_random->coinflip();
bool sslEnabled = deterministicRandom()->random01() < 0.10 && tlsOptions->enabled();
bool sslOnly = sslEnabled && deterministicRandom()->coinflip();
g_simulator.listenersPerProcess = sslEnabled && !sslOnly ? 2 : 1;
TEST( sslEnabled ); // SSL enabled
TEST( !sslEnabled ); // SSL disabled
// Use IPv6 25% of the time
bool useIPv6 = g_random->random01() < 0.25;
bool useIPv6 = deterministicRandom()->random01() < 0.25;
TEST( useIPv6 );
TEST( !useIPv6 );
@ -1204,7 +1204,7 @@ void setupSimulatedSystem(vector<Future<Void>>* systemActors, std::string baseFo
}
}
g_random->randomShuffle(coordinatorAddresses);
deterministicRandom()->randomShuffle(coordinatorAddresses);
for(int i = 0; i < (coordinatorAddresses.size()/2)+1; i++) {
TraceEvent("ProtectCoordinator")
.detail("Address", coordinatorAddresses[i])
@ -1215,7 +1215,7 @@ void setupSimulatedSystem(vector<Future<Void>>* systemActors, std::string baseFo
g_simulator.protectedAddresses.insert(NetworkAddress(coordinatorAddresses[i].ip, 1, true, false));
}
}
g_random->randomShuffle(coordinatorAddresses);
deterministicRandom()->randomShuffle(coordinatorAddresses);
ASSERT( coordinatorAddresses.size() == coordinatorCount );
ClusterConnectionString conn(coordinatorAddresses, LiteralStringRef("TestCluster:0"));
@ -1245,32 +1245,32 @@ void setupSimulatedSystem(vector<Future<Void>>* systemActors, std::string baseFo
int dcCoordinators = coordinatorCount / dataCenters + (dc < coordinatorCount%dataCenters);
printf("Datacenter %d: %d/%d machines, %d/%d coordinators\n", dc, machines, machineCount, dcCoordinators, coordinatorCount);
ASSERT( dcCoordinators <= machines );
int useSeedForMachine = g_random->randomInt(0, machines);
int useSeedForMachine = deterministicRandom()->randomInt(0, machines);
Standalone<StringRef> zoneId;
Standalone<StringRef> newZoneId;
for( int machine = 0; machine < machines; machine++ ) {
Standalone<StringRef> machineId(g_random->randomUniqueID().toString());
if(machine == 0 || machineCount - dataCenters <= 4 || assignedMachines != 4 || simconfig.db.regions.size() || g_random->random01() < 0.5) {
zoneId = g_random->randomUniqueID().toString();
newZoneId = g_random->randomUniqueID().toString();
Standalone<StringRef> machineId(deterministicRandom()->randomUniqueID().toString());
if(machine == 0 || machineCount - dataCenters <= 4 || assignedMachines != 4 || simconfig.db.regions.size() || deterministicRandom()->random01() < 0.5) {
zoneId = deterministicRandom()->randomUniqueID().toString();
newZoneId = deterministicRandom()->randomUniqueID().toString();
}
//Choose a machine class
ProcessClass processClass = ProcessClass(ProcessClass::UnsetClass, ProcessClass::CommandLineSource);
if(assignClasses) {
if(assignedMachines < 4)
processClass = ProcessClass((ProcessClass::ClassType) g_random->randomInt(0, 2), ProcessClass::CommandLineSource); //Unset or Storage
processClass = ProcessClass((ProcessClass::ClassType) deterministicRandom()->randomInt(0, 2), ProcessClass::CommandLineSource); //Unset or Storage
else if(assignedMachines == 4 && !simconfig.db.regions.size())
processClass = ProcessClass((ProcessClass::ClassType) (g_random->randomInt(0, 2) * ProcessClass::ResolutionClass), ProcessClass::CommandLineSource); //Unset or Resolution
processClass = ProcessClass((ProcessClass::ClassType) (deterministicRandom()->randomInt(0, 2) * ProcessClass::ResolutionClass), ProcessClass::CommandLineSource); //Unset or Resolution
else
processClass = ProcessClass((ProcessClass::ClassType) g_random->randomInt(0, 3), ProcessClass::CommandLineSource); //Unset, Storage, or Transaction
processClass = ProcessClass((ProcessClass::ClassType) deterministicRandom()->randomInt(0, 3), ProcessClass::CommandLineSource); //Unset, Storage, or Transaction
if (processClass == ProcessClass::ResolutionClass) // *can't* be assigned to other roles, even in an emergency
nonVersatileMachines++;
}
std::vector<IPAddress> ips;
for (int i = 0; i < processesPerMachine; i++) {
ips.push_back(makeIPAddressForSim(useIPv6, { 2, dc, g_random->randomInt(1, i + 2), machine }));
ips.push_back(makeIPAddressForSim(useIPv6, { 2, dc, deterministicRandom()->randomInt(1, i + 2), machine }));
}
// check the sslEnablementMap using only one ip(
LocalityData localities(Optional<Standalone<StringRef>>(), zoneId, machineId, dcUID);
@ -1281,10 +1281,10 @@ void setupSimulatedSystem(vector<Future<Void>>* systemActors, std::string baseFo
if (extraDB && g_simulator.extraDB->toString() != conn.toString()) {
std::vector<IPAddress> extraIps;
for (int i = 0; i < processesPerMachine; i++){
extraIps.push_back(makeIPAddressForSim(useIPv6, { 4, dc, g_random->randomInt(1, i + 2), machine }));
extraIps.push_back(makeIPAddressForSim(useIPv6, { 4, dc, deterministicRandom()->randomInt(1, i + 2), machine }));
}
Standalone<StringRef> newMachineId(g_random->randomUniqueID().toString());
Standalone<StringRef> newMachineId(deterministicRandom()->randomUniqueID().toString());
LocalityData localities(Optional<Standalone<StringRef>>(), newZoneId, newMachineId, dcUID);
localities.set(LiteralStringRef("data_hall"), dcUID);
@ -1308,12 +1308,12 @@ void setupSimulatedSystem(vector<Future<Void>>* systemActors, std::string baseFo
// SOMEDAY: add locality for testers to simulate network topology
// FIXME: Start workers with tester class instead, at least sometimes run tests with the testers-only flag
int testerCount = *pTesterCount = g_random->randomInt(4, 9);
int useSeedForMachine = g_random->randomInt(0, testerCount);
int testerCount = *pTesterCount = deterministicRandom()->randomInt(4, 9);
int useSeedForMachine = deterministicRandom()->randomInt(0, testerCount);
for(int i=0; i<testerCount; i++) {
std::vector<IPAddress> ips;
ips.push_back(makeIPAddressForSim(useIPv6, { 3, 4, 3, i + 1 }));
Standalone<StringRef> newZoneId = Standalone<StringRef>(g_random->randomUniqueID().toString());
Standalone<StringRef> newZoneId = Standalone<StringRef>(deterministicRandom()->randomUniqueID().toString());
LocalityData localities(Optional<Standalone<StringRef>>(), newZoneId, newZoneId, Optional<Standalone<StringRef>>());
systemActors->push_back( reportErrors( simulatedMachine(
conn, ips, sslEnabled, tlsOptions,
@ -1388,8 +1388,8 @@ ACTOR void setupAndRun(std::string dataFolder, const char *testFile, bool reboot
wait(g_simulator.onProcess(
g_simulator.newProcess("TestSystem", IPAddress(0x01010101), 1, 1,
LocalityData(Optional<Standalone<StringRef>>(),
Standalone<StringRef>(g_random->randomUniqueID().toString()),
Standalone<StringRef>(g_random->randomUniqueID().toString()),
Standalone<StringRef>(deterministicRandom()->randomUniqueID().toString()),
Standalone<StringRef>(deterministicRandom()->randomUniqueID().toString()),
Optional<Standalone<StringRef>>()),
ProcessClass(ProcessClass::TesterClass, ProcessClass::CommandLineSource), "", ""),
TaskDefaultYield));
@ -1412,7 +1412,7 @@ ACTOR void setupAndRun(std::string dataFolder, const char *testFile, bool reboot
minimumReplication, minimumRegions, tlsOptions);
wait( delay(1.0) ); // FIXME: WHY!!! //wait for machines to boot
}
std::string clusterFileDir = joinPath( dataFolder, g_random->randomUniqueID().toString() );
std::string clusterFileDir = joinPath( dataFolder, deterministicRandom()->randomUniqueID().toString() );
platform::createDirectory( clusterFileDir );
writeFile(joinPath(clusterFileDir, "fdb.cluster"), connFile.get().toString());
wait(timeoutError(runTests(Reference<ClusterConnectionFile>(new ClusterConnectionFile(joinPath(clusterFileDir, "fdb.cluster"))), TEST_TYPE_FROM_FILE, TEST_ON_TESTERS, testerCount, testFile, startingConfiguration), buggifyActivated ? 36000.0 : 5400.0));

View File

@ -285,10 +285,10 @@ private:
int randomLevel() {
/*int l = 0;
while (g_random->random01() < 0.5 && l < MaxLevels-1) l++;
while (deterministicRandom()->random01() < 0.5 && l < MaxLevels-1) l++;
return l; */
//g_random->randomInt(0, 1<<(MaxLevels-1));
//deterministicRandom()->randomInt(0, 1<<(MaxLevels-1));
uint32_t i = uint32_t(skfastrand()) >> (32-(MaxLevels-1));
int level = 0;
while (i&1) {
@ -649,7 +649,7 @@ public:
if (length == 1) { // forced down
f.finger[lev] = begin;
} else {
int c = g_random->randomInt(0, length);
int c = deterministicRandom()->randomInt(0, length);
for( Node* x = begin; x != end; x=x->getNext(lev) )
if (!c--) {
f.finger[lev] = begin = x;
@ -1396,13 +1396,13 @@ void miniConflictSetTest() {
int size = 64*5; // Also run 64*64*5 to test multiple words of andValues and orValues
MiniConflictSet mini(size);
for(int j=0; j<2; j++) {
int a = g_random->randomInt(0, size);
int b = g_random->randomInt(a, size);
int a = deterministicRandom()->randomInt(0, size);
int b = deterministicRandom()->randomInt(a, size);
mini.set( a, b );
}
for(int j=0; j<4; j++) {
int a = g_random->randomInt(0, size);
int b = g_random->randomInt(a, size);
int a = deterministicRandom()->randomInt(0, size);
int b = deterministicRandom()->randomInt(a, size);
mini.any( a, b ); // Tests correctness internally
}
}
@ -1438,14 +1438,14 @@ void skipListTest() {
success[i].assign( testData[i].size(), false );
success2[i].assign( testData[i].size(), false );
for(int j=0; j<testData[i].size(); j++) {
int key = g_random->randomInt(0, 20000000);
int key2 = key + 1 + g_random->randomInt(0, 10);
int key = deterministicRandom()->randomInt(0, 20000000);
int key2 = key + 1 + deterministicRandom()->randomInt(0, 10);
testData[i][j] = KeyRangeRef(
setK( testDataArena, key ),
setK( testDataArena, key2 ) );
}
}
printf("Test data generated (%d)\n", g_random->randomInt(0,100000));
printf("Test data generated (%d)\n", deterministicRandom()->randomInt(0,100000));
printf(" %d threads, %d batches, %d/batch\n", PARALLEL_THREAD_COUNT, testData.size(), testData[0].size());
printf("Running\n");

View File

@ -2277,27 +2277,27 @@ JsonBuilderArray randomArray(const std::vector<std::string> &strings, int &limit
JsonBuilderArray randomArray(const std::vector<std::string> &strings, int &limit, int level) {
JsonBuilderArray r;
int size = g_random->randomInt(0, 50);
int size = deterministicRandom()->randomInt(0, 50);
while(--size) {
if(--limit <= 0)
break;
if(level > 0 && g_random->coinflip()) {
if(g_random->coinflip())
if(level > 0 && deterministicRandom()->coinflip()) {
if(deterministicRandom()->coinflip())
r.push_back(randomDocument(strings, limit, level - 1));
else
r.push_back(randomArray(strings, limit, level - 1));
}
else {
switch(g_random->randomInt(0, 3)) {
switch(deterministicRandom()->randomInt(0, 3)) {
case 0:
r.push_back(g_random->randomInt(0, 10000000));
r.push_back(deterministicRandom()->randomInt(0, 10000000));
case 1:
r.push_back(strings[g_random->randomInt(0, strings.size())]);
r.push_back(strings[deterministicRandom()->randomInt(0, strings.size())]);
case 2:
default:
r.push_back(g_random->random01());
r.push_back(deterministicRandom()->random01());
}
}
}
@ -2307,29 +2307,29 @@ JsonBuilderArray randomArray(const std::vector<std::string> &strings, int &limit
JsonBuilderObject randomDocument(const std::vector<std::string> &strings, int &limit, int level) {
JsonBuilderObject r;
int size = g_random->randomInt(0, 300);
int size = deterministicRandom()->randomInt(0, 300);
while(--size) {
if(--limit <= 0)
break;
const std::string &key = strings[g_random->randomInt(0, strings.size())];
const std::string &key = strings[deterministicRandom()->randomInt(0, strings.size())];
if(level > 0 && g_random->coinflip()) {
if(g_random->coinflip())
if(level > 0 && deterministicRandom()->coinflip()) {
if(deterministicRandom()->coinflip())
r[key] = randomDocument(strings, limit, level - 1);
else
r[key] = randomArray(strings, limit, level - 1);
}
else {
switch(g_random->randomInt(0, 3)) {
switch(deterministicRandom()->randomInt(0, 3)) {
case 0:
r[key] = g_random->randomInt(0, 10000000);
r[key] = deterministicRandom()->randomInt(0, 10000000);
case 1:
r[key] = strings[g_random->randomInt(0, strings.size())];
r[key] = strings[deterministicRandom()->randomInt(0, strings.size())];
case 2:
default:
r[key] = g_random->random01();
r[key] = deterministicRandom()->random01();
}
}
}
@ -2342,7 +2342,7 @@ TEST_CASE("/status/json/builderPerf") {
int c = 1000000;
printf("Generating random strings\n");
while(--c)
strings.push_back(g_random->randomAlphaNumeric(g_random->randomInt(0, 50)));
strings.push_back(deterministicRandom()->randomAlphaNumeric(deterministicRandom()->randomInt(0, 50)));
int elements = 100000;
int level = 6;

View File

@ -101,7 +101,7 @@ struct TransientStorageMetricSample : StorageMetricSample {
TransientStorageMetricSample( int64_t metricUnitsPerSample ) : StorageMetricSample(metricUnitsPerSample) {}
bool roll( KeyRef key, int64_t metric ) {
return g_random->random01() < (double)metric / metricUnitsPerSample; //< SOMEDAY: Better randomInt64?
return deterministicRandom()->random01() < (double)metric / metricUnitsPerSample; //< SOMEDAY: Better randomInt64?
}
// Returns the sampled metric value (possibly 0, possibly increased by the sampling factor)
@ -292,7 +292,7 @@ struct StorageServerMetrics {
double offset = (expectedSize - used) / divisor;
if( offset <= 0 )
return hasUsed ? lastKey : key;
return sample.splitEstimate( KeyRangeRef(lastKey, key), offset * ( ( 1.0 - SERVER_KNOBS->SPLIT_JITTER_AMOUNT ) + 2 * g_random->random01() * SERVER_KNOBS->SPLIT_JITTER_AMOUNT ) );
return sample.splitEstimate( KeyRangeRef(lastKey, key), offset * ( ( 1.0 - SERVER_KNOBS->SPLIT_JITTER_AMOUNT ) + 2 * deterministicRandom()->random01() * SERVER_KNOBS->SPLIT_JITTER_AMOUNT ) );
}
}
@ -348,7 +348,7 @@ struct StorageServerMetrics {
// SOMEDAY: make bytes dynamic with hard disk space
rep.load = getMetrics(allKeys);
if (sb.free < 1e9 && g_random->random01() < 0.1)
if (sb.free < 1e9 && deterministicRandom()->random01() < 0.1)
TraceEvent(SevWarn, "PhysicalDiskMetrics")
.detail("Free", sb.free)
.detail("Total", sb.total)

View File

@ -46,8 +46,8 @@ struct TLogInterface {
RequestStream< struct TLogRecoveryFinishedRequest > recoveryFinished;
TLogInterface() {}
explicit TLogInterface(LocalityData locality) : uniqueID( g_random->randomUniqueID() ), locality(locality) { sharedTLogID = uniqueID; }
TLogInterface(UID sharedTLogID, LocalityData locality) : uniqueID( g_random->randomUniqueID() ), sharedTLogID(sharedTLogID), locality(locality) {}
explicit TLogInterface(LocalityData locality) : uniqueID( deterministicRandom()->randomUniqueID() ), locality(locality) { sharedTLogID = uniqueID; }
TLogInterface(UID sharedTLogID, LocalityData locality) : uniqueID( deterministicRandom()->randomUniqueID() ), sharedTLogID(sharedTLogID), locality(locality) {}
TLogInterface(UID uniqueID, UID sharedTLogID, LocalityData locality) : uniqueID(uniqueID), sharedTLogID(sharedTLogID), locality(locality) {}
UID id() const { return uniqueID; }
UID getSharedTLogID() const { return sharedTLogID; }

View File

@ -328,7 +328,7 @@ struct TLogData : NonCopyable {
Reference<AsyncVar<bool>> degraded;
TLogData(UID dbgid, IKeyValueStore* persistentData, IDiskQueue * persistentQueue, Reference<AsyncVar<ServerDBInfo>> dbInfo, Reference<AsyncVar<bool>> degraded)
: dbgid(dbgid), instanceID(g_random->randomUniqueID().first()),
: dbgid(dbgid), instanceID(deterministicRandom()->randomUniqueID().first()),
persistentData(persistentData), rawPersistentQueue(persistentQueue), persistentQueue(new TLogQueue(persistentQueue, dbgid)),
dbInfo(dbInfo), degraded(degraded), queueCommitBegin(0), queueCommitEnd(0),
diskQueueCommitBytes(0), largeDiskQueueCommitBytes(false), bytesInput(0), bytesDurable(0), overheadBytesInput(0), overheadBytesDurable(0),
@ -1591,7 +1591,7 @@ ACTOR Future<Void> tLogCommit(
state Optional<UID> tlogDebugID;
if(req.debugID.present())
{
tlogDebugID = g_nondeterministic_random->randomUniqueID();
tlogDebugID = nondeterministicRandom()->randomUniqueID();
g_traceBatch.addAttach("CommitAttachID", req.debugID.get().first(), tlogDebugID.get().first());
g_traceBatch.addEvent("CommitDebug", tlogDebugID.get().first(), "TLog.tLogCommit.BeforeWaitForVersion");
}
@ -1725,7 +1725,7 @@ ACTOR Future<Void> rejoinMasters( TLogData* self, TLogInterface tli, DBRecoveryC
{
TraceEvent("TLogDisplaced", tli.id()).detail("Reason", "DBInfoDoesNotContain").detail("RecoveryCount", recoveryCount).detail("InfRecoveryCount", inf.recoveryCount).detail("RecoveryState", (int)inf.recoveryState)
.detail("LogSysConf", describe(inf.logSystemConfig.tLogs)).detail("PriorLogs", describe(inf.priorCommittedLogServers)).detail("OldLogGens", inf.logSystemConfig.oldTLogs.size());
if (BUGGIFY) wait( delay( SERVER_KNOBS->BUGGIFY_WORKER_REMOVED_MAX_LAG * g_random->random01() ) );
if (BUGGIFY) wait( delay( SERVER_KNOBS->BUGGIFY_WORKER_REMOVED_MAX_LAG * deterministicRandom()->random01() ) );
throw worker_removed();
}
@ -1858,7 +1858,7 @@ ACTOR Future<Void> serveTLogInterface( TLogData* self, TLogInterface tli, Refere
}
when (TLogConfirmRunningRequest req = waitNext(tli.confirmRunning.getFuture())){
if (req.debugID.present() ) {
UID tlogDebugID = g_nondeterministic_random->randomUniqueID();
UID tlogDebugID = nondeterministicRandom()->randomUniqueID();
g_traceBatch.addAttach("TransactionAttachID", req.debugID.get().first(), tlogDebugID.first());
g_traceBatch.addEvent("TransactionDebug", tlogDebugID.first(), "TLogServer.TLogConfirmRunningRequest");
}
@ -2592,12 +2592,12 @@ TEST_CASE("/fdbserver/tlogserver/VersionMessagesOverheadFactor" ) {
DequeAllocator<TestType> allocator;
std::deque<TestType, DequeAllocator<TestType>> d(allocator);
int numElements = g_random->randomInt(pow(10, i-1), pow(10, i));
int numElements = deterministicRandom()->randomInt(pow(10, i-1), pow(10, i));
for(int k = 0; k < numElements; ++k) {
d.push_back(TestType());
}
int removedElements = 0;//g_random->randomInt(0, numElements); // FIXME: the overhead factor does not accurately account for removal!
int removedElements = 0;//deterministicRandom()->randomInt(0, numElements); // FIXME: the overhead factor does not accurately account for removal!
for(int k = 0; k < removedElements; ++k) {
d.pop_front();
}

View File

@ -1123,7 +1123,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
virtual Tag getRandomRouterTag() {
return Tag(tagLocalityLogRouter, g_random->randomInt(0, logRouterTags));
return Tag(tagLocalityLogRouter, deterministicRandom()->randomInt(0, logRouterTags));
}
ACTOR static Future<Void> monitorLog(Reference<AsyncVar<OptionalInterface<TLogInterface>>> logServer, Reference<AsyncVar<bool>> failed) {
@ -1780,7 +1780,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
logSystem->expectedLogSets = 1;
logSystem->recoveredAt = oldLogSystem->recoverAt;
logSystem->repopulateRegionAntiQuorum = configuration.repopulateRegionAntiQuorum;
logSystem->recruitmentID = g_random->randomUniqueID();
logSystem->recruitmentID = deterministicRandom()->randomUniqueID();
oldLogSystem->recruitmentID = logSystem->recruitmentID;
if(configuration.usableRegions > 1) {
@ -2129,7 +2129,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
for (int i = 0; i < prevState.tLogLocalities.size(); i++) {
tlogs.emplace_back(prevState.tLogLocalities[i], i);
}
g_random->randomShuffle(tlogs);
deterministicRandom()->randomShuffle(tlogs);
// Rearrange the array such that things that the left is logs closer to being locked, and
// the right is logs that can't be locked. This makes us prefer locking already-locked TLogs,
// which is how we respect the decisions made in the previous execution.
@ -2186,7 +2186,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
// with independant sets of data. This case will validated that no code is relying on the old
// quorum=(RF-AQ) logic, and now goes through the policy engine instead.
TEST(can_omit >= prevState.tLogReplicationFactor - prevState.tLogWriteAntiQuorum); // Locking a subset of the TLogs while ending an epoch.
const bool reboot_a_tlog = g_network->now() - g_simulator.lastConnectionFailure > g_simulator.connectionFailuresDisableDuration && BUGGIFY && g_random->random01() < 0.25;
const bool reboot_a_tlog = g_network->now() - g_simulator.lastConnectionFailure > g_simulator.connectionFailuresDisableDuration && BUGGIFY && deterministicRandom()->random01() < 0.25;
TraceEvent(SevInfo, "MasterRecoveryTLogLocking", dbgid)
detail("Locks", tlogs.size() - can_omit)
detail("Skipped", can_omit)

View File

@ -696,7 +696,7 @@ static void asyncDlClose(sqlite3_vfs *pVfs, void *pHandle){
*/
static int asyncRandomness(sqlite3_vfs *pVfs, int nByte, char *zByte){
for(int i=0; i<nByte; i++)
zByte[i] = g_random->randomInt(0,256);
zByte[i] = deterministicRandom()->randomInt(0,256);
return SQLITE_OK;
}

View File

@ -1958,7 +1958,7 @@ IKeyValueStore* keyValueStoreRedwoodV1( std::string const& filename, UID logID)
}
int randomSize(int max) {
return g_random->randomInt(0, max);
return deterministicRandom()->randomInt(0, max);
}
KeyValue randomKV(int keySize = 10, int valueSize = 5) {
@ -1968,9 +1968,9 @@ KeyValue randomKV(int keySize = 10, int valueSize = 5) {
kv.key = makeString(kLen, kv.arena());
kv.value = makeString(vLen, kv.arena());
for(int i = 0; i < kLen; ++i)
mutateString(kv.key)[i] = (uint8_t)g_random->randomInt('a', 'm');
mutateString(kv.key)[i] = (uint8_t)deterministicRandom()->randomInt('a', 'm');
for(int i = 0; i < vLen; ++i)
mutateString(kv.value)[i] = (uint8_t)g_random->randomInt('n', 'z');
mutateString(kv.value)[i] = (uint8_t)deterministicRandom()->randomInt('n', 'z');
return kv;
}
@ -1989,10 +1989,10 @@ ACTOR Future<int> verifyRandomRange(VersionedBTree *btree, Version v, std::map<s
state Reference<IStoreCursor> cur = btree->readAtVersion(v);
// Randomly use the cursor for something else first.
if(g_random->coinflip()) {
if(deterministicRandom()->coinflip()) {
debug_printf("VerifyRange: Dummy seek\n");
state Key randomKey = randomKV().key;
wait(g_random->coinflip() ? cur->findFirstEqualOrGreater(randomKey, true, 0) : cur->findLastLessOrEqual(randomKey, true, 0));
wait(deterministicRandom()->coinflip() ? cur->findFirstEqualOrGreater(randomKey, true, 0) : cur->findLastLessOrEqual(randomKey, true, 0));
}
debug_printf("VerifyRange: Actual seek\n");
@ -2063,7 +2063,7 @@ ACTOR Future<int> verifyRandomRange(VersionedBTree *btree, Version v, std::map<s
debug_printf("VerifyRangeReverse '%s' to '%s' @%" PRId64 "\n", printable(start).c_str(), printable(end).c_str(), v);
// Randomly use a new cursor for the revere range read
if(g_random->coinflip()) {
if(deterministicRandom()->coinflip()) {
cur = btree->readAtVersion(v);
}
@ -2150,7 +2150,7 @@ ACTOR Future<Void> verify(VersionedBTree *btree, FutureStream<Version> vStream,
debug_printf("Verifying through version %lld\n", v);
state Future<int> vall = verifyAll(btree, v, written);
state Future<int> vrange = verifyRandomRange(btree, g_random->randomInt(1, v + 1), written);
state Future<int> vrange = verifyRandomRange(btree, deterministicRandom()->randomInt(1, v + 1), written);
wait(success(vall) && success(vrange));
int errors = vall.get() + vrange.get();
@ -2174,13 +2174,13 @@ ACTOR Future<Void> randomReader(VersionedBTree *btree) {
state Reference<IStoreCursor> cur;
loop {
wait(yield());
if(!cur || g_random->random01() > .1) {
Version v = g_random->randomInt(1, btree->getLastCommittedVersion() + 1);
if(!cur || deterministicRandom()->random01() > .1) {
Version v = deterministicRandom()->randomInt(1, btree->getLastCommittedVersion() + 1);
cur = btree->readAtVersion(v);
}
wait(cur->findFirstEqualOrGreater(randomKV(10, 0).key, true, 0));
state int c = g_random->randomInt(0, 100);
state int c = deterministicRandom()->randomInt(0, 100);
while(cur->isValid() && c-- > 0) {
wait(success(cur->next(true)));
wait(yield());
@ -2203,11 +2203,11 @@ TEST_CASE("!/redwood/correctness") {
else
pager = createMemoryPager();
state int pageSize = g_random->coinflip() ? pager->getUsablePageSize() : g_random->randomInt(200, 400);
state int pageSize = deterministicRandom()->coinflip() ? pager->getUsablePageSize() : deterministicRandom()->randomInt(200, 400);
state VersionedBTree *btree = new VersionedBTree(pager, pagerFile, pageSize);
wait(btree->init());
state int mutationBytesTarget = g_random->randomInt(100, 20e6);
state int mutationBytesTarget = deterministicRandom()->randomInt(100, 20e6);
// We must be able to fit at least two any two keys plus overhead in a page to prevent
// a situation where the tree cannot be grown upward with decreasing level size.
@ -2241,23 +2241,23 @@ TEST_CASE("!/redwood/correctness") {
while(mutationBytes < mutationBytesTarget) {
// Sometimes advance the version
if(g_random->random01() < 0.10) {
if(deterministicRandom()->random01() < 0.10) {
++version;
btree->setWriteVersion(version);
}
// Sometimes do a clear range
if(g_random->random01() < .10) {
if(deterministicRandom()->random01() < .10) {
Key start = randomKV(maxKeySize, 1).key;
Key end = (g_random->random01() < .01) ? keyAfter(start) : randomKV(maxKeySize, 1).key;
Key end = (deterministicRandom()->random01() < .01) ? keyAfter(start) : randomKV(maxKeySize, 1).key;
// Sometimes replace start and/or end with a close actual (previously used) value
if(g_random->random01() < .10) {
if(deterministicRandom()->random01() < .10) {
auto i = keys.upper_bound(start);
if(i != keys.end())
start = *i;
}
if(g_random->random01() < .10) {
if(deterministicRandom()->random01() < .10) {
auto i = keys.upper_bound(end);
if(i != keys.end())
end = *i;
@ -2300,7 +2300,7 @@ TEST_CASE("!/redwood/correctness") {
// Set a key
KeyValue kv = randomKV(maxKeySize, maxValueSize);
// Sometimes change key to a close previously used key
if(g_random->random01() < .01) {
if(deterministicRandom()->random01() < .01) {
auto i = keys.upper_bound(kv.key);
if(i != keys.end())
kv.key = StringRef(kv.arena(), *i);
@ -2315,7 +2315,7 @@ TEST_CASE("!/redwood/correctness") {
}
// Sometimes (and at end) commit then check all results
if(mutationBytes >= std::min(mutationBytesTarget, (int)20e6) || g_random->random01() < .002) {
if(mutationBytes >= std::min(mutationBytesTarget, (int)20e6) || deterministicRandom()->random01() < .002) {
// Wait for btree commit and send the new version to committedVersions.
// Avoid capture of version as a member of *this
Version v = version;
@ -2328,7 +2328,7 @@ TEST_CASE("!/redwood/correctness") {
printf("Cumulative: %d total mutation bytes, %lu key changes, %" PRId64 " key bytes, %" PRId64 " value bytes\n", mutationBytes, written.size(), keyBytesInserted, ValueBytesInserted);
// Recover from disk at random
if(useDisk && g_random->random01() < .1) {
if(useDisk && deterministicRandom()->random01() < .1) {
printf("Recovering from disk.\n");
// Wait for outstanding commit
@ -2407,19 +2407,19 @@ TEST_CASE("!/redwood/performance/set") {
Version lastVer = wait(btree->getLatestVersion());
state Version version = lastVer + 1;
btree->setWriteVersion(version);
int changes = g_random->randomInt(0, maxChangesPerVersion);
int changes = deterministicRandom()->randomInt(0, maxChangesPerVersion);
while(changes--) {
KeyValue kv;
// Change first 4 bytes of key to an int
*(uint32_t *)key.data() = g_random->randomInt(0, nodeCount);
kv.key = StringRef((uint8_t *)key.data(), g_random->randomInt(10, key.size()));
kv.value = StringRef((uint8_t *)value.data(), g_random->randomInt(0, value.size()));
*(uint32_t *)key.data() = deterministicRandom()->randomInt(0, nodeCount);
kv.key = StringRef((uint8_t *)key.data(), deterministicRandom()->randomInt(10, key.size()));
kv.value = StringRef((uint8_t *)value.data(), deterministicRandom()->randomInt(0, value.size()));
btree->set(kv);
kvBytes += kv.key.size() + kv.value.size();
++records;
}
if(g_random->random01() < (1.0 / 300)) {
if(deterministicRandom()->random01() < (1.0 / 300)) {
wait(commit);
commit = btree->commit();
double elapsed = now() - startTime;

View File

@ -175,7 +175,6 @@ extern IPAddress determinePublicIPAutomatically(ClusterConnectionString const& c
extern const char* getHGVersion();
extern IRandom* trace_random;
extern void flushTraceFileVoid();
extern bool noUnseed;
@ -327,7 +326,7 @@ UID getSharedMemoryMachineId() {
try {
// "0" is the default parameter "addr"
boost::interprocess::managed_shared_memory segment(boost::interprocess::open_or_create, sharedMemoryIdentifier.c_str(), 1000, 0, p.permission);
machineId = segment.find_or_construct<UID>("machineId")(g_random->randomUniqueID());
machineId = segment.find_or_construct<UID>("machineId")(deterministicRandom()->randomUniqueID());
if (!machineId)
criticalError(FDB_EXIT_ERROR, "SharedMemoryError", "Could not locate or create shared memory - 'machineId'");
return *machineId;
@ -1372,19 +1371,7 @@ int main(int argc, char* argv[]) {
if( zoneId.present() )
printf("ZoneId set to %s, dcId to %s\n", printable(zoneId).c_str(), printable(dcId).c_str());
g_random = new DeterministicRandom(randomSeed);
if (role == Simulation)
trace_random = new DeterministicRandom(1);
else
trace_random = new DeterministicRandom(platform::getRandomSeed());
if (role == Simulation)
g_nondeterministic_random = new DeterministicRandom(2);
else
g_nondeterministic_random = new DeterministicRandom(platform::getRandomSeed());
if (role == Simulation)
g_debug_random = new DeterministicRandom(3);
else
g_debug_random = new DeterministicRandom(platform::getRandomSeed());
setThreadLocalDeterministicRandomSeed(randomSeed);
if(role==Simulation) {
Optional<bool> buggifyOverride = checkBuggifyOverride(testFile);
@ -1694,7 +1681,7 @@ int main(int argc, char* argv[]) {
rc = FDB_EXIT_ERROR;
}
int unseed = noUnseed ? 0 : g_random->randomInt(0, 100001);
int unseed = noUnseed ? 0 : deterministicRandom()->randomInt(0, 100001);
TraceEvent("ElapsedTime").detail("SimTime", now()-startNow).detail("RealTime", timer()-start)
.detail("RandomUnseed", unseed);

View File

@ -353,8 +353,8 @@ ACTOR Future<Void> newSeedServers( Reference<MasterData> self, RecruitFromConfig
InitializeStorageRequest isr;
isr.seedTag = dcId_tags.count(recruits.storageServers[idx].locality.dcId()) ? dcId_tags[recruits.storageServers[idx].locality.dcId()] : Tag(nextLocality, 0);
isr.storeType = self->configuration.storageServerStoreType;
isr.reqId = g_random->randomUniqueID();
isr.interfaceId = g_random->randomUniqueID();
isr.reqId = deterministicRandom()->randomUniqueID();
isr.interfaceId = deterministicRandom()->randomUniqueID();
ErrorOr<InitializeStorageReply> newServer = wait( recruits.storageServers[idx].storage.tryGetReply( isr ) );
@ -654,7 +654,7 @@ ACTOR Future<Void> readTransactionSystemState( Reference<MasterData> self, Refer
}
if(BUGGIFY) {
self->recoveryTransactionVersion += g_random->randomInt64(0, SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT);
self->recoveryTransactionVersion += deterministicRandom()->randomInt64(0, SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT);
}
if ( self->recoveryTransactionVersion < minRequiredCommitVersion ) self->recoveryTransactionVersion = minRequiredCommitVersion;
}
@ -1311,7 +1311,7 @@ ACTOR Future<Void> masterCore( Reference<MasterData> self ) {
if(self->forceRecovery) {
tr.set(recoveryCommitRequest.arena, rebootWhenDurableKey, StringRef());
tr.set(recoveryCommitRequest.arena, moveKeysLockOwnerKey, BinaryWriter::toValue(g_random->randomUniqueID(),Unversioned()));
tr.set(recoveryCommitRequest.arena, moveKeysLockOwnerKey, BinaryWriter::toValue(deterministicRandom()->randomUniqueID(),Unversioned()));
}
} else {
// Recruit and seed initial shard servers

View File

@ -58,7 +58,7 @@ ACTOR Future<Void> networkTestServer() {
ACTOR Future<Void> testClient( std::vector<NetworkTestInterface> interfs, int* sent ) {
loop {
NetworkTestReply rep = wait( retryBrokenPromise(interfs[g_random->randomInt(0, interfs.size())].test, NetworkTestRequest( LiteralStringRef("."), 600000 ) ) );
NetworkTestReply rep = wait( retryBrokenPromise(interfs[deterministicRandom()->randomInt(0, interfs.size())].test, NetworkTestRequest( LiteralStringRef("."), 600000 ) ) );
(*sent)++;
}
}

View File

@ -105,14 +105,14 @@ PubSub::PubSub(Database _cx)
}
ACTOR Future<uint64_t> _createFeed(Database cx, Standalone<StringRef> metadata) {
state uint64_t id(g_random->randomUniqueID().first()); // SOMEDAY: this should be an atomic increment
state uint64_t id(deterministicRandom()->randomUniqueID().first()); // SOMEDAY: this should be an atomic increment
TraceEvent("PubSubCreateFeed").detail("Feed", id);
state Transaction tr(cx);
loop {
try {
state Optional<Value> val = wait(tr.get(keyForFeed(id)));
while(val.present()) {
id = id + g_random->randomInt(1, 100);
id = id + deterministicRandom()->randomInt(1, 100);
Optional<Value> v = wait(tr.get(keyForFeed(id)));
val = v;
}
@ -133,14 +133,14 @@ Future<uint64_t> PubSub::createFeed(Standalone<StringRef> metadata) {
}
ACTOR Future<uint64_t> _createInbox(Database cx, Standalone<StringRef> metadata) {
state uint64_t id = g_random->randomUniqueID().first();
state uint64_t id = deterministicRandom()->randomUniqueID().first();
TraceEvent("PubSubCreateInbox").detail("Inbox", id);
state Transaction tr(cx);
loop {
try {
state Optional<Value> val = wait(tr.get(keyForInbox(id)));
while(val.present()) {
id += g_random->randomInt(1, 100);
id += deterministicRandom()->randomInt(1, 100);
Optional<Value> v = wait(tr.get(keyForFeed(id)));
val = v;
}

View File

@ -501,7 +501,7 @@ public:
} counters;
StorageServer(IKeyValueStore* storage, Reference<AsyncVar<ServerDBInfo>> const& db, StorageServerInterface const& ssi)
: instanceID(g_random->randomUniqueID().first()),
: instanceID(deterministicRandom()->randomUniqueID().first()),
storage(this, storage), db(db),
lastTLogVersion(0), lastVersionWithData(0), restoredVersion(0),
rebootAfterDurableVersion(std::numeric_limits<Version>::max()),
@ -719,7 +719,7 @@ ACTOR Future<Version> waitForVersion( StorageServer* data, Version version ) {
throw process_behind();
}
if(g_random->random01() < 0.001)
if(deterministicRandom()->random01() < 0.001)
TraceEvent("WaitForVersion1000x");
choose {
when ( wait( data->version.whenAtLeast(version) ) ) {
@ -729,7 +729,7 @@ ACTOR Future<Version> waitForVersion( StorageServer* data, Version version ) {
return version;
}
when ( wait( delay( SERVER_KNOBS->FUTURE_VERSION_DELAY ) ) ) {
if(g_random->random01() < 0.001)
if(deterministicRandom()->random01() < 0.001)
TraceEvent(SevWarn, "ShardServerFutureVersion1000x", data->thisServerID)
.detail("Version", version)
.detail("MyVersion", data->version.get())
@ -750,7 +750,7 @@ ACTOR Future<Version> waitForVersionNoTooOld( StorageServer* data, Version versi
return version;
}
when ( wait( delay( SERVER_KNOBS->FUTURE_VERSION_DELAY ) ) ) {
if(g_random->random01() < 0.001)
if(deterministicRandom()->random01() < 0.001)
TraceEvent(SevWarn, "ShardServerFutureVersion1000x", data->thisServerID)
.detail("Version", version)
.detail("MyVersion", data->version.get())
@ -1016,7 +1016,7 @@ ACTOR Future<GetKeyValuesReply> readRange( StorageServer* data, Version version,
state KeyRef readEnd;
state Key readBeginTemp;
state int vCount;
//state UID rrid = g_random->randomUniqueID();
//state UID rrid = deterministicRandom()->randomUniqueID();
//state int originalLimit = limit;
//state int originalLimitBytes = *pLimitBytes;
//state bool track = rrid.first() == 0x1bc134c2f752187cLL;
@ -2962,7 +2962,7 @@ ACTOR Future<Void> restoreByteSample(StorageServer* data, IKeyValueStore* storag
state std::vector<Standalone<VectorRef<KeyValueRef>>> byteSampleSample;
wait( applyByteSampleResult(data, storage, persistByteSampleSampleKeys.begin, persistByteSampleSampleKeys.end, &byteSampleSample) );
byteSampleSampleRecovered.send(Void());
wait( delay( BUGGIFY ? g_random->random01() * 2.0 : 0.0001 ) );
wait( delay( BUGGIFY ? deterministicRandom()->random01() * 2.0 : 0.0001 ) );
size_t bytes_per_fetch = 0;
// Since the expected size also includes (as of now) the space overhead of the container, we calculate our own number here
@ -2994,7 +2994,7 @@ ACTOR Future<Void> restoreByteSample(StorageServer* data, IKeyValueStore* storag
TraceEvent("RecoveredByteSampleChunkedRead", data->thisServerID).detail("Ranges",sampleRanges.size());
if( BUGGIFY )
wait( delay( g_random->random01() * 10.0 ) );
wait( delay( deterministicRandom()->random01() * 10.0 ) );
return Void();
}
@ -3358,7 +3358,7 @@ ACTOR Future<Void> storageServerCore( StorageServer* self, StorageServerInterfac
double loopTopTime = now();
double elapsedTime = loopTopTime - lastLoopTopTime;
if( elapsedTime > 0.050 ) {
if (g_random->random01() < 0.01)
if (deterministicRandom()->random01() < 0.01)
TraceEvent(SevWarn, "SlowSSLoopx100", self->thisServerID).detail("Elapsed", elapsedTime);
}
lastLoopTopTime = loopTopTime;
@ -3701,7 +3701,7 @@ void versionedMapTest() {
for(int v=1; v<=1000; ++v) {
vm.createNewVersion(v);
for(int i=0; i<1000; i++) {
int k = g_random->randomInt(0, 2000000);
int k = deterministicRandom()->randomInt(0, 2000000);
/*for(int k2=k-5; k2<k+5; k2++)
if (vm.atLatest().find(k2) != vm.atLatest().end())
vm.erase(k2);*/

View File

@ -80,19 +80,19 @@ Key KVWorkload::getRandomKey() {
Key KVWorkload::getRandomKey(double absentFrac) {
if ( absentFrac > 0.0000001 ) {
return getRandomKey(g_random->random01() < absentFrac);
return getRandomKey(deterministicRandom()->random01() < absentFrac);
} else {
return getRandomKey(false);
}
}
Key KVWorkload::getRandomKey(bool absent) {
return keyForIndex(g_random->randomInt( 0, nodeCount ), absent);
return keyForIndex(deterministicRandom()->randomInt( 0, nodeCount ), absent);
}
Key KVWorkload::keyForIndex( uint64_t index ) {
if ( absentFrac > 0.0000001 ) {
return keyForIndex(index, g_random->random01() < absentFrac);
return keyForIndex(index, deterministicRandom()->random01() < absentFrac);
} else {
return keyForIndex(index, false);
}
@ -122,7 +122,7 @@ double testKeyToDouble(const KeyRef& p, const KeyRef& prefix) {
}
ACTOR Future<Void> poisson( double *last, double meanInterval ) {
*last += meanInterval*-log( g_random->random01() );
*last += meanInterval*-log( deterministicRandom()->random01() );
wait( delayUntil( *last ) );
return Void();
}
@ -360,7 +360,7 @@ ACTOR Future<Void> pingDatabase( Database cx ) {
loop {
try {
tr.setOption( FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE );
Optional<Value> v = wait( tr.get( StringRef("/Liveness/" + g_random->randomUniqueID().toString() ) ) );
Optional<Value> v = wait( tr.get( StringRef("/Liveness/" + deterministicRandom()->randomUniqueID().toString() ) ) );
tr.makeSelfConflicting();
wait( tr.commit() );
return Void();
@ -646,7 +646,7 @@ ACTOR Future<DistributedTestResults> runWorkload( Database cx, std::vector< Test
state int i = 0;
state int success = 0;
state int failure = 0;
int64_t sharedRandom = g_random->randomInt64(0,10000000);
int64_t sharedRandom = deterministicRandom()->randomInt64(0,10000000);
for(; i < testers.size(); i++) {
WorkloadRequest req;
req.title = spec.title;

View File

@ -973,7 +973,7 @@ ACTOR Future<Void> workerServer(
auto& logData = sharedLogs[std::make_tuple(req.logVersion, req.storeType, req.spillType)];
logData.second.send(req);
if(!logData.first.isValid() || logData.first.isReady()) {
UID logId = g_random->randomUniqueID();
UID logId = deterministicRandom()->randomUniqueID();
std::map<std::string, std::string> details;
details["ForMaster"] = req.recruitmentID.shortString();
details["StorageEngine"] = req.storeType.toString();
@ -1282,7 +1282,7 @@ ACTOR Future<UID> createAndLockProcessIdFile(std::string folder) {
if (lockFile.isError() && lockFile.getError().code() == error_code_file_not_found && !fileExists(lockFilePath)) {
Reference<IAsyncFile> _lockFile = wait(IAsyncFileSystem::filesystem()->open(lockFilePath, IAsyncFile::OPEN_ATOMIC_WRITE_AND_CREATE | IAsyncFile::OPEN_CREATE | IAsyncFile::OPEN_LOCK | IAsyncFile::OPEN_READWRITE, 0600));
lockFile = _lockFile;
processIDUid = g_random->randomUniqueID();
processIDUid = deterministicRandom()->randomUniqueID();
BinaryWriter wr(IncludeVersion());
wr << processIDUid;
wait(lockFile.get()->write(wr.getData(), wr.getLength(), 0));

View File

@ -236,7 +236,7 @@ public:
totalDensity += pdf[i];
int cumulativeDensity = 0;
int random = g_random->randomInt(0, totalDensity);
int random = deterministicRandom()->randomInt(0, totalDensity);
for(int i = 0; i < pdf.size() - 1; i++) {
if(cumulativeDensity + pdf[i] <= random && random < cumulativeDensity + pdf[i] + pdf[i + 1]) {
operation = (OperationType)i;
@ -250,7 +250,7 @@ public:
//Test the set operation
if(operation == SET) {
bool useShortKeys = g_random->randomInt(0, 2) == 1;
bool useShortKeys = deterministicRandom()->randomInt(0, 2) == 1;
int minKeyLength = useShortKeys ? self->minShortKeyLength : self->minLongKeyLength;
int maxKeyLength = useShortKeys ? self->maxShortKeyLength : self->maxLongKeyLength;
@ -415,8 +415,8 @@ public:
state Key end = std::max(key, key2);
//Generate a random maximum number of results
state int limit = g_random->randomInt(0, 101);
state bool reverse = g_random->random01() > 0.5 ? false : true;
state int limit = deterministicRandom()->randomInt(0, 101);
state bool reverse = deterministicRandom()->random01() > 0.5 ? false : true;
//Get the range from memory
state Standalone<RangeResultRef> storeResults = self->store.getRange(KeyRangeRef(start, end), limit, reverse);
@ -501,8 +501,8 @@ public:
}
//Choose a random maximum number of results
state int limit = g_random->randomInt(0, 101);
state bool reverse = g_random->random01() < 0.5 ? false : true;
state int limit = deterministicRandom()->randomInt(0, 101);
state bool reverse = deterministicRandom()->random01() < 0.5 ? false : true;
//Get the range from the memory store
state Standalone<RangeResultRef> storeResults = self->store.getRange(KeyRangeRef(startKey, endKey), limit, reverse);

View File

@ -211,22 +211,22 @@ Standalone<VectorRef<KeyValueRef>> ApiWorkload::generateData(int numKeys, int mi
//Generates a random key
Key ApiWorkload::generateKey(VectorRef<KeyValueRef> const& data, int minKeyLength, int maxKeyLength, std::string prefix) {
int keyLength = g_random->randomInt(minKeyLength, maxKeyLength + 1);
int keyLength = deterministicRandom()->randomInt(minKeyLength, maxKeyLength + 1);
char *keyBuffer = new char[keyLength + 1];
if(onlyLowerCase) {
for(int i = 0; i < keyLength; i++)
keyBuffer[i] = g_random->randomInt('a', 'z' + 1);
keyBuffer[i] = deterministicRandom()->randomInt('a', 'z' + 1);
}
else {
for(int i = 0; i < keyLength; i+= sizeof(uint32_t)) {
uint32_t val = g_random->randomUInt32();
uint32_t val = deterministicRandom()->randomUInt32();
memcpy(&keyBuffer[i], &val, std::min(keyLength - i, (int)sizeof(uint32_t)));
}
//Don't allow the first character of the key to be 0xff
if(keyBuffer[0] == '\xff')
keyBuffer[0] = g_random->randomInt(0, 255);
keyBuffer[0] = deterministicRandom()->randomInt(0, 255);
}
keyBuffer[keyLength] = '\0';
@ -240,21 +240,21 @@ Key ApiWorkload::generateKey(VectorRef<KeyValueRef> const& data, int minKeyLengt
//Generates a random key selector with a specified maximum offset
KeySelector ApiWorkload::generateKeySelector(VectorRef<KeyValueRef> const& data, int maxOffset) {
Key key = selectRandomKey(data, 0.5);
return KeySelector(KeySelectorRef(key, g_random->randomInt(0, 2) == 1, g_random->randomInt(-maxOffset, maxOffset + 1)));
return KeySelector(KeySelectorRef(key, deterministicRandom()->randomInt(0, 2) == 1, deterministicRandom()->randomInt(-maxOffset, maxOffset + 1)));
}
//Selects a random key. There is a <probabilityKeyExists> probability that the key will be chosen from the keyset in data, otherwise the key will
//be a randomly generated key
Key ApiWorkload::selectRandomKey(VectorRef<KeyValueRef> const& data, double probabilityKeyExists) {
if(g_random->random01() < probabilityKeyExists)
return data[g_random->randomInt(0, data.size())].key;
if(deterministicRandom()->random01() < probabilityKeyExists)
return data[deterministicRandom()->randomInt(0, data.size())].key;
else
return generateKey(data, minLongKeyLength, maxLongKeyLength, clientPrefix);
}
//Generates a random value
Value ApiWorkload::generateValue(int minValueLength, int maxValueLength) {
int valueLength = g_random->randomInt(minValueLength, maxValueLength + 1);
int valueLength = deterministicRandom()->randomInt(minValueLength, maxValueLength + 1);
return Value(std::string(valueLength, 'x'));
}
@ -265,7 +265,7 @@ Value ApiWorkload::generateValue() {
//Creates a random transaction factory to produce transaction of one of the TransactionType choices
ACTOR Future<Void> chooseTransactionFactory(Database cx, std::vector<TransactionType> choices, ApiWorkload *self) {
TransactionType transactionType = g_random->randomChoice(choices);
TransactionType transactionType = deterministicRandom()->randomChoice(choices);
if(transactionType == NATIVE) {
printf("client %d: Running NativeAPI Transactions\n", self->clientPrefixInt);

View File

@ -90,7 +90,7 @@ struct FlowTransactionWrapper : public TransactionWrapper {
T transaction;
T lastTransaction;
FlowTransactionWrapper(Database cx, Database extraDB, bool useExtraDB) : cx(cx), extraDB(extraDB), useExtraDB(useExtraDB), transaction(cx) {
if(useExtraDB && g_random->random01() < 0.5) {
if(useExtraDB && deterministicRandom()->random01() < 0.5) {
transaction = T(extraDB);
}
}
@ -141,7 +141,7 @@ struct FlowTransactionWrapper : public TransactionWrapper {
Future<Void> returnVal = transaction.onError(e);
if( useExtraDB ) {
lastTransaction = std::move(transaction);
transaction = T( g_random->random01() < 0.5 ? extraDB : cx );
transaction = T( deterministicRandom()->random01() < 0.5 ? extraDB : cx );
}
return returnVal;
}

View File

@ -99,7 +99,7 @@ struct AsyncFileWorkload : TestWorkload
state bool fileCreated = self->path.length() == 0;
if(fileCreated)
{
self->path = "asyncfile." + g_random->randomUniqueID().toString();
self->path = "asyncfile." + deterministicRandom()->randomUniqueID().toString();
flags &= ~IAsyncFile::OPEN_READONLY;
flags |= IAsyncFile::OPEN_READWRITE | IAsyncFile::OPEN_CREATE;
}

View File

@ -30,7 +30,7 @@ RandomByteGenerator::RandomByteGenerator(){
BUF_SIZE = 16*(1<<20);
b1 = new char[BUF_SIZE];
for (int i=0;i<BUF_SIZE/sizeof(uint32_t);i++)
((uint32_t*)b1)[i] = g_random->randomUInt32();
((uint32_t*)b1)[i] = deterministicRandom()->randomUInt32();
}
RandomByteGenerator::~RandomByteGenerator(){
@ -41,9 +41,9 @@ RandomByteGenerator::~RandomByteGenerator(){
void RandomByteGenerator::writeRandomBytesToBuffer(void *buf, int bytes){
ASSERT(bytes < BUF_SIZE-1);
int o1, o2;
o1 = g_random->randomInt(0, BUF_SIZE-bytes) / 8;
o1 = deterministicRandom()->randomInt(0, BUF_SIZE-bytes) / 8;
do {
o2 = g_random->randomInt(0, BUF_SIZE-bytes) / 8;
o2 = deterministicRandom()->randomInt(0, BUF_SIZE-bytes) / 8;
} while (o1==o2);
int64_t *out64 = (int64_t*)buf;

View File

@ -286,7 +286,7 @@ struct AsyncFileCorrectnessWorkload : public AsyncFileWorkload
vector<int> cdf = vector<int>(cdfArray, cdfArray + 6);
//Choose a random operation type (READ, WRITE, SYNC, REOPEN, TRUNCATE).
int random = g_random->randomInt(0, cdf.back());
int random = deterministicRandom()->randomInt(0, cdf.back());
for(int i = 0; i < cdf.size() - 1; i++)
{
if(cdf[i] <= random && random < cdf[i + 1])
@ -322,13 +322,13 @@ struct AsyncFileCorrectnessWorkload : public AsyncFileWorkload
//Generate random length and offset
if(unbufferedIO)
{
info.length = g_random->randomInt(1, maxOperationSize / _PAGE_SIZE + 1) * _PAGE_SIZE;
info.offset = (int64_t)(g_random->random01() * maxOffset / _PAGE_SIZE) * _PAGE_SIZE;
info.length = deterministicRandom()->randomInt(1, maxOperationSize / _PAGE_SIZE + 1) * _PAGE_SIZE;
info.offset = (int64_t)(deterministicRandom()->random01() * maxOffset / _PAGE_SIZE) * _PAGE_SIZE;
}
else
{
info.length = g_random->randomInt(1, maxOperationSize);
info.offset = (int64_t)(g_random->random01() * maxOffset);
info.length = deterministicRandom()->randomInt(1, maxOperationSize);
info.offset = (int64_t)(deterministicRandom()->random01() * maxOffset);
}
} while(checkFileLocked(info.operation, info.offset, info.length));
@ -362,9 +362,9 @@ struct AsyncFileCorrectnessWorkload : public AsyncFileWorkload
//Choose a random length to truncate to
if(unbufferedIO)
info.offset = (int64_t)(g_random->random01() * (2 * targetFileSize) / _PAGE_SIZE) * _PAGE_SIZE;
info.offset = (int64_t)(deterministicRandom()->random01() * (2 * targetFileSize) / _PAGE_SIZE) * _PAGE_SIZE;
else
info.offset = (int64_t)(g_random->random01() * (2 * targetFileSize));
info.offset = (int64_t)(deterministicRandom()->random01() * (2 * targetFileSize));
}
} while(!allowFlushingOperations && info.flushOperations);
@ -389,7 +389,7 @@ struct AsyncFileCorrectnessWorkload : public AsyncFileWorkload
{
for(int i = 0; i < length; i+= sizeof(uint32_t))
{
uint32_t val = g_random->randomUInt32();
uint32_t val = deterministicRandom()->randomUInt32();
memcpy(&buffer[i], &val, std::min(length - i, (int)sizeof(uint32_t)));
}
}

View File

@ -243,14 +243,14 @@ struct AsyncFileReadWorkload : public AsyncFileWorkload
if (fixedRate)
wait( poisson( &lastTime, 1.0 / fixedRate ) );
//state Future<Void> d = delay( 1/25. * (.75 + 0.5*g_random->random01()) );
//state Future<Void> d = delay( 1/25. * (.75 + 0.5*deterministicRandom()->random01()) );
int64_t offset;
if(self->unbufferedIO)
offset = (int64_t)(g_random->random01() * (self->fileSize - 1) / AsyncFileWorkload::_PAGE_SIZE) * AsyncFileWorkload::_PAGE_SIZE;
offset = (int64_t)(deterministicRandom()->random01() * (self->fileSize - 1) / AsyncFileWorkload::_PAGE_SIZE) * AsyncFileWorkload::_PAGE_SIZE;
else
offset = (int64_t)(g_random->random01() * (self->fileSize - 1));
offset = (int64_t)(deterministicRandom()->random01() * (self->fileSize - 1));
writeFlag = g_random->random01() < self->writeFraction;
writeFlag = deterministicRandom()->random01() < self->writeFraction;
if (writeFlag)
self->rbg.writeRandomBytesToBuffer((char*)self->readBuffers[bufferIndex]->buffer, self->readSize);
@ -305,9 +305,9 @@ struct AsyncFileReadWorkload : public AsyncFileWorkload
offset = 0;
}
else if(self->unbufferedIO)
offset = (int64_t)(g_random->random01() * (self->fileSize - 1) / AsyncFileWorkload::_PAGE_SIZE) * AsyncFileWorkload::_PAGE_SIZE;
offset = (int64_t)(deterministicRandom()->random01() * (self->fileSize - 1) / AsyncFileWorkload::_PAGE_SIZE) * AsyncFileWorkload::_PAGE_SIZE;
else
offset = (int64_t)(g_random->random01() * (self->fileSize - 1));
offset = (int64_t)(deterministicRandom()->random01() * (self->fileSize - 1));
//Perform the read. Don't allow it to be cancelled (because the underlying IO may not be cancellable) and don't allow
//objects that the read uses to be deleted

View File

@ -147,9 +147,9 @@ struct AsyncFileWriteWorkload : public AsyncFileWorkload
offset = 0;
}
else if(self->unbufferedIO)
offset = (int64_t)(g_random->random01() * (self->fileSize - 1) / AsyncFileWorkload::_PAGE_SIZE) * AsyncFileWorkload::_PAGE_SIZE;
offset = (int64_t)(deterministicRandom()->random01() * (self->fileSize - 1) / AsyncFileWorkload::_PAGE_SIZE) * AsyncFileWorkload::_PAGE_SIZE;
else
offset = (int64_t)(g_random->random01() * (self->fileSize - 1));
offset = (int64_t)(deterministicRandom()->random01() * (self->fileSize - 1));
}
wait(waitForAll(self->writeFutures));

View File

@ -147,11 +147,11 @@ struct AtomicOpsWorkload : TestWorkload {
state ReadYourWritesTransaction tr(cx);
loop {
try {
int group = g_random->randomInt(0,100);
uint64_t intValue = g_random->randomInt( 0, 10000000 );
int group = deterministicRandom()->randomInt(0,100);
uint64_t intValue = deterministicRandom()->randomInt( 0, 10000000 );
Key val = StringRef((const uint8_t*) &intValue, sizeof(intValue));
tr.set(self->logKey(group), val);
tr.atomicOp(StringRef(format("ops%08x%08x",group,g_random->randomInt(0,self->nodeCount/100))), val, self->opType);
tr.atomicOp(StringRef(format("ops%08x%08x",group,deterministicRandom()->randomInt(0,self->nodeCount/100))), val, self->opType);
wait( tr.commit() );
break;
} catch( Error &e ) {

View File

@ -103,7 +103,7 @@ public:
// Test Atomic ops on non existing keys that results in a set
ACTOR Future<Void> testAtomicOpSetOnNonExistingKey(Database cx, AtomicOpsApiCorrectnessWorkload* self, uint32_t opType, Key key) {
state uint64_t intValue = g_random->randomInt(0, 10000000);
state uint64_t intValue = deterministicRandom()->randomInt(0, 10000000);
state Value val = StringRef((const uint8_t*)&intValue, sizeof(intValue));
// Do operation on Storage Server
@ -145,7 +145,7 @@ public:
// Test Atomic ops on non existing keys that results in a unset
ACTOR Future<Void> testAtomicOpUnsetOnNonExistingKey(Database cx, AtomicOpsApiCorrectnessWorkload* self, uint32_t opType, Key key) {
state uint64_t intValue = g_random->randomInt(0, 10000000);
state uint64_t intValue = deterministicRandom()->randomInt(0, 10000000);
state Value val = StringRef((const uint8_t*)&intValue, sizeof(intValue));
// Do operation on Storage Server
@ -191,8 +191,8 @@ public:
ACTOR Future<Void> testAtomicOpOnEmptyValue(Database cx, AtomicOpsApiCorrectnessWorkload* self, uint32_t opType, Key key, DoAtomicOpOnEmptyValueFunction opFunc) {
state Value existingVal;
state Value otherVal;
state uint64_t val = g_random->randomInt(0, 10000000);
if (g_random->random01() < 0.5) {
state uint64_t val = deterministicRandom()->randomInt(0, 10000000);
if (deterministicRandom()->random01() < 0.5) {
existingVal = StringRef((const uint8_t*)&val, sizeof(val));
otherVal = StringRef();
}
@ -240,8 +240,8 @@ public:
// Test atomic ops in the normal case when the existing value is present
ACTOR Future<Void> testAtomicOpApi(Database cx, AtomicOpsApiCorrectnessWorkload* self, uint32_t opType, Key key, DoAtomicOpFunction opFunc) {
state uint64_t intValue1 = g_random->randomInt(0, 10000000);
state uint64_t intValue2 = g_random->randomInt(0, 10000000);
state uint64_t intValue1 = deterministicRandom()->randomInt(0, 10000000);
state uint64_t intValue2 = deterministicRandom()->randomInt(0, 10000000);
state Value val1 = StringRef((const uint8_t*)&intValue1, sizeof(intValue1));
state Value val2 = StringRef((const uint8_t *)&intValue2, sizeof(intValue2));
@ -290,8 +290,8 @@ public:
ACTOR Future<Void> testCompareAndClearAtomicOpApi(Database cx, AtomicOpsApiCorrectnessWorkload* self, Key key,
bool keySet) {
state uint64_t opType = MutationRef::CompareAndClear;
state uint64_t intValue1 = g_random->randomInt(0, 10000000);
state uint64_t intValue2 = g_random->coinflip() ? intValue1 : g_random->randomInt(0, 10000000);
state uint64_t intValue1 = deterministicRandom()->randomInt(0, 10000000);
state uint64_t intValue2 = deterministicRandom()->coinflip() ? intValue1 : deterministicRandom()->randomInt(0, 10000000);
state Value val1 = StringRef((const uint8_t*)&intValue1, sizeof(intValue1));
state Value val2 = StringRef((const uint8_t*)&intValue2, sizeof(intValue2));

View File

@ -61,12 +61,12 @@ struct AtomicRestoreWorkload : TestWorkload {
ACTOR static Future<Void> _start(Database cx, AtomicRestoreWorkload* self) {
state FileBackupAgent backupAgent;
wait( delay(self->startAfter * g_random->random01()) );
wait( delay(self->startAfter * deterministicRandom()->random01()) );
TraceEvent("AtomicRestore_Start");
state std::string backupContainer = "file://simfdb/backups/";
try {
wait(backupAgent.submitBackup(cx, StringRef(backupContainer), g_random->randomInt(0, 100), BackupAgentBase::getDefaultTagName(), self->backupRanges, false));
wait(backupAgent.submitBackup(cx, StringRef(backupContainer), deterministicRandom()->randomInt(0, 100), BackupAgentBase::getDefaultTagName(), self->backupRanges, false));
}
catch (Error& e) {
if (e.code() != error_code_backup_unneeded && e.code() != error_code_backup_duplicate)
@ -76,12 +76,12 @@ struct AtomicRestoreWorkload : TestWorkload {
TraceEvent("AtomicRestore_Wait");
wait(success( backupAgent.waitBackup(cx, BackupAgentBase::getDefaultTagName(), false) ));
TraceEvent("AtomicRestore_BackupStart");
wait( delay(self->restoreAfter * g_random->random01()) );
wait( delay(self->restoreAfter * deterministicRandom()->random01()) );
TraceEvent("AtomicRestore_RestoreStart");
loop {
std::vector<Future<Version>> restores;
if (g_random->random01() < 0.5) {
if (deterministicRandom()->random01() < 0.5) {
for (auto &range : self->backupRanges)
restores.push_back(backupAgent.atomicRestore(cx, BackupAgentBase::getDefaultTag(), range, StringRef(), StringRef()));
}

View File

@ -155,19 +155,19 @@ struct AtomicSwitchoverWorkload : TestWorkload {
TraceEvent("AS_Wait1");
wait(success( backupAgent.waitBackup(self->extraDB, BackupAgentBase::getDefaultTag(), false) ));
TraceEvent("AS_Ready1");
wait( delay(g_random->random01()*self->switch1delay) );
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( restoreAgent.waitBackup(cx, BackupAgentBase::getDefaultTag(), false) ));
TraceEvent("AS_Ready2");
wait( delay(g_random->random01()*self->switch2delay) );
wait( delay(deterministicRandom()->random01()*self->switch2delay) );
TraceEvent("AS_Switch2");
wait( restoreAgent.atomicSwitchover(cx, BackupAgentBase::getDefaultTag(), self->backupRanges, StringRef(), StringRef()) );
TraceEvent("AS_Wait3");
wait(success( backupAgent.waitBackup(self->extraDB, BackupAgentBase::getDefaultTag(), false) ));
TraceEvent("AS_Ready3");
wait( delay(g_random->random01()*self->stopDelay) );
wait( delay(deterministicRandom()->random01()*self->stopDelay) );
TraceEvent("AS_Abort");
wait( backupAgent.abortBackup(self->extraDB, BackupAgentBase::getDefaultTag()) );
TraceEvent("AS_Done");

View File

@ -25,7 +25,7 @@
#include "flow/actorcompiler.h" // This must be the last #include.
KeySelector randomizedSelector(const KeyRef &key, bool orEqual, int offset ) {
if( orEqual && g_random->random01() > 0.5 )
if( orEqual && deterministicRandom()->random01() > 0.5 )
return KeySelectorRef( keyAfter(key), false, offset );
return KeySelectorRef( key, orEqual, offset );
}
@ -101,9 +101,9 @@ struct BackgroundSelectorWorkload : TestWorkload {
state bool restartProcess;
loop {
forward = g_random->randomInt(0,2) != 0;
forward = deterministicRandom()->randomInt(0,2) != 0;
direction = forward ? 1 : -1;
diff = g_random->randomInt(0, self->maxDiff);
diff = deterministicRandom()->randomInt(0, self->maxDiff);
//Setup start and end key
loop {
@ -138,8 +138,8 @@ struct BackgroundSelectorWorkload : TestWorkload {
loop {
wait( poisson( &lastTime, 1.0 / self->transactionsPerSecond ) );
tr.reset();
startDrift = direction * g_random->randomInt( self->minDrift, self->maxDrift );
endDrift = direction * g_random->randomInt( self->minDrift, self->maxDrift );
startDrift = direction * deterministicRandom()->randomInt( self->minDrift, self->maxDrift );
endDrift = direction * deterministicRandom()->randomInt( self->minDrift, self->maxDrift );
//max sure the end drift does not violate maxDiff
endDrift = std::max( endDrift, startDrift - self->maxDiff - diff );

View File

@ -52,18 +52,18 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
backupTag = getOption(options, LiteralStringRef("backupTag"), BackupAgentBase::getDefaultTag());
backupRangesCount = getOption(options, LiteralStringRef("backupRangesCount"), 5);
backupRangeLengthMax = getOption(options, LiteralStringRef("backupRangeLengthMax"), 1);
abortAndRestartAfter = getOption(options, LiteralStringRef("abortAndRestartAfter"), g_random->random01() < 0.5 ? g_random->random01() * (restoreAfter - backupAfter) + backupAfter : 0.0);
differentialBackup = getOption(options, LiteralStringRef("differentialBackup"), g_random->random01() < 0.5 ? true : false);
abortAndRestartAfter = getOption(options, LiteralStringRef("abortAndRestartAfter"), deterministicRandom()->random01() < 0.5 ? deterministicRandom()->random01() * (restoreAfter - backupAfter) + backupAfter : 0.0);
differentialBackup = getOption(options, LiteralStringRef("differentialBackup"), deterministicRandom()->random01() < 0.5 ? true : false);
stopDifferentialAfter = getOption(options, LiteralStringRef("stopDifferentialAfter"),
differentialBackup ? g_random->random01() * (restoreAfter - std::max(abortAndRestartAfter,backupAfter)) + std::max(abortAndRestartAfter,backupAfter) : 0.0);
differentialBackup ? deterministicRandom()->random01() * (restoreAfter - std::max(abortAndRestartAfter,backupAfter)) + std::max(abortAndRestartAfter,backupAfter) : 0.0);
agentRequest = getOption(options, LiteralStringRef("simBackupAgents"), true);
allowPauses = getOption(options, LiteralStringRef("allowPauses"), true);
shareLogRange = getOption(options, LiteralStringRef("shareLogRange"), false);
prefixesMandatory = getOption(options, LiteralStringRef("prefixesMandatory"), std::vector<std::string>());
shouldSkipRestoreRanges = g_random->random01() < 0.3 ? true : false;
shouldSkipRestoreRanges = deterministicRandom()->random01() < 0.3 ? true : false;
TraceEvent("BARW_ClientId").detail("Id", wcx.clientId);
UID randomID = g_nondeterministic_random->randomUniqueID();
UID randomID = nondeterministicRandom()->randomUniqueID();
TraceEvent("BARW_PerformRestore", randomID).detail("Value", performRestore);
if (shareLogRange) {
bool beforePrefix = sharedRandomNumber & 1;
@ -77,7 +77,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
// Add backup ranges
std::set<std::string> rangeEndpoints;
while (rangeEndpoints.size() < backupRangesCount * 2) {
rangeEndpoints.insert(g_random->randomAlphaNumeric(g_random->randomInt(1, backupRangeLengthMax + 1)));
rangeEndpoints.insert(deterministicRandom()->randomAlphaNumeric(deterministicRandom()->randomInt(1, backupRangeLengthMax + 1)));
}
// Create ranges from the keys, in order, to prevent overlaps
@ -101,7 +101,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
intersection = true;
TraceEvent("BARW_PrefixSkipRangeDetails").detail("PrefixMandatory", printable(mandatoryRange)).detail("BackUpRange", printable(range)).detail("Intersection", intersection);
}
if (!intersection && g_random->random01() < 0.5)
if (!intersection && deterministicRandom()->random01() < 0.5)
skipRestoreRanges.push_back(skipRestoreRanges.arena(), range);
else
restoreRanges.push_back(restoreRanges.arena(), range);
@ -181,9 +181,9 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
ACTOR static Future<Void> changePaused(Database cx, FileBackupAgent* backupAgent) {
loop {
wait( backupAgent->taskBucket->changePause(cx, true) );
wait( delay(30*g_random->random01()) );
wait( delay(30*deterministicRandom()->random01()) );
wait( backupAgent->taskBucket->changePause(cx, false) );
wait( delay(120*g_random->random01()) );
wait( delay(120*deterministicRandom()->random01()) );
}
}
@ -201,7 +201,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
ACTOR static Future<Void> doBackup(BackupAndRestoreCorrectnessWorkload* self, double startDelay, FileBackupAgent* backupAgent, Database cx,
Key tag, Standalone<VectorRef<KeyRangeRef>> backupRanges, double stopDifferentialDelay, Promise<Void> submittted) {
state UID randomID = g_nondeterministic_random->randomUniqueID();
state UID randomID = nondeterministicRandom()->randomUniqueID();
state Future<Void> stopDifferentialFuture = delay(stopDifferentialDelay);
wait( delay( startDelay ));
@ -225,7 +225,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
state Future<Void> status = statusLoop(cx, tag.toString());
try {
wait(backupAgent->submitBackup(cx, StringRef(backupContainer), g_random->randomInt(0, 100), tag.toString(), backupRanges, stopDifferentialDelay ? false : true));
wait(backupAgent->submitBackup(cx, StringRef(backupContainer), deterministicRandom()->randomInt(0, 100), tag.toString(), backupRanges, stopDifferentialDelay ? false : true));
}
catch (Error& e) {
TraceEvent("BARW_DoBackupSubmitBackupException", randomID).error(e).detail("Tag", printable(tag));
@ -368,7 +368,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
.detail("BackupAfter", self->backupAfter).detail("RestoreAfter", self->restoreAfter)
.detail("AbortAndRestartAfter", self->abortAndRestartAfter).detail("DifferentialAfter", self->stopDifferentialAfter);
state UID randomID = g_nondeterministic_random->randomUniqueID();
state UID randomID = nondeterministicRandom()->randomUniqueID();
if(self->allowPauses && BUGGIFY) {
state Future<Void> cp = changePaused(cx, &backupAgent);
}
@ -416,7 +416,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
if (!self->locked && BUGGIFY) {
TraceEvent("BARW_SubmitBackup2", randomID).detail("Tag", printable(self->backupTag));
try {
extraBackup = backupAgent.submitBackup(cx, LiteralStringRef("file://simfdb/backups/"), g_random->randomInt(0, 100), self->backupTag.toString(), self->backupRanges, true);
extraBackup = backupAgent.submitBackup(cx, LiteralStringRef("file://simfdb/backups/"), deterministicRandom()->randomInt(0, 100), self->backupTag.toString(), self->backupRanges, true);
}
catch (Error& e) {
TraceEvent("BARW_SubmitBackup2Exception", randomID).error(e).detail("BackupTag", printable(self->backupTag));
@ -429,7 +429,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
wait(startRestore);
if (lastBackupContainer && self->performRestore) {
if (g_random->random01() < 0.5) {
if (deterministicRandom()->random01() < 0.5) {
wait(attemptDirtyRestore(self, cx, &backupAgent, StringRef(lastBackupContainer->getURL()), randomID));
}
wait(runRYWTransaction(cx, [=](Reference<ReadYourWritesTransaction> tr) -> Future<Void> {
@ -446,14 +446,14 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
Version targetVersion = -1;
if(desc.maxRestorableVersion.present()) {
if( g_random->random01() < 0.1 ) {
if( deterministicRandom()->random01() < 0.1 ) {
targetVersion = desc.minRestorableVersion.get();
}
else if( g_random->random01() < 0.1 ) {
else if( deterministicRandom()->random01() < 0.1 ) {
targetVersion = desc.maxRestorableVersion.get();
}
else if( g_random->random01() < 0.5 ) {
targetVersion = g_random->randomInt64(desc.minRestorableVersion.get(), desc.contiguousLogEnd.get());
else if( deterministicRandom()->random01() < 0.5 ) {
targetVersion = deterministicRandom()->randomInt64(desc.minRestorableVersion.get(), desc.contiguousLogEnd.get());
}
}
@ -461,7 +461,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
state std::vector<Standalone<StringRef>> restoreTags;
state bool multipleRangesInOneTag = false;
state int restoreIndex = 0;
if (g_random->random01() < 0.5) {
if (deterministicRandom()->random01() < 0.5) {
for (restoreIndex = 0; restoreIndex < self->restoreRanges.size(); restoreIndex++) {
auto range = self->restoreRanges[restoreIndex];
Standalone<StringRef> restoreTag(self->backupTag.toString() + "_" + std::to_string(restoreIndex));
@ -478,7 +478,7 @@ struct BackupAndRestoreCorrectnessWorkload : TestWorkload {
// Sometimes kill and restart the restore
if (BUGGIFY) {
wait(delay(g_random->randomInt(0, 10)));
wait(delay(deterministicRandom()->randomInt(0, 10)));
if (multipleRangesInOneTag) {
FileBackupAgent::ERestoreState rs = wait(backupAgent.abortRestore(cx, restoreTags[0]));
// The restore may have already completed, or the abort may have been done before the restore

View File

@ -51,15 +51,15 @@ struct BackupToDBCorrectnessWorkload : TestWorkload {
backupPrefix = getOption(options, LiteralStringRef("backupPrefix"), StringRef());
backupRangesCount = getOption(options, LiteralStringRef("backupRangesCount"), 5); //tests can hangs if set higher than 1 + BACKUP_MAP_KEY_LOWER_LIMIT
backupRangeLengthMax = getOption(options, LiteralStringRef("backupRangeLengthMax"), 1);
abortAndRestartAfter = getOption(options, LiteralStringRef("abortAndRestartAfter"), (!locked && g_random->random01() < 0.5) ? g_random->random01() * (restoreAfter - backupAfter) + backupAfter : 0.0);
differentialBackup = getOption(options, LiteralStringRef("differentialBackup"), g_random->random01() < 0.5 ? true : false);
abortAndRestartAfter = getOption(options, LiteralStringRef("abortAndRestartAfter"), (!locked && deterministicRandom()->random01() < 0.5) ? deterministicRandom()->random01() * (restoreAfter - backupAfter) + backupAfter : 0.0);
differentialBackup = getOption(options, LiteralStringRef("differentialBackup"), deterministicRandom()->random01() < 0.5 ? true : false);
stopDifferentialAfter = getOption(options, LiteralStringRef("stopDifferentialAfter"),
differentialBackup ? g_random->random01() * (restoreAfter - std::max(abortAndRestartAfter,backupAfter)) + std::max(abortAndRestartAfter,backupAfter) : 0.0);
differentialBackup ? deterministicRandom()->random01() * (restoreAfter - std::max(abortAndRestartAfter,backupAfter)) + std::max(abortAndRestartAfter,backupAfter) : 0.0);
agentRequest = getOption(options, LiteralStringRef("simDrAgents"), true);
shareLogRange = getOption(options, LiteralStringRef("shareLogRange"), false);
// Use sharedRandomNumber if shareLogRange is true so that we can ensure backup and DR both backup the same range
beforePrefix = shareLogRange ? (sharedRandomNumber & 1) : (g_random->random01() < 0.5);
beforePrefix = shareLogRange ? (sharedRandomNumber & 1) : (deterministicRandom()->random01() < 0.5);
if (beforePrefix) {
extraPrefix = backupPrefix.withPrefix(LiteralStringRef("\xfe\xff\xfe"));
@ -74,7 +74,7 @@ struct BackupToDBCorrectnessWorkload : TestWorkload {
KeyRef beginRange;
KeyRef endRange;
UID randomID = g_nondeterministic_random->randomUniqueID();
UID randomID = nondeterministicRandom()->randomUniqueID();
if (shareLogRange) {
if (beforePrefix)
@ -91,8 +91,8 @@ struct BackupToDBCorrectnessWorkload : TestWorkload {
for (int rangeLoop = 0; rangeLoop < backupRangesCount; rangeLoop++)
{
// Get a random range of a random sizes
beginRange = KeyRef(backupRanges.arena(), g_random->randomAlphaNumeric(g_random->randomInt(1, backupRangeLengthMax + 1)));
endRange = KeyRef(backupRanges.arena(), g_random->randomAlphaNumeric(g_random->randomInt(1, backupRangeLengthMax + 1)));
beginRange = KeyRef(backupRanges.arena(), deterministicRandom()->randomAlphaNumeric(deterministicRandom()->randomInt(1, backupRangeLengthMax + 1)));
endRange = KeyRef(backupRanges.arena(), deterministicRandom()->randomAlphaNumeric(deterministicRandom()->randomInt(1, backupRangeLengthMax + 1)));
// Add the range to the array
backupRanges.push_back_deep(backupRanges.arena(), (beginRange < endRange) ? KeyRangeRef(beginRange, endRange) : KeyRangeRef(endRange, beginRange));
@ -202,7 +202,7 @@ struct BackupToDBCorrectnessWorkload : TestWorkload {
ACTOR static Future<Void> doBackup(BackupToDBCorrectnessWorkload* self, double startDelay, DatabaseBackupAgent* backupAgent, Database cx,
Key tag, Standalone<VectorRef<KeyRangeRef>> backupRanges, double stopDifferentialDelay, Promise<Void> submitted) {
state UID randomID = g_nondeterministic_random->randomUniqueID();
state UID randomID = nondeterministicRandom()->randomUniqueID();
state Future<Void> stopDifferentialFuture = delay(stopDifferentialDelay);
wait( delay( startDelay ));
@ -442,7 +442,7 @@ struct BackupToDBCorrectnessWorkload : TestWorkload {
TraceEvent("BARW_Arguments").detail("BackupTag", printable(self->backupTag)).detail("BackupAfter", self->backupAfter)
.detail("AbortAndRestartAfter", self->abortAndRestartAfter).detail("DifferentialAfter", self->stopDifferentialAfter);
state UID randomID = g_nondeterministic_random->randomUniqueID();
state UID randomID = nondeterministicRandom()->randomUniqueID();
// Increment the backup agent requets
if (self->agentRequest) {

View File

@ -34,7 +34,7 @@ struct BackupToDBUpgradeWorkload : TestWorkload {
Database extraDB;
BackupToDBUpgradeWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) {
backupAfter = getOption(options, LiteralStringRef("backupAfter"), g_random->random01() * 10.0);
backupAfter = getOption(options, LiteralStringRef("backupAfter"), deterministicRandom()->random01() * 10.0);
backupPrefix = getOption(options, LiteralStringRef("backupPrefix"), StringRef());
backupRangeLengthMax = getOption(options, LiteralStringRef("backupRangeLengthMax"), 1);
stopDifferentialAfter = getOption(options, LiteralStringRef("stopDifferentialAfter"), 60.0);
@ -56,8 +56,8 @@ struct BackupToDBUpgradeWorkload : TestWorkload {
for (int rangeLoop = 0; rangeLoop < backupRangesCount; rangeLoop++)
{
// Get a random range of a random sizes
beginRange = KeyRef(backupRanges.arena(), g_random->randomAlphaNumeric(g_random->randomInt(1, backupRangeLengthMax + 1)));
endRange = KeyRef(backupRanges.arena(), g_random->randomAlphaNumeric(g_random->randomInt(1, backupRangeLengthMax + 1)));
beginRange = KeyRef(backupRanges.arena(), deterministicRandom()->randomAlphaNumeric(deterministicRandom()->randomInt(1, backupRangeLengthMax + 1)));
endRange = KeyRef(backupRanges.arena(), deterministicRandom()->randomAlphaNumeric(deterministicRandom()->randomInt(1, backupRangeLengthMax + 1)));
// Add the range to the array
backupRanges.push_back_deep(backupRanges.arena(), (beginRange < endRange) ? KeyRangeRef(beginRange, endRange) : KeyRangeRef(endRange, beginRange));

View File

@ -61,8 +61,8 @@ Future<uint64_t> setupRange( Database cx, T* workload, uint64_t begin, uint64_t
loop {
state Transaction tr(cx);
try {
//if( g_random->random01() < 0.001 )
// tr.debugTransaction( g_random->randomUniqueID() );
//if( deterministicRandom()->random01() < 0.001 )
// tr.debugTransaction( deterministicRandom()->randomUniqueID() );
state Standalone<KeyValueRef> sampleKV = (*workload)( begin );
Optional<Value> f = wait( tr.get( sampleKV.key ) );
@ -236,13 +236,13 @@ Future<Void> bulkSetup( Database cx, T* workload, uint64_t nodeCount, Promise<do
}
}
wait( delay( g_random->random01() / 4 ) ); // smear over .25 seconds
wait( delay( deterministicRandom()->random01() / 4 ) ); // smear over .25 seconds
state int BULK_SETUP_WORKERS = 40;
// See that each chunk inserted is about 10KB
int size_total = 0;
for( int i = 0; i < 100; i++ ) {
Standalone<KeyValueRef> sampleKV = (*workload)( startNode + (uint64_t)(g_random->random01()*(endNode - startNode)) );
Standalone<KeyValueRef> sampleKV = (*workload)( startNode + (uint64_t)(deterministicRandom()->random01()*(endNode - startNode)) );
size_total += sampleKV.key.size() + sampleKV.value.size();
}
state int BULK_SETUP_RANGE_SIZE = size_total == 0 ? 50 : std::max(1, 10000 / (size_total / 100));
@ -258,7 +258,7 @@ Future<Void> bulkSetup( Database cx, T* workload, uint64_t nodeCount, Promise<do
// create a random vector of range-create jobs
for(uint64_t n=startNode; n<endNode; n+=BULK_SETUP_RANGE_SIZE)
jobs.push_back( std::make_pair( n, std::min(endNode, n+BULK_SETUP_RANGE_SIZE) ) );
g_random->randomShuffle(jobs);
deterministicRandom()->randomShuffle(jobs);
// fire up the workers and wait for them to eat all the jobs
double maxWorkerInsertRate = maxKeyInsertRate / BULK_SETUP_WORKERS / workload->clientCount;

View File

@ -59,7 +59,7 @@ struct ChangeConfigWorkload : TestWorkload {
Reference<ClusterConnectionFile> extraFile(new ClusterConnectionFile(*g_simulator.extraDB));
state Database extraDB = Database::createDatabase(extraFile, -1);
wait(delay(5*g_random->random01()));
wait(delay(5*deterministicRandom()->random01()));
if (self->configMode.size()) {
wait(success(changeConfig(extraDB, self->configMode, true)));
TraceEvent("WaitForReplicasExtra");
@ -71,15 +71,15 @@ struct ChangeConfigWorkload : TestWorkload {
else
wait(success(changeQuorum(extraDB, specifiedQuorumChange(NetworkAddress::parseList(self->networkAddresses)))));
}
wait(delay(5*g_random->random01()));
wait(delay(5*deterministicRandom()->random01()));
}
return Void();
}
ACTOR Future<Void> ChangeConfigClient( Database cx, ChangeConfigWorkload *self) {
wait( delay( self->minDelayBeforeChange + g_random->random01() * ( self->maxDelayBeforeChange - self->minDelayBeforeChange ) ) );
wait( delay( self->minDelayBeforeChange + deterministicRandom()->random01() * ( self->maxDelayBeforeChange - self->minDelayBeforeChange ) ) );
state bool extraConfigureBefore = g_random->random01() < 0.5;
state bool extraConfigureBefore = deterministicRandom()->random01() < 0.5;
if(extraConfigureBefore) {
wait( self->extraDatabaseConfigure(self) );

View File

@ -105,8 +105,8 @@ struct ClientTransactionProfileCorrectnessWorkload : TestWorkload {
: TestWorkload(wcx)
{
if (clientId == 0) {
samplingProbability = getOption(options, LiteralStringRef("samplingProbability"), g_random->random01() / 10); //rand range 0 - 0.1
trInfoSizeLimit = getOption(options, LiteralStringRef("trInfoSizeLimit"), g_random->randomInt(100 * 1024, 10 * 1024 * 1024)); // 100 KB - 10 MB
samplingProbability = getOption(options, LiteralStringRef("samplingProbability"), deterministicRandom()->random01() / 10); //rand range 0 - 0.1
trInfoSizeLimit = getOption(options, LiteralStringRef("trInfoSizeLimit"), deterministicRandom()->randomInt(100 * 1024, 10 * 1024 * 1024)); // 100 KB - 10 MB
TraceEvent(SevInfo, "ClientTransactionProfilingSetup").detail("SamplingProbability", samplingProbability).detail("TrInfoSizeLimit", trInfoSizeLimit);
}
}

View File

@ -32,17 +32,17 @@ static const char* redundancies[] = { "single", "double", "triple" };
std::string generateRegions() {
std::string result;
if(g_simulator.physicalDatacenters == 1 || (g_simulator.physicalDatacenters == 2 && g_random->random01() < 0.25) || g_simulator.physicalDatacenters == 3) {
if(g_simulator.physicalDatacenters == 1 || (g_simulator.physicalDatacenters == 2 && deterministicRandom()->random01() < 0.25) || g_simulator.physicalDatacenters == 3) {
return " usable_regions=1 regions=\"\"";
}
if(g_random->random01() < 0.25) {
return format(" usable_regions=%d", g_random->randomInt(1,3));
if(deterministicRandom()->random01() < 0.25) {
return format(" usable_regions=%d", deterministicRandom()->randomInt(1,3));
}
int primaryPriority = 1;
int remotePriority = -1;
double priorityType = g_random->random01();
double priorityType = deterministicRandom()->random01();
if(priorityType < 0.1) {
primaryPriority = -1;
remotePriority = 1;
@ -65,7 +65,7 @@ std::string generateRegions() {
StatusArray remoteDcArr;
remoteDcArr.push_back(remoteDcObj);
if(g_simulator.physicalDatacenters > 3 && g_random->random01() < 0.5) {
if(g_simulator.physicalDatacenters > 3 && deterministicRandom()->random01() < 0.5) {
StatusObject primarySatelliteObj;
primarySatelliteObj["id"] = "2";
primarySatelliteObj["priority"] = 1;
@ -78,7 +78,7 @@ std::string generateRegions() {
remoteSatelliteObj["satellite"] = 1;
remoteDcArr.push_back(remoteSatelliteObj);
if(g_simulator.physicalDatacenters > 5 && g_random->random01() < 0.5) {
if(g_simulator.physicalDatacenters > 5 && deterministicRandom()->random01() < 0.5) {
StatusObject primarySatelliteObjB;
primarySatelliteObjB["id"] = "4";
primarySatelliteObjB["priority"] = 1;
@ -91,7 +91,7 @@ std::string generateRegions() {
remoteSatelliteObjB["satellite"] = 1;
remoteDcArr.push_back(remoteSatelliteObjB);
int satellite_replication_type = g_random->randomInt(0,3);
int satellite_replication_type = deterministicRandom()->randomInt(0,3);
switch (satellite_replication_type) {
case 0: {
TEST( true ); // Simulated cluster using no satellite redundancy mode
@ -113,7 +113,7 @@ std::string generateRegions() {
ASSERT(false); // Programmer forgot to adjust cases.
}
} else {
int satellite_replication_type = g_random->randomInt(0,4);
int satellite_replication_type = deterministicRandom()->randomInt(0,4);
switch (satellite_replication_type) {
case 0: {
//FIXME: implement
@ -141,13 +141,13 @@ std::string generateRegions() {
}
}
if (g_random->random01() < 0.25) {
int logs = g_random->randomInt(1,7);
if (deterministicRandom()->random01() < 0.25) {
int logs = deterministicRandom()->randomInt(1,7);
primaryObj["satellite_logs"] = logs;
remoteObj["satellite_logs"] = logs;
}
int remote_replication_type = g_random->randomInt(0, 4);
int remote_replication_type = deterministicRandom()->randomInt(0, 4);
switch (remote_replication_type) {
case 0: {
//FIXME: implement
@ -172,8 +172,8 @@ std::string generateRegions() {
ASSERT(false); // Programmer forgot to adjust cases.
}
result += format(" log_routers=%d", g_random->randomInt(1,7));
result += format(" remote_logs=%d", g_random->randomInt(1,7));
result += format(" log_routers=%d", deterministicRandom()->randomInt(1,7));
result += format(" remote_logs=%d", deterministicRandom()->randomInt(1,7));
}
primaryObj["datacenters"] = primaryDcArr;
@ -182,10 +182,10 @@ std::string generateRegions() {
StatusArray regionArr;
regionArr.push_back(primaryObj);
if(g_random->random01() < 0.8) {
if(deterministicRandom()->random01() < 0.8) {
regionArr.push_back(remoteObj);
if(g_random->random01() < 0.25) {
result += format(" usable_regions=%d", g_random->randomInt(1,3));
if(deterministicRandom()->random01() < 0.25) {
result += format(" usable_regions=%d", deterministicRandom()->randomInt(1,3));
}
}
@ -255,7 +255,7 @@ struct ConfigureDatabaseWorkload : TestWorkload {
}
static int randomRoleNumber() {
int i = g_random->randomInt(0,4);
int i = deterministicRandom()->randomInt(0,4);
return i ? i : -1;
}
@ -266,9 +266,9 @@ struct ConfigureDatabaseWorkload : TestWorkload {
if(g_simulator.speedUpSimulation) {
return Void();
}
state int randomChoice = g_random->randomInt(0, 7);
state int randomChoice = deterministicRandom()->randomInt(0, 7);
if( randomChoice == 0 ) {
double waitDuration = 3.0 * g_random->random01();
double waitDuration = 3.0 * deterministicRandom()->random01();
//TraceEvent("ConfigureTestWaitAfter").detail("WaitDuration",waitDuration);
wait( delay( waitDuration ) );
}
@ -285,7 +285,7 @@ struct ConfigureDatabaseWorkload : TestWorkload {
}
}
else if( randomChoice == 2 ) {
state double loadDuration = g_random->random01() * 10.0;
state double loadDuration = deterministicRandom()->random01() * 10.0;
state double startTime = now();
state int amtLoaded = 0;
@ -296,7 +296,7 @@ struct ConfigureDatabaseWorkload : TestWorkload {
tr = Transaction( cx );
try {
for( i = 0; i < 10; i++ ) {
state Key randomKey( "ConfigureTest" + g_random->randomUniqueID().toString() );
state Key randomKey( "ConfigureTest" + deterministicRandom()->randomUniqueID().toString() );
Optional<Value> val = wait( tr.get( randomKey ) );
uint64_t nextVal = val.present() ? valueToUInt64( val.get() ) + 1 : 0;
tr.set( randomKey, format( "%016llx", nextVal ) );
@ -321,7 +321,7 @@ struct ConfigureDatabaseWorkload : TestWorkload {
if(g_simulator.physicalDatacenters == 2 || g_simulator.physicalDatacenters > 3) {
maxRedundancies--; //There are not enough machines for triple replication in fearless configurations
}
int redundancy = g_random->randomInt(0, maxRedundancies);
int redundancy = deterministicRandom()->randomInt(0, maxRedundancies);
std::string config = redundancies[redundancy];
if(config == "triple" && g_simulator.physicalDatacenters == 3) {
@ -330,9 +330,9 @@ struct ConfigureDatabaseWorkload : TestWorkload {
config += generateRegions();
if (g_random->random01() < 0.5) config += " logs=" + format("%d", randomRoleNumber());
if (g_random->random01() < 0.5) config += " proxies=" + format("%d", randomRoleNumber());
if (g_random->random01() < 0.5) config += " resolvers=" + format("%d", randomRoleNumber());
if (deterministicRandom()->random01() < 0.5) config += " logs=" + format("%d", randomRoleNumber());
if (deterministicRandom()->random01() < 0.5) config += " proxies=" + format("%d", randomRoleNumber());
if (deterministicRandom()->random01() < 0.5) config += " resolvers=" + format("%d", randomRoleNumber());
wait(success( IssueConfigurationChange( cx, config, false ) ));
@ -341,17 +341,17 @@ struct ConfigureDatabaseWorkload : TestWorkload {
else if( randomChoice == 4 ) {
//TraceEvent("ConfigureTestQuorumBegin").detail("NewQuorum", s);
auto ch = autoQuorumChange();
if (g_random->randomInt(0,2))
ch = nameQuorumChange( format("NewName%d", g_random->randomInt(0,100)), ch );
if (deterministicRandom()->randomInt(0,2))
ch = nameQuorumChange( format("NewName%d", deterministicRandom()->randomInt(0,100)), ch );
wait(success( changeQuorum( cx, ch ) ));
//TraceEvent("ConfigureTestConfigureEnd").detail("NewQuorum", s);
}
else if ( randomChoice == 5) {
wait(success( IssueConfigurationChange( cx, storeTypes[g_random->randomInt( 0, sizeof(storeTypes)/sizeof(storeTypes[0]))], true ) ));
wait(success( IssueConfigurationChange( cx, storeTypes[deterministicRandom()->randomInt( 0, sizeof(storeTypes)/sizeof(storeTypes[0]))], true ) ));
}
else if ( randomChoice == 6 ) {
// Some configurations will be invalid, and that's fine.
wait(success( IssueConfigurationChange( cx, logTypes[g_random->randomInt( 0, sizeof(logTypes)/sizeof(logTypes[0]))], false ) ));
wait(success( IssueConfigurationChange( cx, logTypes[deterministicRandom()->randomInt( 0, sizeof(logTypes)/sizeof(logTypes[0]))], false ) ));
}
else {
ASSERT(false);

View File

@ -108,8 +108,8 @@ struct ConflictRangeWorkload : TestWorkload {
if( self->testReadYourWrites ) {
clearedSet.clear();
int clearedA = g_random->randomInt(0, self->maxKeySpace-1);
int clearedB = g_random->randomInt(0, self->maxKeySpace-1);
int clearedA = deterministicRandom()->randomInt(0, self->maxKeySpace-1);
int clearedB = deterministicRandom()->randomInt(0, self->maxKeySpace-1);
clearedBegin = std::min(clearedA, clearedB);
clearedEnd = std::max(clearedA, clearedB)+1;
TraceEvent("ConflictRangeClear").detail("Begin",clearedBegin).detail("End",clearedEnd);
@ -117,13 +117,13 @@ struct ConflictRangeWorkload : TestWorkload {
tr0.clear( KeyRangeRef( StringRef( format( "%010d", 0 ) ), StringRef( format( "%010d", self->maxKeySpace ) ) ) );
for(int i = 0; i < self->maxKeySpace; i++) {
if( g_random->random01() > 0.5) {
if( deterministicRandom()->random01() > 0.5) {
TraceEvent("ConflictRangeInit").detail("Key",i);
if( self->testReadYourWrites && i >= clearedBegin && i < clearedEnd )
clearedSet.insert( i );
else {
insertedSet.insert( i );
tr0.set(StringRef( format( "%010d", i ) ),g_random->randomUniqueID().toString());
tr0.set(StringRef( format( "%010d", i ) ),deterministicRandom()->randomUniqueID().toString());
}
}
}
@ -146,13 +146,13 @@ struct ConflictRangeWorkload : TestWorkload {
try {
//Generate a random getRange operation and execute it, if it produces results, save them, otherwise retry.
loop {
myKeyA = format( "%010d", g_random->randomInt( 0, self->maxKeySpace ) );
myKeyB = format( "%010d", g_random->randomInt( 0, self->maxKeySpace ) );
onEqualA = g_random->randomInt( 0, 2 ) != 0;
onEqualB = g_random->randomInt( 0, 2 ) != 0;
offsetA = g_random->randomInt( -1*self->maxOffset, self->maxOffset );
offsetB = g_random->randomInt( -1*self->maxOffset, self->maxOffset );
randomLimit = g_random->randomInt( 1, self->maxKeySpace );
myKeyA = format( "%010d", deterministicRandom()->randomInt( 0, self->maxKeySpace ) );
myKeyB = format( "%010d", deterministicRandom()->randomInt( 0, self->maxKeySpace ) );
onEqualA = deterministicRandom()->randomInt( 0, 2 ) != 0;
onEqualB = deterministicRandom()->randomInt( 0, 2 ) != 0;
offsetA = deterministicRandom()->randomInt( -1*self->maxOffset, self->maxOffset );
offsetB = deterministicRandom()->randomInt( -1*self->maxOffset, self->maxOffset );
randomLimit = deterministicRandom()->randomInt( 1, self->maxKeySpace );
Standalone<RangeResultRef> res = wait( tr1.getRange(KeySelectorRef(StringRef(myKeyA),onEqualA,offsetA),KeySelectorRef(StringRef(myKeyB),onEqualB,offsetB),randomLimit) );
if( res.size() ) {
@ -164,7 +164,7 @@ struct ConflictRangeWorkload : TestWorkload {
if( self->testReadYourWrites ) {
for( auto iter = clearedSet.begin(); iter != clearedSet.end(); ++iter )
tr1.set(StringRef( format( "%010d", (*iter) ) ),g_random->randomUniqueID().toString());
tr1.set(StringRef( format( "%010d", (*iter) ) ),deterministicRandom()->randomUniqueID().toString());
wait( tr1.commit() );
tr1 = Transaction(cx);
}
@ -181,22 +181,22 @@ struct ConflictRangeWorkload : TestWorkload {
//Do random operations in one of the transactions and commit.
//Either do all sets in locations without existing data or all clears in locations with data.
for (int i = 0;
i < g_random->randomInt(self->minOperationsPerTransaction, self->maxOperationsPerTransaction + 1);
i < deterministicRandom()->randomInt(self->minOperationsPerTransaction, self->maxOperationsPerTransaction + 1);
i++) {
if( randomSets ) {
for( int j = 0; j < 5; j++) {
int proposedKey = g_random->randomInt( 0, self->maxKeySpace );
int proposedKey = deterministicRandom()->randomInt( 0, self->maxKeySpace );
if( !insertedSet.count( proposedKey ) ) {
TraceEvent("ConflictRangeSet").detail("Key",proposedKey);
insertedSet.insert( proposedKey );
tr2.set(StringRef(format( "%010d", proposedKey )),g_random->randomUniqueID().toString());
tr2.set(StringRef(format( "%010d", proposedKey )),deterministicRandom()->randomUniqueID().toString());
break;
}
}
}
else {
for( int j = 0; j < 5; j++) {
int proposedKey = g_random->randomInt( 0, self->maxKeySpace );
int proposedKey = deterministicRandom()->randomInt( 0, self->maxKeySpace );
if( insertedSet.count( proposedKey ) ) {
TraceEvent("ConflictRangeClear").detail("Key",proposedKey);
insertedSet.erase( proposedKey );

View File

@ -96,7 +96,7 @@ struct CycleWorkload : TestWorkload {
wait( poisson( &lastTime, delay ) );
state double tstart = now();
state int r = g_random->randomInt(0, self->nodeCount);
state int r = deterministicRandom()->randomInt(0, self->nodeCount);
state Transaction tr(cx);
while (true) {
try {

View File

@ -50,7 +50,7 @@ struct DDBalanceWorkload : TestWorkload {
nodesPerActor = nodes/(actorsPerClient*clientCount);
currentbin = g_random->randomInt(0,binCount);
currentbin = deterministicRandom()->randomInt(0,binCount);
}
virtual std::string description() { return "DDBalance"; }
@ -130,7 +130,7 @@ struct DDBalanceWorkload : TestWorkload {
for(int o = 0; o <= self->nodesPerActor * self->actorsPerClient / 10; o++) order.push_back(o*10);
g_random->randomShuffle(order);
deterministicRandom()->randomShuffle(order);
for(i=0; i<order.size(); ) {
vector<Future<Void>> fs;
for(int j=0; j<100 && i<order.size(); j++) {
@ -219,8 +219,8 @@ struct DDBalanceWorkload : TestWorkload {
state double lastTime = now();
loop {
nextBin = g_random->randomInt(key_space_drift,self->binCount+key_space_drift);
while(nextBin == currentBin) nextBin = g_random->randomInt(key_space_drift,self->binCount+key_space_drift);
nextBin = deterministicRandom()->randomInt(key_space_drift,self->binCount+key_space_drift);
while(nextBin == currentBin) nextBin = deterministicRandom()->randomInt(key_space_drift,self->binCount+key_space_drift);
vector<Future<Void>> fs;
for (int i = 0; i < self->actorsPerClient / self->moversPerClient; i++)

View File

@ -42,7 +42,7 @@ struct DiskDurabilityWorkload : public AsyncFileWorkload
state int64_t newData;
if(self->lastData == 0)
newData = g_random->randomInt64(std::numeric_limits<int64_t>::min(), std::numeric_limits<int64_t>::max());
newData = deterministicRandom()->randomInt64(std::numeric_limits<int64_t>::min(), std::numeric_limits<int64_t>::max());
else {
++newData;
int readBytes = wait(file->file->read(buffer->buffer, size, offset));
@ -156,14 +156,14 @@ struct DiskDurabilityWorkload : public AsyncFileWorkload
state Reference<AsyncFileBuffer> buffer = Reference<AsyncFileBuffer>(new AsyncFileBuffer(_PAGE_SIZE, true));
state int logfp = (int)ceil(log2(self->filePages));
loop {
int block = intHash(std::min<int>(g_random->randomInt(0, 1 << g_random->randomInt(0, logfp)), self->filePages - 1)) % self->filePages;
int block = intHash(std::min<int>(deterministicRandom()->randomInt(0, 1 << deterministicRandom()->randomInt(0, logfp)), self->filePages - 1)) % self->filePages;
wait(self->blocks[block].test(self->fileHandle, self->pagesPerWrite, buffer));
}
}
ACTOR static Future<Void> syncLoop(DiskDurabilityWorkload *self) {
loop {
wait(delay(g_random->random01() * self->syncInterval));
wait(delay(deterministicRandom()->random01() * self->syncInterval));
wait(self->fileHandle->file->sync());
}
}

View File

@ -115,19 +115,19 @@ struct DiskDurabilityTest : TestWorkload {
state bool first = true;
loop {
state vector<int64_t> targetPages;
for(int i=g_random->randomInt(1, 100); i>0 && targetPages.size() < size/4096; i--) {
auto p = g_random->randomInt(0, size/4096);
for(int i=deterministicRandom()->randomInt(1, 100); i>0 && targetPages.size() < size/4096; i--) {
auto p = deterministicRandom()->randomInt(0, size/4096);
if (!std::count(targetPages.begin(), targetPages.end(), p))
targetPages.push_back( p );
}
for(int i=g_random->randomInt(1,4); i>0; i--) {
for(int i=deterministicRandom()->randomInt(1,4); i>0; i--) {
targetPages.push_back( size/4096 );
size += 4096;
}
state vector<int64_t> targetValues(targetPages.size());
for(auto& v : targetValues)
v = g_random->randomUniqueID().first();
v = deterministicRandom()->randomUniqueID().first();
tr.reset();
loop {

View File

@ -38,7 +38,7 @@ struct FastTriggeredWatchesWorkload : TestWorkload {
{
testDuration = getOption( options, LiteralStringRef("testDuration"), 600.0 );
nodes = getOption( options, LiteralStringRef("nodes"), 100 );
defaultValue = StringRef(format( "%010d", g_random->randomInt( 0, 1000 ) ));
defaultValue = StringRef(format( "%010d", deterministicRandom()->randomInt( 0, 1000 ) ));
keyBytes = std::max( getOption( options, LiteralStringRef("keyBytes"), 16 ), 16 );
}
@ -75,7 +75,7 @@ struct FastTriggeredWatchesWorkload : TestWorkload {
ACTOR Future<Version> setter( Database cx, Key key, Optional<Value> value ) {
state ReadYourWritesTransaction tr( cx );
wait( delay( g_random->random01() ) );
wait( delay( deterministicRandom()->random01() ) );
loop {
try {
if( value.present() )
@ -101,12 +101,12 @@ struct FastTriggeredWatchesWorkload : TestWorkload {
state double getDuration = 0;
state double watchEnd = 0;
state bool first = true;
state Key setKey = self->keyForIndex(g_random->randomInt(0,self->nodes));
state Key setKey = self->keyForIndex(deterministicRandom()->randomInt(0,self->nodes));
state Optional<Value> setValue;
if( g_random->random01() > 0.5 )
setValue = StringRef(format( "%010d", g_random->randomInt( 0, 1000 )));
if( deterministicRandom()->random01() > 0.5 )
setValue = StringRef(format( "%010d", deterministicRandom()->randomInt( 0, 1000 )));
state Future<Version> setFuture = self->setter( cx, setKey, setValue );
wait( delay( g_random->random01() ) );
wait( delay( deterministicRandom()->random01() ) );
loop {
state ReadYourWritesTransaction tr( cx );

Some files were not shown because too many files have changed in this diff Show More