Extract ProcessInfo, MachineInfo, KillType out from ISimulator

This commit is contained in:
Xiaoge Su 2022-12-31 15:22:28 -08:00
parent 50de69c897
commit 0a60142160
33 changed files with 635 additions and 516 deletions

View File

@ -24,6 +24,7 @@
#include "flow/Platform.actor.h"
#include "flow/Platform.h"
#include "fdbrpc/simulator.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "flow/actorcompiler.h" // This must be the last #include.
namespace {

View File

@ -43,6 +43,7 @@
#include "fdbrpc/ReplicationPolicy.h"
#include "fdbrpc/Replication.h"
#include "fdbclient/Schemas.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "flow/actorcompiler.h" // This must be the last #include.

View File

@ -19,10 +19,19 @@
*/
#include "fdbrpc/AsyncFileNonDurable.actor.h"
#include "fdbrpc/SimulatorMachineInfo.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "flow/actorcompiler.h" // has to be last include
std::map<std::string, Future<Void>> AsyncFileNonDurable::filesBeingDeleted;
Future<Void> waitShutdownSignal() {
return success(g_simulator->getCurrentProcess()->shutdownSignal.getFuture());
}
ACTOR Future<Void> sendOnProcess(ISimulator::ProcessInfo* process, Promise<Void> promise, TaskPriority taskID) {
wait(g_simulator->onProcess(process, taskID));
promise.send(Void());
@ -37,3 +46,180 @@ ACTOR Future<Void> sendErrorOnProcess(ISimulator::ProcessInfo* process,
promise.sendError(e);
return Void();
}
ACTOR Future<Void> AsyncFileDetachable::doShutdown(AsyncFileDetachable* self) {
wait(success(g_simulator->getCurrentProcess()->shutdownSignal.getFuture()));
self->file = Reference<IAsyncFile>();
return Void();
}
ACTOR Future<Reference<IAsyncFile>> AsyncFileDetachable::open(Future<Reference<IAsyncFile>> wrappedFile) {
choose {
when(wait(success(g_simulator->getCurrentProcess()->shutdownSignal.getFuture()))) {
throw io_error().asInjectedFault();
}
when(Reference<IAsyncFile> f = wait(wrappedFile)) { return makeReference<AsyncFileDetachable>(f); }
}
}
Future<int> AsyncFileDetachable::read(void* data, int length, int64_t offset) {
if (!file.getPtr() || g_simulator->getCurrentProcess()->shutdownSignal.getFuture().isReady())
return io_error().asInjectedFault();
return sendErrorOnShutdown(file->read(data, length, offset), assertOnReadWriteCancel);
}
Future<Void> AsyncFileDetachable::write(void const* data, int length, int64_t offset) {
if (!file.getPtr() || g_simulator->getCurrentProcess()->shutdownSignal.getFuture().isReady())
return io_error().asInjectedFault();
return sendErrorOnShutdown(file->write(data, length, offset), assertOnReadWriteCancel);
}
Future<Void> AsyncFileDetachable::truncate(int64_t size) {
if (!file.getPtr() || g_simulator->getCurrentProcess()->shutdownSignal.getFuture().isReady())
return io_error().asInjectedFault();
return sendErrorOnShutdown(file->truncate(size));
}
Future<Void> AsyncFileDetachable::sync() {
if (!file.getPtr() || g_simulator->getCurrentProcess()->shutdownSignal.getFuture().isReady())
return io_error().asInjectedFault();
return sendErrorOnShutdown(file->sync());
}
Future<int64_t> AsyncFileDetachable::size() const {
if (!file.getPtr() || g_simulator->getCurrentProcess()->shutdownSignal.getFuture().isReady())
return io_error().asInjectedFault();
return sendErrorOnShutdown(file->size());
}
ACTOR Future<Reference<IAsyncFile>> AsyncFileNonDurable::open(std::string filename,
std::string actualFilename,
Future<Reference<IAsyncFile>> wrappedFile,
Reference<DiskParameters> diskParameters,
bool aio) {
state ISimulator::ProcessInfo* currentProcess = g_simulator->getCurrentProcess();
state TaskPriority currentTaskID = g_network->getCurrentTask();
state Future<Void> shutdown = success(currentProcess->shutdownSignal.getFuture());
//TraceEvent("AsyncFileNonDurableOpenBegin").detail("Filename", filename).detail("Addr", g_simulator->getCurrentProcess()->address);
wait(g_simulator->onMachine(currentProcess));
try {
wait(success(wrappedFile) || shutdown);
if (shutdown.isReady())
throw io_error().asInjectedFault();
state Reference<IAsyncFile> file = wrappedFile.get();
// If we are in the process of deleting a file, we can't let someone else modify it at the same time. We
// therefore block the creation of new files until deletion is complete
state std::map<std::string, Future<Void>>::iterator deletedFile = AsyncFileNonDurable::filesBeingDeleted.find(filename);
if (deletedFile != AsyncFileNonDurable::filesBeingDeleted.end()) {
//TraceEvent("AsyncFileNonDurableOpenWaitOnDelete1").detail("Filename", filename);
wait(deletedFile->second || shutdown);
//TraceEvent("AsyncFileNonDurableOpenWaitOnDelete2").detail("Filename", filename);
if (shutdown.isReady())
throw io_error().asInjectedFault();
wait(g_simulator->onProcess(currentProcess, currentTaskID));
}
state Reference<AsyncFileNonDurable> nonDurableFile(
new AsyncFileNonDurable(filename, actualFilename, file, diskParameters, currentProcess->address, aio));
// Causes the approximateSize member to be set
state Future<int64_t> sizeFuture = nonDurableFile->size();
wait(success(sizeFuture) || shutdown);
if (shutdown.isReady())
throw io_error().asInjectedFault();
//TraceEvent("AsyncFileNonDurableOpenComplete").detail("Filename", filename);
wait(g_simulator->onProcess(currentProcess, currentTaskID));
return nonDurableFile;
} catch (Error& e) {
state Error err = e;
std::string currentFilename =
(wrappedFile.isReady() && !wrappedFile.isError()) ? wrappedFile.get()->getFilename() : actualFilename;
currentProcess->machine->openFiles.erase(currentFilename);
//TraceEvent("AsyncFileNonDurableOpenError").errorUnsuppressed(e).detail("Filename", filename).detail("Address", currentProcess->address).detail("Addr", g_simulator->getCurrentProcess()->address);
wait(g_simulator->onProcess(currentProcess, currentTaskID));
throw err;
}
}
ACTOR Future<int> AsyncFileNonDurable::read(AsyncFileNonDurable* self, void* data, int length, int64_t offset) {
state ISimulator::ProcessInfo* currentProcess = g_simulator->getCurrentProcess();
state TaskPriority currentTaskID = g_network->getCurrentTask();
wait(g_simulator->onMachine(currentProcess));
try {
state int rep = wait(self->onRead(self, data, length, offset));
wait(g_simulator->onProcess(currentProcess, currentTaskID));
return rep;
} catch (Error& e) {
state Error err = e;
wait(g_simulator->onProcess(currentProcess, currentTaskID));
throw err;
}
}
ACTOR Future<Void> AsyncFileNonDurable::closeFile(AsyncFileNonDurable* self) {
state ISimulator::ProcessInfo* currentProcess = g_simulator->getCurrentProcess();
state TaskPriority currentTaskID = g_network->getCurrentTask();
state std::string filename = self->filename;
g_simulator->getMachineByNetworkAddress(self->openedAddress)->deletingOrClosingFiles.insert(self->getFilename());
wait(g_simulator->onMachine(currentProcess));
try {
// Make sure all writes have gone through.
Promise<bool> startSyncPromise = self->startSyncPromise;
self->startSyncPromise = Promise<bool>();
startSyncPromise.send(true);
std::vector<Future<Void>> outstandingModifications;
for (auto itr = self->pendingModifications.ranges().begin(); itr != self->pendingModifications.ranges().end();
++itr)
if (itr->value().isValid() && !itr->value().isReady())
outstandingModifications.push_back(itr->value());
// Ignore errors here so that all modifications can finish
wait(waitForAllReady(outstandingModifications));
// Make sure we aren't in the process of killing the file
if (self->killed.isSet())
wait(self->killComplete.getFuture());
// Remove this file from the filesBeingDeleted map so that new files can be created with this filename
g_simulator->getMachineByNetworkAddress(self->openedAddress)->closingFiles.erase(self->getFilename());
g_simulator->getMachineByNetworkAddress(self->openedAddress)->deletingOrClosingFiles.erase(self->getFilename());
AsyncFileNonDurable::filesBeingDeleted.erase(self->filename);
//TraceEvent("AsyncFileNonDurable_FinishDelete", self->id).detail("Filename", self->filename);
delete self;
return Void();
} catch (Error& e) {
state Error err = e;
throw err;
}
}
void AsyncFileNonDurable::removeOpenFile(std::string filename, AsyncFileNonDurable* file) {
auto& openFiles = g_simulator->getCurrentProcess()->machine->openFiles;
auto iter = openFiles.find(filename);
// Various actions (e.g. simulated delete) can remove a file from openFiles prematurely, so it may already
// be gone. Renamed files (from atomic write and create) will also be present under only one of the two
// names.
if (iter != openFiles.end()) {
// even if the filename exists, it doesn't mean that it references the same file. It could be that the
// file was renamed and later a file with the same name was opened.
if (iter->second.getPtrIfReady().orDefault(nullptr) == file) {
openFiles.erase(iter);
}
}
}

View File

@ -45,16 +45,14 @@ ACTOR Future<Void> sendErrorOnProcess(ISimulator::ProcessInfo* process,
Error e,
TaskPriority taskID);
extern Future<Void> waitShutdownSignal();
ACTOR template <class T>
Future<T> sendErrorOnShutdown(Future<T> in, bool assertOnCancel = false) {
try {
choose {
when(wait(success(g_simulator->getCurrentProcess()->shutdownSignal.getFuture()))) {
throw io_error().asInjectedFault();
}
when(T rep = wait(in)) {
return rep;
}
when(wait(waitShutdownSignal())) { throw io_error().asInjectedFault(); }
when(T rep = wait(in)) { return rep; }
}
} catch (Error& e) {
ASSERT(e.code() != error_code_actor_cancelled || !assertOnCancel);
@ -73,55 +71,17 @@ public:
shutdown = doShutdown(this);
}
ACTOR Future<Void> doShutdown(AsyncFileDetachable* self) {
wait(success(g_simulator->getCurrentProcess()->shutdownSignal.getFuture()));
self->file = Reference<IAsyncFile>();
return Void();
}
ACTOR static Future<Reference<IAsyncFile>> open(Future<Reference<IAsyncFile>> wrappedFile) {
choose {
when(wait(success(g_simulator->getCurrentProcess()->shutdownSignal.getFuture()))) {
throw io_error().asInjectedFault();
}
when(Reference<IAsyncFile> f = wait(wrappedFile)) {
return makeReference<AsyncFileDetachable>(f);
}
}
}
ACTOR Future<Void> doShutdown(AsyncFileDetachable* self);
ACTOR static Future<Reference<IAsyncFile>> open(Future<Reference<IAsyncFile>> wrappedFile);
void addref() override { ReferenceCounted<AsyncFileDetachable>::addref(); }
void delref() override { ReferenceCounted<AsyncFileDetachable>::delref(); }
Future<int> read(void* data, int length, int64_t offset) override {
if (!file.getPtr() || g_simulator->getCurrentProcess()->shutdownSignal.getFuture().isReady())
return io_error().asInjectedFault();
return sendErrorOnShutdown(file->read(data, length, offset), assertOnReadWriteCancel);
}
Future<Void> write(void const* data, int length, int64_t offset) override {
if (!file.getPtr() || g_simulator->getCurrentProcess()->shutdownSignal.getFuture().isReady())
return io_error().asInjectedFault();
return sendErrorOnShutdown(file->write(data, length, offset), assertOnReadWriteCancel);
}
Future<Void> truncate(int64_t size) override {
if (!file.getPtr() || g_simulator->getCurrentProcess()->shutdownSignal.getFuture().isReady())
return io_error().asInjectedFault();
return sendErrorOnShutdown(file->truncate(size));
}
Future<Void> sync() override {
if (!file.getPtr() || g_simulator->getCurrentProcess()->shutdownSignal.getFuture().isReady())
return io_error().asInjectedFault();
return sendErrorOnShutdown(file->sync());
}
Future<int64_t> size() const override {
if (!file.getPtr() || g_simulator->getCurrentProcess()->shutdownSignal.getFuture().isReady())
return io_error().asInjectedFault();
return sendErrorOnShutdown(file->size());
}
Future<int> read(void* data, int length, int64_t offset) override;
Future<Void> write(void const* data, int length, int64_t offset) override;
Future<Void> truncate(int64_t size) override;
Future<Void> sync() override;
Future<int64_t> size() const override;
int64_t debugFD() const override {
if (!file.getPtr())
@ -225,58 +185,7 @@ public:
std::string actualFilename,
Future<Reference<IAsyncFile>> wrappedFile,
Reference<DiskParameters> diskParameters,
bool aio) {
state ISimulator::ProcessInfo* currentProcess = g_simulator->getCurrentProcess();
state TaskPriority currentTaskID = g_network->getCurrentTask();
state Future<Void> shutdown = success(currentProcess->shutdownSignal.getFuture());
//TraceEvent("AsyncFileNonDurableOpenBegin").detail("Filename", filename).detail("Addr", g_simulator->getCurrentProcess()->address);
wait(g_simulator->onMachine(currentProcess));
try {
wait(success(wrappedFile) || shutdown);
if (shutdown.isReady())
throw io_error().asInjectedFault();
state Reference<IAsyncFile> file = wrappedFile.get();
// If we are in the process of deleting a file, we can't let someone else modify it at the same time. We
// therefore block the creation of new files until deletion is complete
state std::map<std::string, Future<Void>>::iterator deletedFile = filesBeingDeleted.find(filename);
if (deletedFile != filesBeingDeleted.end()) {
//TraceEvent("AsyncFileNonDurableOpenWaitOnDelete1").detail("Filename", filename);
wait(deletedFile->second || shutdown);
//TraceEvent("AsyncFileNonDurableOpenWaitOnDelete2").detail("Filename", filename);
if (shutdown.isReady())
throw io_error().asInjectedFault();
wait(g_simulator->onProcess(currentProcess, currentTaskID));
}
state Reference<AsyncFileNonDurable> nonDurableFile(
new AsyncFileNonDurable(filename, actualFilename, file, diskParameters, currentProcess->address, aio));
// Causes the approximateSize member to be set
state Future<int64_t> sizeFuture = nonDurableFile->size();
wait(success(sizeFuture) || shutdown);
if (shutdown.isReady())
throw io_error().asInjectedFault();
//TraceEvent("AsyncFileNonDurableOpenComplete").detail("Filename", filename);
wait(g_simulator->onProcess(currentProcess, currentTaskID));
return nonDurableFile;
} catch (Error& e) {
state Error err = e;
std::string currentFilename =
(wrappedFile.isReady() && !wrappedFile.isError()) ? wrappedFile.get()->getFilename() : actualFilename;
currentProcess->machine->openFiles.erase(currentFilename);
//TraceEvent("AsyncFileNonDurableOpenError").errorUnsuppressed(e).detail("Filename", filename).detail("Address", currentProcess->address).detail("Addr", g_simulator->getCurrentProcess()->address);
wait(g_simulator->onProcess(currentProcess, currentTaskID));
throw err;
}
}
bool aio);
~AsyncFileNonDurable() override {
//TraceEvent("AsyncFileNonDurable_Destroy", id).detail("Filename", filename);
@ -301,22 +210,7 @@ public:
}
// Removes a file from the openFiles map
static void removeOpenFile(std::string filename, AsyncFileNonDurable* file) {
auto& openFiles = g_simulator->getCurrentProcess()->machine->openFiles;
auto iter = openFiles.find(filename);
// Various actions (e.g. simulated delete) can remove a file from openFiles prematurely, so it may already
// be gone. Renamed files (from atomic write and create) will also be present under only one of the two
// names.
if (iter != openFiles.end()) {
// even if the filename exists, it doesn't mean that it references the same file. It could be that the
// file was renamed and later a file with the same name was opened.
if (iter->second.getPtrIfReady().orDefault(nullptr) == file) {
openFiles.erase(iter);
}
}
}
static void removeOpenFile(std::string filename, AsyncFileNonDurable* file);
// Passes along reads straight to the underlying file, waiting for any outstanding changes that could affect the
// results
@ -443,21 +337,7 @@ private:
return readFuture.get();
}
ACTOR Future<int> read(AsyncFileNonDurable* self, void* data, int length, int64_t offset) {
state ISimulator::ProcessInfo* currentProcess = g_simulator->getCurrentProcess();
state TaskPriority currentTaskID = g_network->getCurrentTask();
wait(g_simulator->onMachine(currentProcess));
try {
state int rep = wait(self->onRead(self, data, length, offset));
wait(g_simulator->onProcess(currentProcess, currentTaskID));
return rep;
} catch (Error& e) {
state Error err = e;
wait(g_simulator->onProcess(currentProcess, currentTaskID));
throw err;
}
}
ACTOR Future<int> read(AsyncFileNonDurable* self, void* data, int length, int64_t offset);
// Delays writes a random amount of time before passing them through to the underlying file.
// If a kill interrupts the delay, then the output could be the correct write, part of the write,
@ -511,9 +391,7 @@ private:
state bool saveDurable = true;
choose {
when(wait(delay(delayDuration))) {}
when(bool durable = wait(startSyncFuture)) {
saveDurable = durable;
}
when(bool durable = wait(startSyncFuture)) { saveDurable = durable; }
}
debugFileCheck("AsyncFileNonDurableWriteAfterWait", self->filename, dataCopy.begin(), offset, length);
@ -690,9 +568,7 @@ private:
state bool saveDurable = true;
choose {
when(wait(delay(delayDuration))) {}
when(bool durable = wait(startSyncFuture)) {
saveDurable = durable;
}
when(bool durable = wait(startSyncFuture)) { saveDurable = durable; }
}
if (g_network->check_yield(TaskPriority::DefaultYield)) {
@ -840,50 +716,7 @@ private:
}
// Finishes all outstanding actors on an AsyncFileNonDurable and then deletes it
ACTOR Future<Void> closeFile(AsyncFileNonDurable* self) {
state ISimulator::ProcessInfo* currentProcess = g_simulator->getCurrentProcess();
state TaskPriority currentTaskID = g_network->getCurrentTask();
state std::string filename = self->filename;
g_simulator->getMachineByNetworkAddress(self->openedAddress)
->deletingOrClosingFiles.insert(self->getFilename());
wait(g_simulator->onMachine(currentProcess));
try {
// Make sure all writes have gone through.
Promise<bool> startSyncPromise = self->startSyncPromise;
self->startSyncPromise = Promise<bool>();
startSyncPromise.send(true);
std::vector<Future<Void>> outstandingModifications;
for (auto itr = self->pendingModifications.ranges().begin();
itr != self->pendingModifications.ranges().end();
++itr)
if (itr->value().isValid() && !itr->value().isReady())
outstandingModifications.push_back(itr->value());
// Ignore errors here so that all modifications can finish
wait(waitForAllReady(outstandingModifications));
// Make sure we aren't in the process of killing the file
if (self->killed.isSet())
wait(self->killComplete.getFuture());
// Remove this file from the filesBeingDeleted map so that new files can be created with this filename
g_simulator->getMachineByNetworkAddress(self->openedAddress)->closingFiles.erase(self->getFilename());
g_simulator->getMachineByNetworkAddress(self->openedAddress)
->deletingOrClosingFiles.erase(self->getFilename());
AsyncFileNonDurable::filesBeingDeleted.erase(self->filename);
//TraceEvent("AsyncFileNonDurable_FinishDelete", self->id).detail("Filename", self->filename);
delete self;
return Void();
} catch (Error& e) {
state Error err = e;
throw err;
}
}
ACTOR Future<Void> closeFile(AsyncFileNonDurable* self);
};
#include "flow/unactorcompiler.h"

View File

@ -0,0 +1,21 @@
#ifndef FDBRPC_SIMULATOR_KILLTYPE_H
#define FDBRPC_SIMULATOR_KILLTYPE_H
namespace simulator {
// Order matters!
enum KillType {
KillInstantly,
InjectFaults,
FailDisk,
RebootAndDelete,
RebootProcessAndDelete,
RebootProcessAndSwitch, // Reboot and switch cluster file
Reboot,
RebootProcess,
None
};
} // namespace simulator
#endif // FDBRPC_SIMULATOR_KILLTYPE_H

View File

@ -0,0 +1,55 @@
#ifndef FDBRPC_SIMULATORMACHINEINFO_H
#define FDBRPC_SIMULATORMACHINEINFO_H
#include <map>
#include <set>
#include <string>
#include <vector>
#include "flow/Optional.h"
namespace simulator {
struct ProcessInfo;
// A set of data associated with a simulated machine
struct MachineInfo {
ProcessInfo* machineProcess;
std::vector<ProcessInfo*> processes;
// A map from filename to file handle for all open files on a machine
std::map<std::string, UnsafeWeakFutureReference<IAsyncFile>> openFiles;
std::set<std::string> deletingOrClosingFiles;
std::set<std::string> closingFiles;
Optional<Standalone<StringRef>> machineId;
const uint16_t remotePortStart;
std::vector<uint16_t> usedRemotePorts;
MachineInfo() : machineProcess(nullptr), remotePortStart(1000) {}
short getRandomPort() {
for (uint16_t i = remotePortStart; i < 60000; i++) {
if (std::find(usedRemotePorts.begin(), usedRemotePorts.end(), i) == usedRemotePorts.end()) {
TraceEvent(SevDebug, "RandomPortOpened").detail("PortNum", i);
usedRemotePorts.push_back(i);
return i;
}
}
UNREACHABLE();
}
void removeRemotePort(uint16_t port) {
if (port < remotePortStart)
return;
auto pos = std::find(usedRemotePorts.begin(), usedRemotePorts.end(), port);
if (pos != usedRemotePorts.end()) {
usedRemotePorts.erase(pos);
}
}
};
} // namespace simulator
#endif // FDBRPC_SIMULATORMACHINEINFO_H

View File

@ -0,0 +1,164 @@
#ifndef FDBRPC_SIMULATOR_PROCESSINFO_H
#define FDBRPC_SIMULATOR_PROCESSINFO_H
#include <map>
#include <string>
#include "flow/NetworkAddress.h"
#include "flow/IConnection.h"
#include "flow/IUDPSocket.h"
#include "fdbrpc/SimulatorMachineInfo.h"
#include "fdbrpc/SimulatorKillType.h"
struct MachineInfo;
namespace simulator {
struct ProcessInfo : NonCopyable {
std::string name;
std::string coordinationFolder;
std::string dataFolder;
MachineInfo* machine;
NetworkAddressList addresses;
NetworkAddress address;
LocalityData locality;
ProcessClass startingClass;
TDMetricCollection tdmetrics;
MetricCollection metrics;
ChaosMetrics chaosMetrics;
HistogramRegistry histograms;
std::map<NetworkAddress, Reference<IListener>> listenerMap;
std::map<NetworkAddress, Reference<IUDPSocket>> boundUDPSockets;
bool failed;
bool excluded;
bool cleared;
bool rebooting;
bool drProcess;
std::vector<flowGlobalType> globals;
INetworkConnections* network;
uint64_t fault_injection_r;
double fault_injection_p1, fault_injection_p2;
bool failedDisk;
UID uid;
ProtocolVersion protocolVersion;
bool excludeFromRestarts = false;
std::vector<ProcessInfo*> childs;
ProcessInfo(const char* name,
LocalityData locality,
ProcessClass startingClass,
NetworkAddressList addresses,
INetworkConnections* net,
const char* dataFolder,
const char* coordinationFolder)
: name(name), coordinationFolder(coordinationFolder), dataFolder(dataFolder), machine(nullptr),
addresses(addresses), address(addresses.address), locality(locality), startingClass(startingClass),
failed(false), excluded(false), cleared(false), rebooting(false), drProcess(false), network(net),
fault_injection_r(0), fault_injection_p1(0), fault_injection_p2(0), failedDisk(false) {
uid = deterministicRandom()->randomUniqueID();
}
Future<KillType> onShutdown() { return shutdownSignal.getFuture(); }
bool isSpawnedKVProcess() const {
// SOMEDAY: use a separate bool may be better?
return name == "remote flow process";
}
bool isReliable() const {
return !failed && fault_injection_p1 == 0 && fault_injection_p2 == 0 && !failedDisk &&
(!machine ||
(machine->machineProcess->fault_injection_p1 == 0 && machine->machineProcess->fault_injection_p2 == 0));
}
bool isAvailable() const { return !isExcluded() && isReliable(); }
bool isExcluded() const { return excluded; }
bool isCleared() const { return cleared; }
std::string getReliableInfo() const {
std::stringstream ss;
ss << "failed:" << failed << " fault_injection_p1:" << fault_injection_p1
<< " fault_injection_p2:" << fault_injection_p2;
return ss.str();
}
std::vector<ProcessInfo*> const& getChilds() const { return childs; }
// Return true if the class type is suitable for stateful roles, such as tLog and StorageServer.
bool isAvailableClass() const {
switch (startingClass._class) {
case ProcessClass::UnsetClass:
return true;
case ProcessClass::StorageClass:
return true;
case ProcessClass::TransactionClass:
return true;
case ProcessClass::ResolutionClass:
return false;
case ProcessClass::CommitProxyClass:
return false;
case ProcessClass::GrvProxyClass:
return false;
case ProcessClass::MasterClass:
return false;
case ProcessClass::TesterClass:
return false;
case ProcessClass::StatelessClass:
return false;
case ProcessClass::LogClass:
return true;
case ProcessClass::LogRouterClass:
return false;
case ProcessClass::ClusterControllerClass:
return false;
case ProcessClass::DataDistributorClass:
return false;
case ProcessClass::RatekeeperClass:
return false;
case ProcessClass::ConsistencyScanClass:
return false;
case ProcessClass::BlobManagerClass:
return false;
case ProcessClass::StorageCacheClass:
return false;
case ProcessClass::BackupClass:
return false;
case ProcessClass::EncryptKeyProxyClass:
return false;
default:
return false;
}
}
Reference<IListener> getListener(const NetworkAddress& addr) const {
auto listener = listenerMap.find(addr);
ASSERT(listener != listenerMap.end());
return listener->second;
}
inline flowGlobalType global(int id) const { return (globals.size() > id) ? globals[id] : nullptr; };
inline void setGlobal(size_t id, flowGlobalType v) {
globals.resize(std::max(globals.size(), id + 1));
globals[id] = v;
};
std::string toString() const {
return format("name: %s address: %s zone: %s datahall: %s class: %s excluded: %d cleared: %d",
name.c_str(),
formatIpPort(addresses.address.ip, addresses.address.port).c_str(),
(locality.zoneId().present() ? locality.zoneId().get().printable().c_str() : "[unset]"),
(locality.dataHallId().present() ? locality.dataHallId().get().printable().c_str() : "[unset]"),
startingClass.toString().c_str(),
excluded,
cleared);
}
// Members not for external use
Promise<KillType> shutdownSignal;
};
} // namespace simulator
#endif // FDBRPC_SIMULATOR_PROCESSINFO_H

View File

@ -42,29 +42,26 @@
#include "fdbrpc/TokenSign.h"
#include "flow/IUDPSocket.h"
#include "flow/IConnection.h"
#include "fdbrpc/SimulatorKillType.h"
enum ClogMode { ClogDefault, ClogAll, ClogSend, ClogReceive };
struct ValidationData {
// global validation that missing refreshed feeds were previously destroyed
std::unordered_set<std::string> allDestroyedChangeFeedIDs;
};
namespace simulator {
struct ProcessInfo;
struct MachineInfo;
} // namespace simulator
class ISimulator : public INetwork {
public:
// Order matters!
enum KillType {
KillInstantly,
InjectFaults,
FailDisk,
RebootAndDelete,
RebootProcessAndDelete,
RebootProcessAndSwitch, // Reboot and switch cluster file
Reboot,
RebootProcess,
None
};
using KillType = simulator::KillType;
using ProcessInfo = simulator::ProcessInfo;
using MachineInfo = simulator::MachineInfo;
// Order matters! all modes >= 2 are fault injection modes
enum TSSMode { Disabled, EnabledNormal, EnabledAddDelay, EnabledDropMutations };
@ -90,192 +87,6 @@ public:
}
};
// Subclasses may subclass ProcessInfo as well
struct MachineInfo;
struct ProcessInfo : NonCopyable {
std::string name;
std::string coordinationFolder;
std::string dataFolder;
MachineInfo* machine;
NetworkAddressList addresses;
NetworkAddress address;
LocalityData locality;
ProcessClass startingClass;
TDMetricCollection tdmetrics;
MetricCollection metrics;
ChaosMetrics chaosMetrics;
HistogramRegistry histograms;
std::map<NetworkAddress, Reference<IListener>> listenerMap;
std::map<NetworkAddress, Reference<IUDPSocket>> boundUDPSockets;
bool failed;
bool excluded;
bool cleared;
bool rebooting;
bool drProcess;
std::vector<flowGlobalType> globals;
INetworkConnections* network;
uint64_t fault_injection_r;
double fault_injection_p1, fault_injection_p2;
bool failedDisk;
UID uid;
ProtocolVersion protocolVersion;
bool excludeFromRestarts = false;
std::vector<ProcessInfo*> childs;
ProcessInfo(const char* name,
LocalityData locality,
ProcessClass startingClass,
NetworkAddressList addresses,
INetworkConnections* net,
const char* dataFolder,
const char* coordinationFolder)
: name(name), coordinationFolder(coordinationFolder), dataFolder(dataFolder), machine(nullptr),
addresses(addresses), address(addresses.address), locality(locality), startingClass(startingClass),
failed(false), excluded(false), cleared(false), rebooting(false), drProcess(false), network(net),
fault_injection_r(0), fault_injection_p1(0), fault_injection_p2(0), failedDisk(false) {
uid = deterministicRandom()->randomUniqueID();
}
Future<KillType> onShutdown() { return shutdownSignal.getFuture(); }
bool isSpawnedKVProcess() const {
// SOMEDAY: use a separate bool may be better?
return name == "remote flow process";
}
bool isReliable() const {
return !failed && fault_injection_p1 == 0 && fault_injection_p2 == 0 && !failedDisk &&
(!machine || (machine->machineProcess->fault_injection_p1 == 0 &&
machine->machineProcess->fault_injection_p2 == 0));
}
bool isAvailable() const { return !isExcluded() && isReliable(); }
bool isExcluded() const { return excluded; }
bool isCleared() const { return cleared; }
std::string getReliableInfo() const {
std::stringstream ss;
ss << "failed:" << failed << " fault_injection_p1:" << fault_injection_p1
<< " fault_injection_p2:" << fault_injection_p2;
return ss.str();
}
std::vector<ProcessInfo*> const& getChilds() const { return childs; }
// Return true if the class type is suitable for stateful roles, such as tLog and StorageServer.
bool isAvailableClass() const {
switch (startingClass._class) {
case ProcessClass::UnsetClass:
return true;
case ProcessClass::StorageClass:
return true;
case ProcessClass::TransactionClass:
return true;
case ProcessClass::ResolutionClass:
return false;
case ProcessClass::CommitProxyClass:
return false;
case ProcessClass::GrvProxyClass:
return false;
case ProcessClass::MasterClass:
return false;
case ProcessClass::TesterClass:
return false;
case ProcessClass::StatelessClass:
return false;
case ProcessClass::LogClass:
return true;
case ProcessClass::LogRouterClass:
return false;
case ProcessClass::ClusterControllerClass:
return false;
case ProcessClass::DataDistributorClass:
return false;
case ProcessClass::RatekeeperClass:
return false;
case ProcessClass::ConsistencyScanClass:
return false;
case ProcessClass::BlobManagerClass:
return false;
case ProcessClass::StorageCacheClass:
return false;
case ProcessClass::BackupClass:
return false;
case ProcessClass::EncryptKeyProxyClass:
return false;
default:
return false;
}
}
Reference<IListener> getListener(const NetworkAddress& addr) const {
auto listener = listenerMap.find(addr);
ASSERT(listener != listenerMap.end());
return listener->second;
}
inline flowGlobalType global(int id) const { return (globals.size() > id) ? globals[id] : nullptr; };
inline void setGlobal(size_t id, flowGlobalType v) {
globals.resize(std::max(globals.size(), id + 1));
globals[id] = v;
};
std::string toString() const {
return format(
"name: %s address: %s zone: %s datahall: %s class: %s excluded: %d cleared: %d",
name.c_str(),
formatIpPort(addresses.address.ip, addresses.address.port).c_str(),
(locality.zoneId().present() ? locality.zoneId().get().printable().c_str() : "[unset]"),
(locality.dataHallId().present() ? locality.dataHallId().get().printable().c_str() : "[unset]"),
startingClass.toString().c_str(),
excluded,
cleared);
}
// Members not for external use
Promise<KillType> shutdownSignal;
};
// A set of data associated with a simulated machine
struct MachineInfo {
ProcessInfo* machineProcess;
std::vector<ProcessInfo*> processes;
// A map from filename to file handle for all open files on a machine
std::map<std::string, UnsafeWeakFutureReference<IAsyncFile>> openFiles;
std::set<std::string> deletingOrClosingFiles;
std::set<std::string> closingFiles;
Optional<Standalone<StringRef>> machineId;
const uint16_t remotePortStart;
std::vector<uint16_t> usedRemotePorts;
MachineInfo() : machineProcess(nullptr), remotePortStart(1000) {}
short getRandomPort() {
for (uint16_t i = remotePortStart; i < 60000; i++) {
if (std::find(usedRemotePorts.begin(), usedRemotePorts.end(), i) == usedRemotePorts.end()) {
TraceEvent(SevDebug, "RandomPortOpened").detail("PortNum", i);
usedRemotePorts.push_back(i);
return i;
}
}
UNREACHABLE();
}
void removeRemotePort(uint16_t port) {
if (port < remotePortStart)
return;
auto pos = std::find(usedRemotePorts.begin(), usedRemotePorts.end(), port);
if (pos != usedRemotePorts.end()) {
usedRemotePorts.erase(pos);
}
}
};
ProcessInfo* getProcess(Endpoint const& endpoint) { return getProcessByAddress(endpoint.getPrimaryAddress()); }
ProcessInfo* getCurrentProcess() { return currentProcess; }
ProcessInfo const* getCurrentProcess() const { return currentProcess; }
@ -529,30 +340,19 @@ public:
std::set<std::pair<std::string, unsigned>> corruptedBlocks;
flowGlobalType global(int id) const final { return getCurrentProcess()->global(id); };
void setGlobal(size_t id, flowGlobalType v) final { getCurrentProcess()->setGlobal(id, v); };
flowGlobalType global(int id) const final;
void setGlobal(size_t id, flowGlobalType v) final;
void disableFor(const std::string& desc, double time) { disabledMap[desc] = time; }
void disableFor(const std::string& desc, double time);
double checkDisabled(const std::string& desc) const {
auto iter = disabledMap.find(desc);
if (iter != disabledMap.end()) {
return iter->second;
}
return 0;
}
double checkDisabled(const std::string& desc) const;
// generate authz token for use in simulation environment
Standalone<StringRef> makeToken(StringRef tenantName, uint64_t ttlSecondsFromNow);
static thread_local ProcessInfo* currentProcess;
bool checkInjectedCorruption() {
auto iter = corruptWorkerMap.find(currentProcess->address);
if (iter != corruptWorkerMap.end())
return iter->second;
return false;
}
bool checkInjectedCorruption();
ISimulator();
virtual ~ISimulator();

View File

@ -39,6 +39,7 @@
#include "flow/IAsyncFile.h"
#include "fdbrpc/AsyncFileCached.actor.h"
#include "fdbrpc/AsyncFileEncrypted.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "fdbrpc/AsyncFileNonDurable.actor.h"
#include "fdbrpc/AsyncFileChaos.h"
#include "crc32/crc32c.h"
@ -108,6 +109,33 @@ bool simulator_should_inject_fault(const char* context, const char* file, int li
return false;
}
void ISimulator::disableFor(const std::string& desc, double time) {
disabledMap[desc] = time;
}
double ISimulator::checkDisabled(const std::string& desc) const {
auto iter = disabledMap.find(desc);
if (iter != disabledMap.end()) {
return iter->second;
}
return 0;
}
bool ISimulator::checkInjectedCorruption() {
auto iter = corruptWorkerMap.find(currentProcess->address);
if (iter != corruptWorkerMap.end())
return iter->second;
return false;
}
flowGlobalType ISimulator::global(int id) const {
return getCurrentProcess()->global(id);
};
void ISimulator::setGlobal(size_t id, flowGlobalType v) {
getCurrentProcess()->setGlobal(id, v);
};
void ISimulator::displayWorkers() const {
std::map<std::string, std::vector<ISimulator::ProcessInfo*>> machineMap;
@ -1389,7 +1417,7 @@ public:
}
}
}
return canKillProcesses(processesLeft, processesDead, KillInstantly, nullptr);
return canKillProcesses(processesLeft, processesDead, KillType::KillInstantly, nullptr);
}
bool datacenterDead(Optional<Standalone<StringRef>> dcId) const override {
@ -1428,7 +1456,7 @@ public:
// The following function will determine if a machine can be remove in case when it has a blob worker
bool canKillMachineWithBlobWorkers(Optional<Standalone<StringRef>> machineId, KillType kt, KillType* ktFinal) {
// Allow if no blob workers, or it's a reboot(without removing the machine)
if (!blobGranulesEnabled && kt >= RebootAndDelete) {
if (!blobGranulesEnabled && kt >= KillType::RebootAndDelete) {
return true;
}
@ -1459,7 +1487,7 @@ public:
// Ensure there is at least 1 remaining blob workers after removing current machine
if (nLeft <= 1) {
*ktFinal = RebootAndDelete; // reboot and delete data, but keep this machine
*ktFinal = KillType::RebootAndDelete; // reboot and delete data, but keep this machine
return false;
}
return true;
@ -1475,8 +1503,8 @@ public:
int nQuorum = ((desiredCoordinators + 1) / 2) * 2 - 1;
KillType newKt = kt;
if ((kt == KillInstantly) || (kt == InjectFaults) || (kt == FailDisk) || (kt == RebootAndDelete) ||
(kt == RebootProcessAndDelete)) {
if ((kt == KillType::KillInstantly) || (kt == KillType::InjectFaults) || (kt == KillType::FailDisk) ||
(kt == KillType::RebootAndDelete) || (kt == KillType::RebootProcessAndDelete)) {
LocalityGroup primaryProcessesLeft, primaryProcessesDead;
LocalityGroup primarySatelliteProcessesLeft, primarySatelliteProcessesDead;
LocalityGroup remoteProcessesLeft, remoteProcessesDead;
@ -1643,7 +1671,7 @@ public:
// Reboot if dead machines do fulfill policies
if (tooManyDead) {
newKt = Reboot;
newKt = KillType::Reboot;
canSurvive = false;
TraceEvent("KillChanged")
.detail("KillType", kt)
@ -1652,16 +1680,16 @@ public:
.detail("Reason", "Too many dead processes that cannot satisfy tLogPolicy.");
}
// Reboot and Delete if remaining machines do NOT fulfill policies
else if ((kt < RebootAndDelete) && notEnoughLeft) {
newKt = RebootAndDelete;
else if ((kt < KillType::RebootAndDelete) && notEnoughLeft) {
newKt = KillType::RebootAndDelete;
canSurvive = false;
TraceEvent("KillChanged")
.detail("KillType", kt)
.detail("NewKillType", newKt)
.detail("TLogPolicy", tLogPolicy->info())
.detail("Reason", "Not enough tLog left to satisfy tLogPolicy.");
} else if ((kt < RebootAndDelete) && (nQuorum > uniqueMachines.size())) {
newKt = RebootAndDelete;
} else if ((kt < KillType::RebootAndDelete) && (nQuorum > uniqueMachines.size())) {
newKt = KillType::RebootAndDelete;
canSurvive = false;
TraceEvent("KillChanged")
.detail("KillType", kt)
@ -1697,26 +1725,26 @@ public:
std::swap(*it, processes.back());
}
processes.pop_back();
killProcess_internal(p, KillInstantly);
killProcess_internal(p, KillType::KillInstantly);
}
void killProcess_internal(ProcessInfo* machine, KillType kt) {
CODE_PROBE(
true, "Simulated machine was killed with any kill type", probe::context::sim2, probe::assert::simOnly);
CODE_PROBE(kt == KillInstantly,
CODE_PROBE(kt == KillType::KillInstantly,
"Simulated machine was killed instantly",
probe::context::sim2,
probe::assert::simOnly);
CODE_PROBE(kt == InjectFaults,
CODE_PROBE(kt == KillType::InjectFaults,
"Simulated machine was killed with faults",
probe::context::sim2,
probe::assert::simOnly);
CODE_PROBE(kt == FailDisk,
CODE_PROBE(kt == KillType::FailDisk,
"Simulated machine was killed with a failed disk",
probe::context::sim2,
probe::assert::simOnly,
probe::decoration::rare);
if (kt == KillInstantly) {
if (kt == KillType::KillInstantly) {
TraceEvent(SevWarn, "FailMachine")
.detail("Name", machine->name)
.detail("Address", machine->address)
@ -1729,7 +1757,7 @@ public:
if (!machine->isSpawnedKVProcess())
latestEventCache.clear();
machine->failed = true;
} else if (kt == InjectFaults) {
} else if (kt == KillType::InjectFaults) {
TraceEvent(SevWarn, "FaultMachine")
.detail("Name", machine->name)
.detail("Address", machine->address)
@ -1742,8 +1770,8 @@ public:
machine->fault_injection_r = deterministicRandom()->randomUniqueID().first();
machine->fault_injection_p1 = 0.1;
machine->fault_injection_p2 = deterministicRandom()->random01();
} else if (kt == FailDisk) {
TraceEvent(SevWarn, "FailDiskMachine")
} else if (kt == KillType::FailDisk) {
TraceEvent(SevWarn, "KillType::FailDiskMachine")
.detail("Name", machine->name)
.detail("Address", machine->address)
.detail("ZoneId", machine->locality.zoneId())
@ -1758,13 +1786,13 @@ public:
ASSERT(!protectedAddresses.count(machine->address) || machine->rebooting || machine->isSpawnedKVProcess());
}
void rebootProcess(ProcessInfo* process, KillType kt) override {
if (kt == RebootProcessAndDelete && protectedAddresses.count(process->address)) {
if (kt == KillType::RebootProcessAndDelete && protectedAddresses.count(process->address)) {
TraceEvent("RebootChanged")
.detail("ZoneId", process->locality.describeZone())
.detail("KillType", RebootProcess)
.detail("KillType", KillType::RebootProcess)
.detail("OrigKillType", kt)
.detail("Reason", "Protected process");
kt = RebootProcess;
kt = KillType::RebootProcess;
}
doReboot(process, kt);
}
@ -1773,7 +1801,7 @@ public:
auto processes = getAllProcesses();
for (int i = 0; i < processes.size(); i++)
if (processes[i]->locality.zoneId() == zoneId && !processes[i]->rebooting)
doReboot(processes[i], RebootProcess);
doReboot(processes[i], KillType::RebootProcess);
} else {
auto processes = getAllProcesses();
for (int i = 0; i < processes.size(); i++) {
@ -1782,18 +1810,18 @@ public:
}
}
if (processes.size())
doReboot(deterministicRandom()->randomChoice(processes), RebootProcess);
doReboot(deterministicRandom()->randomChoice(processes), KillType::RebootProcess);
}
}
void killProcess(ProcessInfo* machine, KillType kt) override {
TraceEvent("AttemptingKillProcess").detail("ProcessInfo", machine->toString());
// Refuse to kill a protected process.
if (kt < RebootAndDelete && protectedAddresses.count(machine->address) == 0) {
if (kt < KillType::RebootAndDelete && protectedAddresses.count(machine->address) == 0) {
killProcess_internal(machine, kt);
}
}
void killInterface(NetworkAddress address, KillType kt) override {
if (kt < RebootAndDelete) {
if (kt < KillType::RebootAndDelete) {
std::vector<ProcessInfo*>& processes = machines[addressMap[address]->locality.machineId()].processes;
for (auto& process : processes) {
// Refuse to kill a protected process.
@ -1854,9 +1882,12 @@ public:
auto ktOrig = kt;
CODE_PROBE(true, "Trying to killing a machine", probe::context::sim2, probe::assert::simOnly);
CODE_PROBE(kt == KillInstantly, "Trying to kill instantly", probe::context::sim2, probe::assert::simOnly);
CODE_PROBE(
kt == InjectFaults, "Trying to kill by injecting faults", probe::context::sim2, probe::assert::simOnly);
kt == KillType::KillInstantly, "Trying to kill instantly", probe::context::sim2, probe::assert::simOnly);
CODE_PROBE(kt == KillType::InjectFaults,
"Trying to kill by injecting faults",
probe::context::sim2,
probe::assert::simOnly);
if (speedUpSimulation && !forceKill) {
TraceEvent(SevWarn, "AbortedKill")
@ -1864,7 +1895,7 @@ public:
.detail("Reason", "Unforced kill within speedy simulation.")
.backtrace();
if (ktFinal)
*ktFinal = None;
*ktFinal = KillType::None;
return false;
}
@ -1875,7 +1906,7 @@ public:
// Reboot if any of the processes are protected and count the number of processes not rebooting
for (auto& process : machines[machineId].processes) {
if (protectedAddresses.count(process->address))
kt = Reboot;
kt = KillType::Reboot;
if (!process->rebooting)
processesOnMachine++;
if (process->drProcess) {
@ -1892,13 +1923,14 @@ public:
.detail("ProcessesPerMachine", processesPerMachine)
.backtrace();
if (ktFinal)
*ktFinal = None;
*ktFinal = KillType::None;
return false;
}
// Check if machine can be removed, if requested
if (!forceKill && ((kt == KillInstantly) || (kt == InjectFaults) || (kt == FailDisk) ||
(kt == RebootAndDelete) || (kt == RebootProcessAndDelete))) {
if (!forceKill &&
((kt == KillType::KillInstantly) || (kt == KillType::InjectFaults) || (kt == KillType::FailDisk) ||
(kt == KillType::RebootAndDelete) || (kt == KillType::RebootProcessAndDelete))) {
if (!canKillMachineWithBlobWorkers(machineId, kt, &kt)) {
TraceEvent("CanKillMachineWithBlobWorkers")
@ -1947,7 +1979,8 @@ public:
.detail("ProtectedTotal", protectedAddresses.size())
.detail("TLogPolicy", tLogPolicy->info())
.detail("StoragePolicy", storagePolicy->info());
} else if ((kt == KillInstantly) || (kt == InjectFaults) || (kt == FailDisk)) {
} else if ((kt == KillType::KillInstantly) || (kt == KillType::InjectFaults) ||
(kt == KillType::FailDisk)) {
TraceEvent("DeadMachine")
.detail("MachineId", machineId)
.detail("KillType", kt)
@ -2007,12 +2040,12 @@ public:
probe::context::sim2,
probe::assert::simOnly);
if (isMainCluster && originalKt == RebootProcessAndSwitch) {
if (isMainCluster && originalKt == KillType::RebootProcessAndSwitch) {
// When killing processes with the RebootProcessAndSwitch kill
// type, processes in the original cluster should be rebooted in
// order to kill any zombie processes.
kt = KillType::Reboot;
} else if (processesOnMachine != processesPerMachine && kt != RebootProcessAndSwitch) {
} else if (processesOnMachine != processesPerMachine && kt != KillType::RebootProcessAndSwitch) {
// Check if any processes on machine are rebooting
CODE_PROBE(true,
"Attempted reboot, but the target did not have all of its processes running",
@ -2026,7 +2059,7 @@ public:
.detail("ProcessesPerMachine", processesPerMachine)
.backtrace();
if (ktFinal)
*ktFinal = None;
*ktFinal = KillType::None;
return false;
}
@ -2037,8 +2070,9 @@ public:
.detail("KillableMachines", processesOnMachine)
.detail("ProcessPerMachine", processesPerMachine)
.detail("KillChanged", kt != ktOrig);
if (kt < RebootAndDelete) {
if ((kt == InjectFaults || kt == FailDisk) && machines[machineId].machineProcess != nullptr)
if (kt < KillType::RebootAndDelete) {
if ((kt == KillType::InjectFaults || kt == KillType::FailDisk) &&
machines[machineId].machineProcess != nullptr)
killProcess_internal(machines[machineId].machineProcess, kt);
for (auto& process : machines[machineId].processes) {
TraceEvent("KillMachineProcess")
@ -2052,7 +2086,7 @@ public:
if (process->startingClass != ProcessClass::TesterClass)
killProcess_internal(process, kt);
}
} else if (kt == Reboot || kt == RebootAndDelete || kt == RebootProcessAndSwitch) {
} else if (kt == KillType::Reboot || kt == KillType::RebootAndDelete || kt == KillType::RebootProcessAndSwitch) {
for (auto& process : machines[machineId].processes) {
TraceEvent("KillMachineProcess")
.detail("KillType", kt)
@ -2067,12 +2101,17 @@ public:
}
}
CODE_PROBE(kt == KillType::RebootAndDelete,
"Resulted in a reboot and delete",
probe::context::sim2,
probe::assert::simOnly);
CODE_PROBE(kt == KillType::Reboot, "Resulted in a reboot", probe::context::sim2, probe::assert::simOnly);
CODE_PROBE(
kt == RebootAndDelete, "Resulted in a reboot and delete", probe::context::sim2, probe::assert::simOnly);
CODE_PROBE(kt == Reboot, "Resulted in a reboot", probe::context::sim2, probe::assert::simOnly);
CODE_PROBE(kt == KillInstantly, "Resulted in an instant kill", probe::context::sim2, probe::assert::simOnly);
CODE_PROBE(
kt == InjectFaults, "Resulted in a kill by injecting faults", probe::context::sim2, probe::assert::simOnly);
kt == KillType::KillInstantly, "Resulted in an instant kill", probe::context::sim2, probe::assert::simOnly);
CODE_PROBE(kt == KillType::InjectFaults,
"Resulted in a kill by injecting faults",
probe::context::sim2,
probe::assert::simOnly);
if (ktFinal)
*ktFinal = kt;
@ -2091,8 +2130,8 @@ public:
auto processMachineId = procRecord->locality.machineId();
ASSERT(processMachineId.present());
if (processDcId.present() && (processDcId == dcId)) {
if ((kt != Reboot) && (protectedAddresses.count(procRecord->address))) {
kt = Reboot;
if ((kt != KillType::Reboot) && (protectedAddresses.count(procRecord->address))) {
kt = KillType::Reboot;
TraceEvent(SevWarn, "DcKillChanged")
.detail("DataCenter", dcId)
.detail("KillType", kt)
@ -2111,8 +2150,9 @@ public:
}
// Check if machine can be removed, if requested
if (!forceKill && ((kt == KillInstantly) || (kt == InjectFaults) || (kt == FailDisk) ||
(kt == RebootAndDelete) || (kt == RebootProcessAndDelete))) {
if (!forceKill &&
((kt == KillType::KillInstantly) || (kt == KillType::InjectFaults) || (kt == KillType::FailDisk) ||
(kt == KillType::RebootAndDelete) || (kt == KillType::RebootProcessAndDelete))) {
std::vector<ProcessInfo*> processesLeft, processesDead;
for (auto processInfo : getAllProcesses()) {
if (processInfo->isAvailableClass()) {
@ -2171,7 +2211,7 @@ public:
.detail("KillType", kt)
.detail("KillTypeResult", ktResult)
.detail("KillTypeOrig", ktOrig);
ASSERT(ktResult == None);
ASSERT(ktResult == KillType::None);
}
ktMin = std::min<KillType>(ktResult, ktMin);
}
@ -2191,19 +2231,19 @@ public:
probe::context::sim2,
probe::assert::simOnly,
probe::decoration::rare);
CODE_PROBE((kt == ktMin) && (kt == RebootAndDelete),
CODE_PROBE((kt == ktMin) && (kt == KillType::RebootAndDelete),
"Datacenter kill Resulted in a reboot and delete",
probe::context::sim2,
probe::assert::simOnly);
CODE_PROBE((kt == ktMin) && (kt == Reboot),
CODE_PROBE((kt == ktMin) && (kt == KillType::Reboot),
"Datacenter kill Resulted in a reboot",
probe::context::sim2,
probe::assert::simOnly);
CODE_PROBE((kt == ktMin) && (kt == KillInstantly),
CODE_PROBE((kt == ktMin) && (kt == KillType::KillInstantly),
"Datacenter kill Resulted in an instant kill",
probe::context::sim2,
probe::assert::simOnly);
CODE_PROBE((kt == ktMin) && (kt == InjectFaults),
CODE_PROBE((kt == ktMin) && (kt == KillType::InjectFaults),
"Datacenter kill Resulted in a kill by injecting faults",
probe::context::sim2,
probe::assert::simOnly);
@ -2279,7 +2319,7 @@ public:
ASSERT(process->failed);
}
if (machine.machineProcess) {
killProcess_internal(machine.machineProcess, KillInstantly);
killProcess_internal(machine.machineProcess, KillType::KillInstantly);
}
machines.erase(machineId);
}
@ -2323,7 +2363,7 @@ public:
ASSERT(this->currentProcess == t.machine);
} catch (Error& e) {
TraceEvent(SevError, "UnhandledSimulationEventError").errorUnsuppressed(e);
killProcess(t.machine, KillInstantly);
killProcess(t.machine, KillType::KillInstantly);
}
if (randLog)
@ -2607,24 +2647,25 @@ ACTOR void doReboot(ISimulator::ProcessInfo* p, ISimulator::KillType kt) {
wait(g_sim2.delay(0, TaskPriority::DefaultDelay, p)); // Switch to the machine in question
try {
ASSERT(kt == ISimulator::RebootProcess || kt == ISimulator::Reboot || kt == ISimulator::RebootAndDelete ||
kt == ISimulator::RebootProcessAndDelete || kt == ISimulator::RebootProcessAndSwitch);
ASSERT(kt == ISimulator::KillType::RebootProcess || kt == ISimulator::KillType::Reboot ||
kt == ISimulator::KillType::RebootAndDelete || kt == ISimulator::KillType::RebootProcessAndDelete ||
kt == ISimulator::KillType::RebootProcessAndSwitch);
CODE_PROBE(kt == ISimulator::RebootProcess,
CODE_PROBE(kt == ISimulator::KillType::RebootProcess,
"Simulated process rebooted",
probe::assert::simOnly,
probe::context::sim2);
CODE_PROBE(
kt == ISimulator::Reboot, "Simulated machine rebooted", probe::assert::simOnly, probe::context::sim2);
CODE_PROBE(kt == ISimulator::RebootAndDelete,
kt == ISimulator::KillType::Reboot, "Simulated machine rebooted", probe::assert::simOnly, probe::context::sim2);
CODE_PROBE(kt == ISimulator::KillType::RebootAndDelete,
"Simulated machine rebooted with data and coordination state deletion",
probe::assert::simOnly,
probe::context::sim2);
CODE_PROBE(kt == ISimulator::RebootProcessAndDelete,
CODE_PROBE(kt == ISimulator::KillType::RebootProcessAndDelete,
"Simulated process rebooted with data and coordination state deletion",
probe::assert::simOnly,
probe::context::sim2);
CODE_PROBE(kt == ISimulator::RebootProcessAndSwitch,
CODE_PROBE(kt == ISimulator::KillType::RebootProcessAndSwitch,
"Simulated process rebooted with different cluster file",
probe::assert::simOnly,
probe::context::sim2);
@ -2653,10 +2694,10 @@ ACTOR void doReboot(ISimulator::ProcessInfo* p, ISimulator::KillType kt) {
.detail("Cleared", p->cleared)
.backtrace();
p->rebooting = true;
if ((kt == ISimulator::RebootAndDelete) || (kt == ISimulator::RebootProcessAndDelete)) {
if ((kt == ISimulator::KillType::RebootAndDelete) || (kt == ISimulator::KillType::RebootProcessAndDelete)) {
p->cleared = true;
g_simulator->clearAddress(p->address);
} else if (kt == ISimulator::RebootProcessAndSwitch) {
} else if (kt == ISimulator::KillType::RebootProcessAndSwitch) {
g_simulator->switchCluster(p->address);
}
p->shutdownSignal.send(kt);

View File

@ -35,6 +35,7 @@
#include "fdbrpc/FlowProcess.actor.h"
#include "fdbrpc/Net2FileSystem.h"
#include "fdbrpc/simulator.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "fdbrpc/WellKnownEndpoints.h"
#include "fdbclient/versions.h"
#include "fdbserver/CoroFlow.h"

View File

@ -38,6 +38,7 @@ u32 sqlite3VdbeSerialGet(const unsigned char*, u32, Mem*);
#include "fdbserver/VFSAsync.h"
#include "fdbserver/template_fdb.h"
#include "fdbrpc/simulator.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "flow/actorcompiler.h" // This must be the last #include.
#if SQLITE_THREADSAFE == 0

View File

@ -732,7 +732,7 @@ ACTOR Future<Void> repairDeadDatacenter(Database cx,
.detail("PrimaryDead", primaryDead);
g_simulator->usableRegions = 1;
g_simulator->killDataCenter(
primaryDead ? g_simulator->primaryDcId : g_simulator->remoteDcId, ISimulator::KillInstantly, true);
primaryDead ? g_simulator->primaryDcId : g_simulator->remoteDcId, ISimulator::KillType::KillInstantly, true);
wait(success(ManagementAPI::changeConfig(
cx.getReference(),
(primaryDead ? g_simulator->disablePrimary : g_simulator->disableRemote) + " repopulate_anti_quorum=1",

View File

@ -39,6 +39,7 @@
#include "flow/ActorCollection.h"
#include "fdbserver/RestoreWorker.actor.h"
#include "fdbserver/RestoreController.actor.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "flow/actorcompiler.h" // This must be the last #include.

View File

@ -28,6 +28,7 @@
#include "fdbrpc/Locality.h"
#include "fdbrpc/simulator.h"
#include "fdbrpc/IPAllowList.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "fdbclient/ClusterConnectionFile.h"
#include "fdbclient/ClusterConnectionMemoryRecord.h"
#include "fdbclient/DatabaseContext.h"
@ -62,6 +63,10 @@ extern const char* getSourceVersion();
using namespace std::literals;
bool isSimulatorProcessReliable() {
return g_network->isSimulated() && !g_simulator->getCurrentProcess()->isReliable();
}
namespace probe {
namespace assert {
@ -780,7 +785,7 @@ ACTOR Future<ISimulator::KillType> simulatedFDBDRebooter(Reference<IClusterConne
.backtrace();
if (e.code() == error_code_io_timeout && !onShutdown.isReady()) {
onShutdown = ISimulator::RebootProcess;
onShutdown = ISimulator::KillType::RebootProcess;
}
if (onShutdown.isReady() && onShutdown.isError())
@ -804,10 +809,10 @@ ACTOR Future<ISimulator::KillType> simulatedFDBDRebooter(Reference<IClusterConne
.detail("Address", process->address)
.detail("Excluded", process->excluded)
.detail("ZoneId", localities.zoneId())
.detail("KillType", onShutdown.isReady() ? onShutdown.get() : ISimulator::None);
.detail("KillType", onShutdown.isReady() ? onShutdown.get() : ISimulator::KillType::None);
if (!onShutdown.isReady())
onShutdown = ISimulator::InjectFaults;
onShutdown = ISimulator::KillType::InjectFaults;
} catch (Error& e) {
TraceEvent(destructed ? SevInfo : SevError, "SimulatedFDBDRebooterError")
.errorUnsuppressed(e)
@ -820,7 +825,7 @@ ACTOR Future<ISimulator::KillType> simulatedFDBDRebooter(Reference<IClusterConne
if (!process->shutdownSignal.isSet() && !destructed) {
process->rebooting = true;
process->shutdownSignal.send(ISimulator::None);
process->shutdownSignal.send(ISimulator::KillType::None);
}
TraceEvent("SimulatedFDBDWait")
.detail("Cycles", cycles)
@ -845,7 +850,7 @@ ACTOR Future<ISimulator::KillType> simulatedFDBDRebooter(Reference<IClusterConne
.detail("ZoneId", localities.zoneId())
.detail("KillType", shutdownResult);
if (shutdownResult < ISimulator::RebootProcessAndDelete) {
if (shutdownResult < ISimulator::KillType::RebootProcessAndDelete) {
TraceEvent("SimulatedFDBDLowerReboot")
.detail("Cycles", cycles)
.detail("RandomId", randomId)
@ -856,7 +861,7 @@ ACTOR Future<ISimulator::KillType> simulatedFDBDRebooter(Reference<IClusterConne
return onShutdown.get();
}
if (onShutdown.get() == ISimulator::RebootProcessAndDelete) {
if (onShutdown.get() == ISimulator::KillType::RebootProcessAndDelete) {
TraceEvent("SimulatedFDBDRebootAndDelete")
.detail("Cycles", cycles)
.detail("RandomId", randomId)
@ -874,7 +879,7 @@ ACTOR Future<ISimulator::KillType> simulatedFDBDRebooter(Reference<IClusterConne
connRecord =
makeReference<ClusterConnectionFile>(joinPath(*dataFolder, "fdb.cluster"), connStr.toString());
}
} else if (onShutdown.get() == ISimulator::RebootProcessAndSwitch) {
} else if (onShutdown.get() == ISimulator::KillType::RebootProcessAndSwitch) {
TraceEvent("SimulatedFDBDRebootAndSwitch")
.detail("Cycles", cycles)
.detail("RandomId", randomId)
@ -1147,7 +1152,7 @@ ACTOR Future<Void> simulatedMachine(ClusterConnectionString connStr,
CODE_PROBE(true, "Simulated machine has been rebooted");
state bool swap = killType == ISimulator::Reboot && BUGGIFY_WITH_PROB(0.75) &&
state bool swap = killType == ISimulator::KillType::Reboot && BUGGIFY_WITH_PROB(0.75) &&
g_simulator->canSwapToMachine(localities.zoneId());
if (swap)
availableFolders[localities.dcId()].push_back(myFolders);
@ -1187,7 +1192,7 @@ ACTOR Future<Void> simulatedMachine(ClusterConnectionString connStr,
}
}
}
} else if (killType == ISimulator::RebootAndDelete) {
} else if (killType == ISimulator::KillType::RebootAndDelete) {
for (int i = 0; i < ips.size(); i++) {
coordFolders[i] = joinPath(baseFolder, deterministicRandom()->randomUniqueID().toString());
myFolders[i] = joinPath(baseFolder, deterministicRandom()->randomUniqueID().toString());

View File

@ -26,6 +26,7 @@
#include "flow/IAsyncFile.h"
#include "fdbserver/CoroFlow.h"
#include "fdbrpc/simulator.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "fdbrpc/AsyncFileReadAhead.actor.h"
#include <assert.h>

View File

@ -23,6 +23,7 @@
#include "Coro.h"
#include "flow/TDMetric.actor.h"
#include "fdbrpc/simulator.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "flow/actorcompiler.h" // has to be last include
// Old libcoroutine based implementation. Used on Windows until CI has

View File

@ -45,6 +45,7 @@
#include "fdbclient/versions.h"
#include "fdbclient/BuildFlags.h"
#include "fdbrpc/WellKnownEndpoints.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "fdbclient/SimpleIni.h"
#include "fdbrpc/AsyncFileCached.actor.h"
#include "fdbrpc/IPAllowList.h"
@ -363,7 +364,7 @@ ACTOR void failAfter(Future<Void> trigger, ISimulator::ProcessInfo* m = g_simula
wait(trigger);
if (enableFailures) {
printf("Killing machine: %s at %f\n", m->address.toString().c_str(), now());
g_simulator->killProcess(m, ISimulator::KillInstantly);
g_simulator->killProcess(m, ISimulator::KillType::KillInstantly);
}
}

View File

@ -1301,6 +1301,8 @@ ACTOR Future<Void> tLog(IKeyValueStore* persistentData,
typedef decltype(&tLog) TLogFn;
extern bool isSimulatorProcessReliable();
ACTOR template <class T>
Future<T> ioTimeoutError(Future<T> what, double time, const char* context = nullptr) {
// Before simulation is sped up, IO operations can take a very long time so limit timeouts
@ -1310,12 +1312,10 @@ Future<T> ioTimeoutError(Future<T> what, double time, const char* context = null
}
Future<Void> end = lowPriorityDelay(time);
choose {
when(T t = wait(what)) {
return t;
}
when(T t = wait(what)) { return t; }
when(wait(end)) {
Error err = io_timeout();
if (g_network->isSimulated() && !g_simulator->getCurrentProcess()->isReliable()) {
if (!isSimulatorProcessReliable()) {
err = err.asInjectedFault();
}
TraceEvent e(SevError, "IoTimeoutError");
@ -1346,9 +1346,7 @@ Future<T> ioDegradedOrTimeoutError(Future<T> what,
if (degradedTime < errTime) {
Future<Void> degradedEnd = lowPriorityDelay(degradedTime);
choose {
when(T t = wait(what)) {
return t;
}
when(T t = wait(what)) { return t; }
when(wait(degradedEnd)) {
CODE_PROBE(true, "TLog degraded", probe::func::deduplicate);
TraceEvent(SevWarnAlways, "IoDegraded").log();
@ -1359,12 +1357,10 @@ Future<T> ioDegradedOrTimeoutError(Future<T> what,
Future<Void> end = lowPriorityDelay(errTime - degradedTime);
choose {
when(T t = wait(what)) {
return t;
}
when(T t = wait(what)) { return t; }
when(wait(end)) {
Error err = io_timeout();
if (g_network->isSimulated() && !g_simulator->getCurrentProcess()->isReliable()) {
if (!isSimulatorProcessReliable()) {
err = err.asInjectedFault();
}
TraceEvent e(SevError, "IoTimeoutError");

View File

@ -46,6 +46,7 @@
#include "fdbclient/ManagementAPI.actor.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/WorkerInterface.actor.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "flow/actorcompiler.h" // This must be the last #include.
FDB_DEFINE_BOOLEAN_PARAM(UntrustedMode);

View File

@ -71,6 +71,7 @@
#include "flow/network.h"
#include "flow/serialize.h"
#include "flow/ChaosMetrics.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#ifdef __linux__
#include <fcntl.h>

View File

@ -21,6 +21,7 @@
#include "fdbserver/ServerDBInfo.actor.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "fdbrpc/simulator.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include <fmt/format.h>

View File

@ -21,7 +21,7 @@
#include "fdbclient/NativeAPI.actor.h"
#include "fdbserver/TesterInterface.actor.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "fdbrpc/simulator.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "flow/actorcompiler.h" // This must be the last #include.
class ClogSingleConnectionWorkload : public TestWorkload {

View File

@ -38,6 +38,7 @@
#include "fdbclient/ManagementAPI.actor.h"
#include "fdbclient/StorageServerInterface.h"
#include "flow/network.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "flow/actorcompiler.h" // This must be the last #include.
@ -1114,9 +1115,9 @@ struct ConsistencyCheckWorkload : TestWorkload {
.detail("ReliableInfo", p->getReliableInfo())
.detail("KillOrRebootProcess", p->address);
if (p->isReliable()) {
g_simulator->rebootProcess(p, ISimulator::RebootProcess);
g_simulator->rebootProcess(p, ISimulator::KillType::RebootProcess);
} else {
g_simulator->killProcess(p, ISimulator::KillInstantly);
g_simulator->killProcess(p, ISimulator::KillType::KillInstantly);
}
}

View File

@ -31,6 +31,7 @@
#include "flow/Error.h"
#include "flow/IRandom.h"
#include "flow/flow.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "flow/actorcompiler.h" // This must be the last #include.
namespace {
@ -278,7 +279,7 @@ struct DataLossRecoveryWorkload : TestWorkload {
void killProcess(DataLossRecoveryWorkload* self, const NetworkAddress& addr) {
ISimulator::ProcessInfo* process = g_simulator->getProcessByAddress(addr);
ASSERT(process->addresses.contains(addr));
g_simulator->killProcess(process, ISimulator::KillInstantly);
g_simulator->killProcess(process, ISimulator::KillType::KillInstantly);
TraceEvent("TestTeamKilled").detail("Address", addr);
}

View File

@ -89,16 +89,16 @@ struct KillRegionWorkload : TestWorkload {
// FIXME: killDataCenter breaks simulation if forceKill=false, since some processes can survive and
// partially complete a recovery
g_simulator->killDataCenter("0"_sr,
deterministicRandom()->random01() < 0.5 ? ISimulator::KillInstantly
: ISimulator::RebootAndDelete,
deterministicRandom()->random01() < 0.5 ? ISimulator::KillType::KillInstantly
: ISimulator::KillType::RebootAndDelete,
true);
g_simulator->killDataCenter("2"_sr,
deterministicRandom()->random01() < 0.5 ? ISimulator::KillInstantly
: ISimulator::RebootAndDelete,
deterministicRandom()->random01() < 0.5 ? ISimulator::KillType::KillInstantly
: ISimulator::KillType::RebootAndDelete,
true);
g_simulator->killDataCenter("4"_sr,
deterministicRandom()->random01() < 0.5 ? ISimulator::KillInstantly
: ISimulator::RebootAndDelete,
deterministicRandom()->random01() < 0.5 ? ISimulator::KillType::KillInstantly
: ISimulator::KillType::RebootAndDelete,
true);
TraceEvent("ForceRecovery_Begin").log();

View File

@ -28,6 +28,7 @@
#include "fdbclient/ManagementAPI.actor.h"
#include "flow/FaultInjection.h"
#include "flow/DeterministicRandom.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "flow/actorcompiler.h" // This must be the last #include.
static std::set<int> const& normalAttritionErrors() {
@ -336,17 +337,17 @@ struct MachineAttritionWorkload : FailureInjectionWorkload {
ASSERT(self->machines.size());
Optional<Standalone<StringRef>> target = self->machines.back().dcId();
ISimulator::KillType kt = ISimulator::Reboot;
ISimulator::KillType kt = ISimulator::KillType::Reboot;
if (!self->reboot) {
int killType = deterministicRandom()->randomInt(0, 3); // FIXME: enable disk stalls
if (killType == 0)
kt = ISimulator::KillInstantly;
kt = ISimulator::KillType::KillInstantly;
else if (killType == 1)
kt = ISimulator::InjectFaults;
kt = ISimulator::KillType::InjectFaults;
else if (killType == 2)
kt = ISimulator::RebootAndDelete;
kt = ISimulator::KillType::RebootAndDelete;
else
kt = ISimulator::FailDisk;
kt = ISimulator::KillType::FailDisk;
}
TraceEvent("Assassination")
.detail("TargetDatacenter", target)
@ -362,12 +363,12 @@ struct MachineAttritionWorkload : FailureInjectionWorkload {
ASSERT(self->targetIds.size() == 1);
auto target = self->targetIds.front();
auto kt = ISimulator::KillInstantly;
auto kt = ISimulator::KillType::KillInstantly;
TraceEvent("Assassination").detail("TargetDataHall", target).detail("KillType", kt);
g_simulator->killDataHall(target, kt);
} else if (self->killAll) {
state ISimulator::KillType kt = ISimulator::RebootProcessAndSwitch;
state ISimulator::KillType kt = ISimulator::KillType::RebootProcessAndSwitch;
TraceEvent("Assassination").detail("KillType", kt);
g_simulator->killAll(kt, true);
g_simulator->toggleGlobalSwitchCluster();
@ -428,7 +429,7 @@ struct MachineAttritionWorkload : FailureInjectionWorkload {
if (deterministicRandom()->random01() > 0.5) {
g_simulator->rebootProcess(targetMachine.zoneId(), deterministicRandom()->random01() > 0.5);
} else {
g_simulator->killZone(targetMachine.zoneId(), ISimulator::Reboot);
g_simulator->killZone(targetMachine.zoneId(), ISimulator::KillType::Reboot);
}
} else {
auto randomDouble = deterministicRandom()->random01();
@ -437,19 +438,19 @@ struct MachineAttritionWorkload : FailureInjectionWorkload {
.detail("RandomValue", randomDouble);
if (randomDouble < 0.33) {
TraceEvent("RebootAndDelete").detail("TargetMachine", targetMachine.toString());
g_simulator->killZone(targetMachine.zoneId(), ISimulator::RebootAndDelete);
g_simulator->killZone(targetMachine.zoneId(), ISimulator::KillType::RebootAndDelete);
} else {
auto kt = ISimulator::KillInstantly;
auto kt = ISimulator::KillType::KillInstantly;
if (self->allowFaultInjection) {
if (randomDouble < 0.50) {
kt = ISimulator::InjectFaults;
kt = ISimulator::KillType::InjectFaults;
}
// FIXME: enable disk stalls
/*
if( randomDouble < 0.56 ) {
kt = ISimulator::InjectFaults;
kt = ISimulator::KillType::InjectFaults;
} else if( randomDouble < 0.66 ) {
kt = ISimulator::FailDisk;
kt = ISimulator::KillType::FailDisk;
}
*/
}

View File

@ -19,6 +19,7 @@
*/
#include "fdbserver/workloads/workloads.actor.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "flow/actorcompiler.h" // This must be the last include
struct ProtocolVersionWorkload : TestWorkload {

View File

@ -20,6 +20,7 @@
#include "flow/DeterministicRandom.h"
#include "fdbrpc/simulator.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "fdbclient/NativeAPI.actor.h"
#include "fdbserver/TesterInterface.actor.h"
#include "fdbserver/workloads/workloads.actor.h"

View File

@ -211,7 +211,7 @@ struct MoveKeysWorkload : FailureInjectionWorkload {
ACTOR Future<Void> forceMasterFailure(Database cx, MoveKeysWorkload* self) {
ASSERT(g_network->isSimulated());
loop {
if (g_simulator->killZone(self->dbInfo->get().master.locality.zoneId(), ISimulator::Reboot, true))
if (g_simulator->killZone(self->dbInfo->get().master.locality.zoneId(), ISimulator::KillType::Reboot, true))
return Void();
wait(delay(1.0));
}

View File

@ -23,6 +23,7 @@
#include "fdbserver/WorkerInterface.actor.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "fdbrpc/simulator.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#include "fdbclient/ManagementAPI.actor.h"
#include "flow/actorcompiler.h" // This must be the last #include.
@ -308,7 +309,7 @@ struct RemoveServersSafelyWorkload : TestWorkload {
// Check if we can kill the added process
bCanKillProcess =
g_simulator->canKillProcesses(processesLeft, processesDead, ISimulator::KillInstantly, nullptr);
g_simulator->canKillProcesses(processesLeft, processesDead, ISimulator::KillType::KillInstantly, nullptr);
// Remove the added processes
processesLeft.resize(processesLeft.size() - killProcArray.size());
@ -467,7 +468,7 @@ struct RemoveServersSafelyWorkload : TestWorkload {
.detail("ClusterAvailable", g_simulator->isAvailable())
.detail("Protected", g_simulator->protectedAddresses.count(killProcess->address));
else if (removeViaClear) {
g_simulator->rebootProcess(killProcess, ISimulator::RebootProcessAndDelete);
g_simulator->rebootProcess(killProcess, ISimulator::KillType::RebootProcessAndDelete);
TraceEvent("RemoveAndKill", functionId)
.detail("Step", "Clear Process")
.detail("Process", describe(*killProcess))
@ -478,7 +479,7 @@ struct RemoveServersSafelyWorkload : TestWorkload {
}
/*
else {
g_simulator->killProcess( killProcess, ISimulator::KillInstantly );
g_simulator->killProcess( killProcess, ISimulator::KillType::KillInstantly );
TraceEvent("RemoveAndKill", functionId).detail("Step", "Kill Process").detail("Process", describe(*killProcess)).detail("Failed", killProcess->failed).detail("Rebooting", killProcess->rebooting).detail("ClusterAvailable", g_simulator->isAvailable()).detail("Protected", g_simulator->protectedAddresses.count(killProcess->address));
}
*/
@ -497,7 +498,7 @@ struct RemoveServersSafelyWorkload : TestWorkload {
.detail("ClusterAvailable", g_simulator->isAvailable());
for (auto& zoneId : zoneIds) {
killedMachine = g_simulator->killZone(
zoneId, removeViaClear ? ISimulator::RebootAndDelete : ISimulator::KillInstantly);
zoneId, removeViaClear ? ISimulator::KillType::RebootAndDelete : ISimulator::KillType::KillInstantly);
TraceEvent(killedMachine ? SevInfo : SevWarn, "RemoveAndKill")
.detail("Step", removeViaClear ? "Clear Machine" : "Kill Machine")
.detail("ZoneId", zoneId)

View File

@ -105,7 +105,7 @@ struct RollbackWorkload : FailureInjectionWorkload {
if (self->enableFailures) {
// Reboot the proxy and clog the unclogged tlog.
g_simulator->rebootProcess(g_simulator->getProcessByAddress(proxy.address()), ISimulator::Reboot);
g_simulator->rebootProcess(g_simulator->getProcessByAddress(proxy.address()), ISimulator::KillType::Reboot);
g_simulator->clogInterface(uncloggedTLog.ip, self->clogDuration, ClogAll);
} else {
// Alternatively, if we're not injecting machine failures, clog the proxy and the unclogged tlog.

View File

@ -25,6 +25,7 @@
#include "fdbrpc/simulator.h"
#include "boost/algorithm/string/predicate.hpp"
#include "flow/IConnection.h"
#include "fdbrpc/SimulatorProcessInfo.h"
#undef state
#include "fdbclient/SimpleIni.h"
@ -146,7 +147,7 @@ struct SaveAndKillWorkload : TestWorkload {
ini.SaveFile(self->restartInfo.c_str());
for (auto process = allProcessesMap.begin(); process != allProcessesMap.end(); process++) {
g_simulator->killProcess(process->second, ISimulator::Reboot);
g_simulator->killProcess(process->second, ISimulator::KillType::Reboot);
}
for (i = 0; i < 100; i++) {

View File

@ -63,7 +63,7 @@ struct TargetedKillWorkload : TestWorkload {
Database cx,
TargetedKillWorkload* self) {
if (g_simulator == g_network) {
g_simulator->killInterface(address, ISimulator::KillInstantly);
g_simulator->killInterface(address, ISimulator::KillType::KillInstantly);
return Void();
}