mirror of
https://github.com/apple/foundationdb.git
synced 2025-05-14 01:42:37 +08:00
Remote ikvs debugging (#6465)
* initial structure for remote IKVS server * moved struct to .h file, added new files to CMakeList * happy path implementation, connection error when testing * saved minor local change * changed tracing to debug * fixed onClosed and getError being called before init is finished * fix spawn process bug, now use absolute path * added server knob to set ikvs process port number * added server knob for remote/local kv store * implement simulator remote process spawning * fixed bug for simulator timeout * commit all changes * removed print lines in trace * added FlowProcess implementation by Markus * initial debug of FlowProcess, stuck at parent sending OpenKVStoreRequest to child * temporary fix for process factory throwing segfault on create * specify public address in command * change remote kv store knob to false for jenkins build * made port 0 open random unused port * change remote store knob to true for benchmark * set listening port to randomly opened port * added print lines for jenkins run open kv store timeout debug * removed most tracing and print lines * removed tutorial changes * update handleIOErrors error handling to handle remote-ikvs cases * Push all debugging changes * A version where worker bug exists * A version where restarting tests fail * Use both the name and the port to determine the child process * Remove unnecessary update on local address * Disable remote-kvs for DiskFailureCycle test * A version where restarting stuck * A version where most restarting tests green * Reset connection with child process explicitly * Remove change on unnecessary files * Unify flags from _ to - * fix merging unexpected changes * fix trac.error to .errorUnsuppressed * Add license header * Remove unnecessary header in FlowProcess.actor.cpp * Fix Windows build * Fix Windows build, add missing ; * Fix a stupid bug caused by code dropped by code merging * Disable remote kvs by default * Pass the conn_file path to the flow process, though not needed, but the buildNetwork is difficult to tune * serialization change on readrange * Update traces * Refactor the RemoteIKVS interface * Format files * Update sim2 interface to not clog connections between parent and child processes in simulation * Update comments; remove debugging symbols; Add error handling for remote_kvs_cancelled * Add comments, format files * Change method name from isBuggifyDisabled to isStableConnection; Decrease(0.1x) latency for stable connections * Commit the IConnection interface change, forgot in previous commit * Fix the issue that onClosed request is cancelled by ActorCollection * Enable the remote kv store knob * Remove FlowProcess.actor.cpp and move functions to RemoteIKeyValueStore.actor.cpp; Add remote kv store delay to avoid race; Bind the child process to die with parent process * Fix the bug where one process starts storage server more than once * Add a please_reboot_remote_kv_store error to restart the storage server worker if remote kvs died abnormally * Remove unreachable code path and add comments * Clang format the code * Fix a simple wait error * Clang format after merging the main branch * Testing mixed mode in simulation if remote_kvs knob is enabled, setting the default to false * Disable remote kvs for PhysicalShardMove which is for RocksDB * Cleanup #include orders, remove debugging traces * Revert the reorder in fdbserver.actor.cpp, which fails the gcc build Co-authored-by: “Lincoln <“lincoln.xiao@snowflake.com”>
This commit is contained in:
parent
c7d2f5fee2
commit
7d365bd1bb
@ -233,7 +233,8 @@ def suspend(logger):
|
||||
port = address.split(':')[1]
|
||||
logger.debug("Port: {}".format(port))
|
||||
# use the port number to find the exact fdb process we are connecting to
|
||||
pinfo = list(filter(lambda x: port in x, pinfos))
|
||||
# child process like fdbserver -r flowprocess does not provide `datadir` in the command line
|
||||
pinfo = list(filter(lambda x: port in x and 'datadir' in x, pinfos))
|
||||
assert len(pinfo) == 1
|
||||
pid = pinfo[0].split(' ')[0]
|
||||
logger.debug("Pid: {}".format(pid))
|
||||
|
@ -652,6 +652,7 @@ struct GetRangeLimits {
|
||||
};
|
||||
|
||||
struct RangeResultRef : VectorRef<KeyValueRef> {
|
||||
constexpr static FileIdentifier file_identifier = 3985192;
|
||||
bool more; // True if (but not necessarily only if) values remain in the *key* range requested (possibly beyond the
|
||||
// limits requested) False implies that no such values remain
|
||||
Optional<KeyRef> readThrough; // Only present when 'more' is true. When present, this value represent the end (or
|
||||
@ -958,6 +959,7 @@ struct TLogSpillType {
|
||||
|
||||
// Contains the amount of free and total space for a storage server, in bytes
|
||||
struct StorageBytes {
|
||||
constexpr static FileIdentifier file_identifier = 3928581;
|
||||
// Free space on the filesystem
|
||||
int64_t free;
|
||||
// Total space on the filesystem
|
||||
|
@ -250,6 +250,9 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
|
||||
init( DEBOUNCE_RECRUITING_DELAY, 5.0 );
|
||||
init( DD_FAILURE_TIME, 1.0 ); if( randomize && BUGGIFY ) DD_FAILURE_TIME = 10.0;
|
||||
init( DD_ZERO_HEALTHY_TEAM_DELAY, 1.0 );
|
||||
init( REMOTE_KV_STORE, false ); if( randomize && BUGGIFY ) REMOTE_KV_STORE = true;
|
||||
init( REMOTE_KV_STORE_INIT_DELAY, 0.1 );
|
||||
init( REMOTE_KV_STORE_MAX_INIT_DURATION, 10.0 );
|
||||
init( REBALANCE_MAX_RETRIES, 100 );
|
||||
init( DD_OVERLAP_PENALTY, 10000 );
|
||||
init( DD_EXCLUDE_MIN_REPLICAS, 1 );
|
||||
@ -555,6 +558,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
|
||||
init( MIN_REBOOT_TIME, 4.0 ); if( longReboots ) MIN_REBOOT_TIME = 10.0;
|
||||
init( MAX_REBOOT_TIME, 5.0 ); if( longReboots ) MAX_REBOOT_TIME = 20.0;
|
||||
init( LOG_DIRECTORY, "."); // Will be set to the command line flag.
|
||||
init( CONN_FILE, ""); // Will be set to the command line flag.
|
||||
init( SERVER_MEM_LIMIT, 8LL << 30 );
|
||||
init( SYSTEM_MONITOR_FREQUENCY, 5.0 );
|
||||
|
||||
|
@ -233,6 +233,14 @@ public:
|
||||
double DD_FAILURE_TIME;
|
||||
double DD_ZERO_HEALTHY_TEAM_DELAY;
|
||||
|
||||
// Run storage enginee on a child process on the same machine with storage process
|
||||
bool REMOTE_KV_STORE;
|
||||
// A delay to avoid race on file resources if the new kv store process started immediately after the previous kv
|
||||
// store process died
|
||||
double REMOTE_KV_STORE_INIT_DELAY;
|
||||
// max waiting time for the remote kv store to initialize
|
||||
double REMOTE_KV_STORE_MAX_INIT_DURATION;
|
||||
|
||||
// KeyValueStore SQLITE
|
||||
int CLEAR_BUFFER_SIZE;
|
||||
double READ_VALUE_TIME_ESTIMATE;
|
||||
@ -488,6 +496,7 @@ public:
|
||||
double MIN_REBOOT_TIME;
|
||||
double MAX_REBOOT_TIME;
|
||||
std::string LOG_DIRECTORY;
|
||||
std::string CONN_FILE;
|
||||
int64_t SERVER_MEM_LIMIT;
|
||||
double SYSTEM_MONITOR_FREQUENCY;
|
||||
|
||||
|
@ -10,6 +10,7 @@ set(FDBRPC_SRCS
|
||||
AsyncFileNonDurable.actor.cpp
|
||||
AsyncFileWriteChecker.cpp
|
||||
FailureMonitor.actor.cpp
|
||||
FlowProcess.actor.h
|
||||
FlowTransport.actor.cpp
|
||||
genericactors.actor.h
|
||||
genericactors.actor.cpp
|
||||
|
94
fdbrpc/FlowProcess.actor.h
Normal file
94
fdbrpc/FlowProcess.actor.h
Normal file
@ -0,0 +1,94 @@
|
||||
/*
|
||||
* FlowProcess.actor.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#pragma once
|
||||
|
||||
#if defined(NO_INTELLISENSE) && !defined(FDBRPC_FLOW_PROCESS_ACTOR_G_H)
|
||||
#define FDBRPC_FLOW_PROCESS_ACTOR_G_H
|
||||
#include "fdbrpc/FlowProcess.actor.g.h"
|
||||
#elif !defined(FDBRPC_FLOW_PROCESS_ACTOR_H)
|
||||
#define FDBRPC_FLOW_PROCESS_ACTOR_H
|
||||
|
||||
#include "fdbrpc/fdbrpc.h"
|
||||
|
||||
#include <string>
|
||||
#include <map>
|
||||
|
||||
#include <flow/actorcompiler.h> // has to be last include
|
||||
|
||||
struct FlowProcessInterface {
|
||||
constexpr static FileIdentifier file_identifier = 3491839;
|
||||
RequestStream<struct FlowProcessRegistrationRequest> registerProcess;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, registerProcess);
|
||||
}
|
||||
};
|
||||
|
||||
struct FlowProcessRegistrationRequest {
|
||||
constexpr static FileIdentifier file_identifier = 3411838;
|
||||
Standalone<StringRef> flowProcessInterface;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, flowProcessInterface);
|
||||
}
|
||||
};
|
||||
|
||||
class FlowProcess {
|
||||
|
||||
public:
|
||||
virtual ~FlowProcess() {}
|
||||
virtual StringRef name() const = 0;
|
||||
virtual StringRef serializedInterface() const = 0;
|
||||
virtual Future<Void> run() = 0;
|
||||
virtual void registerEndpoint(Endpoint p) = 0;
|
||||
};
|
||||
|
||||
struct IProcessFactory {
|
||||
static FlowProcess* create(std::string const& name) {
|
||||
auto it = factories().find(name);
|
||||
if (it == factories().end())
|
||||
return nullptr; // or throw?
|
||||
return it->second->create();
|
||||
}
|
||||
static std::map<std::string, IProcessFactory*>& factories() {
|
||||
static std::map<std::string, IProcessFactory*> theFactories;
|
||||
return theFactories;
|
||||
}
|
||||
|
||||
virtual FlowProcess* create() = 0;
|
||||
|
||||
virtual const char* getName() = 0;
|
||||
};
|
||||
|
||||
template <class ProcessType>
|
||||
struct ProcessFactory : IProcessFactory {
|
||||
ProcessFactory(const char* name) : name(name) { factories()[name] = this; }
|
||||
FlowProcess* create() override { return new ProcessType(); }
|
||||
const char* getName() override { return this->name; }
|
||||
|
||||
private:
|
||||
const char* name;
|
||||
};
|
||||
|
||||
#include <flow/unactorcompiler.h>
|
||||
#endif
|
@ -991,7 +991,8 @@ static void scanPackets(TransportData* transport,
|
||||
Arena& arena,
|
||||
NetworkAddress const& peerAddress,
|
||||
ProtocolVersion peerProtocolVersion,
|
||||
Future<Void> disconnect) {
|
||||
Future<Void> disconnect,
|
||||
bool isStableConnection) {
|
||||
// Find each complete packet in the given byte range and queue a ready task to deliver it.
|
||||
// Remove the complete packets from the range by increasing unprocessed_begin.
|
||||
// There won't be more than 64K of data plus one packet, so this shouldn't take a long time.
|
||||
@ -1030,7 +1031,7 @@ static void scanPackets(TransportData* transport,
|
||||
|
||||
if (checksumEnabled) {
|
||||
bool isBuggifyEnabled = false;
|
||||
if (g_network->isSimulated() &&
|
||||
if (g_network->isSimulated() && !isStableConnection &&
|
||||
g_network->now() - g_simulator.lastConnectionFailure > g_simulator.connectionFailuresDisableDuration &&
|
||||
BUGGIFY_WITH_PROB(0.0001)) {
|
||||
g_simulator.lastConnectionFailure = g_network->now();
|
||||
@ -1057,7 +1058,8 @@ static void scanPackets(TransportData* transport,
|
||||
if (isBuggifyEnabled) {
|
||||
TraceEvent(SevInfo, "ChecksumMismatchExp")
|
||||
.detail("PacketChecksum", packetChecksum)
|
||||
.detail("CalculatedChecksum", calculatedChecksum);
|
||||
.detail("CalculatedChecksum", calculatedChecksum)
|
||||
.detail("PeerAddress", peerAddress.toString());
|
||||
} else {
|
||||
TraceEvent(SevWarnAlways, "ChecksumMismatchUnexp")
|
||||
.detail("PacketChecksum", packetChecksum)
|
||||
@ -1305,7 +1307,8 @@ ACTOR static Future<Void> connectionReader(TransportData* transport,
|
||||
arena,
|
||||
peerAddress,
|
||||
peerProtocolVersion,
|
||||
peer->disconnect.getFuture());
|
||||
peer->disconnect.getFuture(),
|
||||
g_network->isSimulated() && conn->isStableConnection());
|
||||
} else {
|
||||
unprocessed_begin = unprocessed_end;
|
||||
peer->resetPing.trigger();
|
||||
@ -1364,6 +1367,11 @@ ACTOR static Future<Void> listen(TransportData* self, NetworkAddress listenAddr)
|
||||
state ActorCollectionNoErrors
|
||||
incoming; // Actors monitoring incoming connections that haven't yet been associated with a peer
|
||||
state Reference<IListener> listener = INetworkConnections::net()->listen(listenAddr);
|
||||
if (!g_network->isSimulated() && self->localAddresses.address.port == 0) {
|
||||
TraceEvent(SevInfo, "UpdatingListenAddress")
|
||||
.detail("AssignedListenAddress", listener->getListenAddress().toString());
|
||||
self->localAddresses.address = listener->getListenAddress();
|
||||
}
|
||||
state uint64_t connectionCount = 0;
|
||||
try {
|
||||
loop {
|
||||
|
@ -20,6 +20,7 @@
|
||||
|
||||
#include <cinttypes>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
|
||||
#include "contrib/fmt-8.1.1/include/fmt/format.h"
|
||||
#include "fdbrpc/simulator.h"
|
||||
@ -121,20 +122,24 @@ void ISimulator::displayWorkers() const {
|
||||
int openCount = 0;
|
||||
|
||||
struct SimClogging {
|
||||
double getSendDelay(NetworkAddress from, NetworkAddress to) const { return halfLatency(); }
|
||||
double getSendDelay(NetworkAddress from, NetworkAddress to, bool stableConnection = false) const {
|
||||
// stable connection here means it's a local connection between processes on the same machine
|
||||
// we expect it to have much lower latency
|
||||
return (stableConnection ? 0.1 : 1.0) * halfLatency();
|
||||
}
|
||||
|
||||
double getRecvDelay(NetworkAddress from, NetworkAddress to) {
|
||||
double getRecvDelay(NetworkAddress from, NetworkAddress to, bool stableConnection = false) {
|
||||
auto pair = std::make_pair(from.ip, to.ip);
|
||||
|
||||
double tnow = now();
|
||||
double t = tnow + halfLatency();
|
||||
if (!g_simulator.speedUpSimulation)
|
||||
double t = tnow + (stableConnection ? 0.1 : 1.0) * halfLatency();
|
||||
if (!g_simulator.speedUpSimulation && !stableConnection)
|
||||
t += clogPairLatency[pair];
|
||||
|
||||
if (!g_simulator.speedUpSimulation && clogPairUntil.count(pair))
|
||||
if (!g_simulator.speedUpSimulation && !stableConnection && clogPairUntil.count(pair))
|
||||
t = std::max(t, clogPairUntil[pair]);
|
||||
|
||||
if (!g_simulator.speedUpSimulation && clogRecvUntil.count(to.ip))
|
||||
if (!g_simulator.speedUpSimulation && !stableConnection && clogRecvUntil.count(to.ip))
|
||||
t = std::max(t, clogRecvUntil[to.ip]);
|
||||
|
||||
return t - tnow;
|
||||
@ -182,8 +187,8 @@ SimClogging g_clogging;
|
||||
|
||||
struct Sim2Conn final : IConnection, ReferenceCounted<Sim2Conn> {
|
||||
Sim2Conn(ISimulator::ProcessInfo* process)
|
||||
: opened(false), closedByCaller(false), process(process), dbgid(deterministicRandom()->randomUniqueID()),
|
||||
stopReceive(Never()) {
|
||||
: opened(false), closedByCaller(false), stableConnection(false), process(process),
|
||||
dbgid(deterministicRandom()->randomUniqueID()), stopReceive(Never()) {
|
||||
pipes = sender(this) && receiver(this);
|
||||
}
|
||||
|
||||
@ -202,7 +207,18 @@ struct Sim2Conn final : IConnection, ReferenceCounted<Sim2Conn> {
|
||||
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);
|
||||
// options like clogging or bitsflip are disabled for stable connections
|
||||
stableConnection = std::any_of(process->childs.begin(),
|
||||
process->childs.end(),
|
||||
[&](ISimulator::ProcessInfo* child) { return child && child == peerProcess; }) ||
|
||||
std::any_of(peerProcess->childs.begin(),
|
||||
peerProcess->childs.end(),
|
||||
[&](ISimulator::ProcessInfo* child) { return child && child == process; });
|
||||
|
||||
TraceEvent("Sim2Connection")
|
||||
.detail("SendBufSize", sendBufSize)
|
||||
.detail("Latency", latency)
|
||||
.detail("StableConnection", stableConnection);
|
||||
}
|
||||
|
||||
~Sim2Conn() { ASSERT_ABORT(!opened || closedByCaller); }
|
||||
@ -222,6 +238,8 @@ struct Sim2Conn final : IConnection, ReferenceCounted<Sim2Conn> {
|
||||
|
||||
bool isPeerGone() const { return !peer || peerProcess->failed; }
|
||||
|
||||
bool isStableConnection() const override { return stableConnection; }
|
||||
|
||||
void peerClosed() {
|
||||
leakedConnectionTracker = trackLeakedConnection(this);
|
||||
stopReceive = delay(1.0);
|
||||
@ -249,7 +267,7 @@ struct Sim2Conn final : IConnection, ReferenceCounted<Sim2Conn> {
|
||||
ASSERT(limit > 0);
|
||||
|
||||
int toSend = 0;
|
||||
if (BUGGIFY) {
|
||||
if (BUGGIFY && !stableConnection) {
|
||||
toSend = std::min(limit, buffer->bytes_written - buffer->bytes_sent);
|
||||
} else {
|
||||
for (auto p = buffer; p; p = p->next) {
|
||||
@ -262,7 +280,7 @@ struct Sim2Conn final : IConnection, ReferenceCounted<Sim2Conn> {
|
||||
}
|
||||
}
|
||||
ASSERT(toSend);
|
||||
if (BUGGIFY)
|
||||
if (BUGGIFY && !stableConnection)
|
||||
toSend = std::min(toSend, deterministicRandom()->randomInt(0, 1000));
|
||||
|
||||
if (!peer)
|
||||
@ -286,7 +304,7 @@ struct Sim2Conn final : IConnection, ReferenceCounted<Sim2Conn> {
|
||||
NetworkAddress getPeerAddress() const override { return peerEndpoint; }
|
||||
UID getDebugID() const override { return dbgid; }
|
||||
|
||||
bool opened, closedByCaller;
|
||||
bool opened, closedByCaller, stableConnection;
|
||||
|
||||
private:
|
||||
ISimulator::ProcessInfo *process, *peerProcess;
|
||||
@ -336,10 +354,12 @@ private:
|
||||
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(delay(g_clogging.getSendDelay(
|
||||
self->process->address, self->peerProcess->address, self->isStableConnection())));
|
||||
wait(g_simulator.onProcess(self->process));
|
||||
ASSERT(g_simulator.getCurrentProcess() == self->process);
|
||||
wait(delay(g_clogging.getRecvDelay(self->process->address, self->peerProcess->address)));
|
||||
wait(delay(g_clogging.getRecvDelay(
|
||||
self->process->address, self->peerProcess->address, self->isStableConnection())));
|
||||
ASSERT(g_simulator.getCurrentProcess() == self->process);
|
||||
if (self->stopReceive.isReady()) {
|
||||
wait(Future<Void>(Never()));
|
||||
@ -389,7 +409,9 @@ private:
|
||||
}
|
||||
|
||||
void rollRandomClose() {
|
||||
if (now() - g_simulator.lastConnectionFailure > g_simulator.connectionFailuresDisableDuration &&
|
||||
// make sure connections between parenta and their childs are not closed
|
||||
if (!stableConnection &&
|
||||
now() - g_simulator.lastConnectionFailure > g_simulator.connectionFailuresDisableDuration &&
|
||||
deterministicRandom()->random01() < .00001) {
|
||||
g_simulator.lastConnectionFailure = now();
|
||||
double a = deterministicRandom()->random01(), b = deterministicRandom()->random01();
|
||||
@ -1101,6 +1123,10 @@ public:
|
||||
if (mustBeDurable || deterministicRandom()->random01() < 0.5) {
|
||||
state ISimulator::ProcessInfo* currentProcess = g_simulator.getCurrentProcess();
|
||||
state TaskPriority currentTaskID = g_network->getCurrentTask();
|
||||
TraceEvent(SevDebug, "Sim2DeleteFileImpl")
|
||||
.detail("CurrentProcess", currentProcess->toString())
|
||||
.detail("Filename", filename)
|
||||
.detail("Durable", mustBeDurable);
|
||||
wait(g_simulator.onMachine(currentProcess));
|
||||
try {
|
||||
wait(::delay(0.05 * deterministicRandom()->random01()));
|
||||
@ -1118,6 +1144,9 @@ public:
|
||||
throw err;
|
||||
}
|
||||
} else {
|
||||
TraceEvent(SevDebug, "Sim2DeleteFileImplNonDurable")
|
||||
.detail("Filename", filename)
|
||||
.detail("Durable", mustBeDurable);
|
||||
TEST(true); // Simulated non-durable delete
|
||||
return Void();
|
||||
}
|
||||
@ -1163,6 +1192,9 @@ public:
|
||||
MachineInfo& machine = machines[locality.machineId().get()];
|
||||
if (!machine.machineId.present())
|
||||
machine.machineId = locality.machineId();
|
||||
if (port == 0 && std::string(name) == "remote flow process") {
|
||||
port = machine.getRandomPort();
|
||||
}
|
||||
for (int i = 0; i < machine.processes.size(); i++) {
|
||||
if (machine.processes[i]->locality.machineId() !=
|
||||
locality.machineId()) { // SOMEDAY: compute ip from locality to avoid this check
|
||||
@ -1220,6 +1252,11 @@ public:
|
||||
.detail("Excluded", m->excluded)
|
||||
.detail("Cleared", m->cleared);
|
||||
|
||||
if (std::string(name) == "remote flow process") {
|
||||
protectedAddresses.insert(m->address);
|
||||
TraceEvent(SevDebug, "NewFlowProcessProtected").detail("Address", m->address);
|
||||
}
|
||||
|
||||
// FIXME: Sometimes, connections to/from this process will explicitly close
|
||||
|
||||
return m;
|
||||
@ -1497,6 +1534,7 @@ public:
|
||||
.detail("MachineId", p->locality.machineId());
|
||||
currentlyRebootingProcesses.insert(std::pair<NetworkAddress, ProcessInfo*>(p->address, p));
|
||||
std::vector<ProcessInfo*>& processes = machines[p->locality.machineId().get()].processes;
|
||||
machines[p->locality.machineId().get()].removeRemotePort(p->address.port);
|
||||
if (p != processes.back()) {
|
||||
auto it = std::find(processes.begin(), processes.end(), p);
|
||||
std::swap(*it, processes.back());
|
||||
@ -1520,7 +1558,8 @@ public:
|
||||
.detail("Protected", protectedAddresses.count(machine->address))
|
||||
.backtrace();
|
||||
// This will remove all the "tracked" messages that came from the machine being killed
|
||||
latestEventCache.clear();
|
||||
if (std::string(machine->name) != "remote flow process")
|
||||
latestEventCache.clear();
|
||||
machine->failed = true;
|
||||
} else if (kt == InjectFaults) {
|
||||
TraceEvent(SevWarn, "FaultMachine")
|
||||
@ -1548,7 +1587,8 @@ public:
|
||||
} else {
|
||||
ASSERT(false);
|
||||
}
|
||||
ASSERT(!protectedAddresses.count(machine->address) || machine->rebooting);
|
||||
ASSERT(!protectedAddresses.count(machine->address) || machine->rebooting ||
|
||||
std::string(machine->name) == "remote flow process");
|
||||
}
|
||||
void rebootProcess(ProcessInfo* process, KillType kt) override {
|
||||
if (kt == RebootProcessAndDelete && protectedAddresses.count(process->address)) {
|
||||
@ -2390,8 +2430,19 @@ ACTOR void doReboot(ISimulator::ProcessInfo* p, ISimulator::KillType kt) {
|
||||
kt ==
|
||||
ISimulator::RebootProcessAndDelete); // Simulated process rebooted with data and coordination state deletion
|
||||
|
||||
if (p->rebooting || !p->isReliable())
|
||||
if (p->rebooting || !p->isReliable()) {
|
||||
TraceEvent(SevDebug, "DoRebootFailed")
|
||||
.detail("Rebooting", p->rebooting)
|
||||
.detail("Reliable", p->isReliable());
|
||||
return;
|
||||
} else if (std::string(p->name) == "remote flow process") {
|
||||
TraceEvent(SevDebug, "DoRebootFailed").detail("Name", p->name).detail("Address", p->address);
|
||||
return;
|
||||
} else if (p->getChilds().size()) {
|
||||
TraceEvent(SevDebug, "DoRebootFailedOnParentProcess").detail("Address", p->address);
|
||||
return;
|
||||
}
|
||||
|
||||
TraceEvent("RebootingProcess")
|
||||
.detail("KillType", kt)
|
||||
.detail("Address", p->address)
|
||||
|
@ -21,6 +21,7 @@
|
||||
#ifndef FLOW_SIMULATOR_H
|
||||
#define FLOW_SIMULATOR_H
|
||||
#include "flow/ProtocolVersion.h"
|
||||
#include <algorithm>
|
||||
#include <string>
|
||||
#pragma once
|
||||
|
||||
@ -87,6 +88,8 @@ public:
|
||||
|
||||
ProtocolVersion protocolVersion;
|
||||
|
||||
std::vector<ProcessInfo*> childs;
|
||||
|
||||
ProcessInfo(const char* name,
|
||||
LocalityData locality,
|
||||
ProcessClass startingClass,
|
||||
@ -117,6 +120,7 @@ public:
|
||||
<< " 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 {
|
||||
@ -202,7 +206,30 @@ public:
|
||||
std::set<std::string> closingFiles;
|
||||
Optional<Standalone<StringRef>> machineId;
|
||||
|
||||
MachineInfo() : machineProcess(nullptr) {}
|
||||
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()); }
|
||||
|
@ -98,6 +98,8 @@ set(FDBSERVER_SRCS
|
||||
Ratekeeper.h
|
||||
RatekeeperInterface.h
|
||||
RecoveryState.h
|
||||
RemoteIKeyValueStore.actor.h
|
||||
RemoteIKeyValueStore.actor.cpp
|
||||
ResolutionBalancer.actor.cpp
|
||||
ResolutionBalancer.actor.h
|
||||
Resolver.actor.cpp
|
||||
|
@ -18,17 +18,30 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "flow/TLSConfig.actor.h"
|
||||
#include "flow/Trace.h"
|
||||
#include "flow/Platform.h"
|
||||
#include "flow/flow.h"
|
||||
#include "flow/genericactors.actor.h"
|
||||
#include "flow/network.h"
|
||||
#include "fdbrpc/FlowProcess.actor.h"
|
||||
#include "fdbrpc/Net2FileSystem.h"
|
||||
#include "fdbrpc/simulator.h"
|
||||
#include "fdbclient/WellKnownEndpoints.h"
|
||||
#include "fdbclient/versions.h"
|
||||
#include "fdbserver/CoroFlow.h"
|
||||
#include "fdbserver/FDBExecHelper.actor.h"
|
||||
#include "fdbserver/Knobs.h"
|
||||
#include "fdbserver/RemoteIKeyValueStore.actor.h"
|
||||
|
||||
#if !defined(_WIN32) && !defined(__APPLE__) && !defined(__INTEL_COMPILER)
|
||||
#define BOOST_SYSTEM_NO_LIB
|
||||
#define BOOST_DATE_TIME_NO_LIB
|
||||
#define BOOST_REGEX_NO_LIB
|
||||
#include <boost/process.hpp>
|
||||
#endif
|
||||
#include "fdbserver/FDBExecHelper.actor.h"
|
||||
#include "flow/Trace.h"
|
||||
#include "flow/flow.h"
|
||||
#include "fdbclient/versions.h"
|
||||
#include "fdbserver/Knobs.h"
|
||||
#include <boost/algorithm/string.hpp>
|
||||
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
ExecCmdValueString::ExecCmdValueString(StringRef pCmdValueString) {
|
||||
@ -90,12 +103,138 @@ void ExecCmdValueString::dbgPrint() const {
|
||||
return;
|
||||
}
|
||||
|
||||
ACTOR void destoryChildProcess(Future<Void> parentSSClosed, ISimulator::ProcessInfo* childInfo, std::string message) {
|
||||
// This code path should be bug free
|
||||
wait(parentSSClosed);
|
||||
TraceEvent(SevDebug, message.c_str()).log();
|
||||
// This one is root cause for most failures, make sure it's okay to destory
|
||||
g_pSimulator->destroyProcess(childInfo);
|
||||
// Explicitly reset the connection with the child process in case re-spawn very quickly
|
||||
FlowTransport::transport().resetConnection(childInfo->address);
|
||||
}
|
||||
|
||||
ACTOR Future<int> spawnSimulated(std::vector<std::string> paramList,
|
||||
double maxWaitTime,
|
||||
bool isSync,
|
||||
double maxSimDelayTime,
|
||||
IClosable* parent) {
|
||||
state ISimulator::ProcessInfo* self = g_pSimulator->getCurrentProcess();
|
||||
state ISimulator::ProcessInfo* child;
|
||||
|
||||
state std::string role;
|
||||
state std::string addr;
|
||||
state std::string flowProcessName;
|
||||
state Endpoint parentProcessEndpoint;
|
||||
state int i = 0;
|
||||
// fdbserver -r flowprocess --process-name ikvs --process-endpoint ip:port,token,id
|
||||
for (; i < paramList.size(); i++) {
|
||||
if (paramList.size() > i + 1) {
|
||||
// temporary args parser that only supports the flowprocess role
|
||||
if (paramList[i] == "-r") {
|
||||
role = paramList[i + 1];
|
||||
} else if (paramList[i] == "-p" || paramList[i] == "--public_address") {
|
||||
addr = paramList[i + 1];
|
||||
} else if (paramList[i] == "--process-name") {
|
||||
flowProcessName = paramList[i + 1];
|
||||
} else if (paramList[i] == "--process-endpoint") {
|
||||
state std::vector<std::string> addressArray;
|
||||
boost::split(addressArray, paramList[i + 1], [](char c) { return c == ','; });
|
||||
if (addressArray.size() != 3) {
|
||||
std::cerr << "Invalid argument, expected 3 elements in --process-endpoint got "
|
||||
<< addressArray.size() << std::endl;
|
||||
flushAndExit(FDB_EXIT_ERROR);
|
||||
}
|
||||
try {
|
||||
auto addr = NetworkAddress::parse(addressArray[0]);
|
||||
uint64_t fst = std::stoul(addressArray[1]);
|
||||
uint64_t snd = std::stoul(addressArray[2]);
|
||||
UID token(fst, snd);
|
||||
NetworkAddressList l;
|
||||
l.address = addr;
|
||||
parentProcessEndpoint = Endpoint(l, token);
|
||||
} catch (Error& e) {
|
||||
std::cerr << "Could not parse network address " << addressArray[0] << std::endl;
|
||||
flushAndExit(FDB_EXIT_ERROR);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
state int result = 0;
|
||||
child = g_pSimulator->newProcess("remote flow process",
|
||||
self->address.ip,
|
||||
0,
|
||||
self->address.isTLS(),
|
||||
self->addresses.secondaryAddress.present() ? 2 : 1,
|
||||
self->locality,
|
||||
ProcessClass(ProcessClass::UnsetClass, ProcessClass::AutoSource),
|
||||
self->dataFolder,
|
||||
self->coordinationFolder, // do we need to customize this coordination folder path?
|
||||
self->protocolVersion);
|
||||
wait(g_pSimulator->onProcess(child));
|
||||
state Future<ISimulator::KillType> onShutdown = child->onShutdown();
|
||||
state Future<ISimulator::KillType> parentShutdown = self->onShutdown();
|
||||
state Future<Void> flowProcessF;
|
||||
|
||||
try {
|
||||
TraceEvent(SevDebug, "SpawnedChildProcess")
|
||||
.detail("Child", child->toString())
|
||||
.detail("Parent", self->toString());
|
||||
std::string role = "";
|
||||
std::string addr = "";
|
||||
for (int i = 0; i < paramList.size(); i++) {
|
||||
if (paramList.size() > i + 1 && paramList[i] == "-r") {
|
||||
role = paramList[i + 1];
|
||||
}
|
||||
}
|
||||
if (role == "flowprocess" && !parentShutdown.isReady()) {
|
||||
self->childs.push_back(child);
|
||||
state Future<Void> parentSSClosed = parent->onClosed();
|
||||
FlowTransport::createInstance(false, 1, WLTOKEN_RESERVED_COUNT);
|
||||
FlowTransport::transport().bind(child->address, child->address);
|
||||
Sim2FileSystem::newFileSystem();
|
||||
ProcessFactory<KeyValueStoreProcess>(flowProcessName.c_str());
|
||||
flowProcessF = runFlowProcess(flowProcessName, parentProcessEndpoint);
|
||||
|
||||
choose {
|
||||
when(wait(flowProcessF)) {
|
||||
TraceEvent(SevDebug, "ChildProcessKilled").log();
|
||||
wait(g_pSimulator->onProcess(self));
|
||||
TraceEvent(SevDebug, "BackOnParentProcess").detail("Result", std::to_string(result));
|
||||
destoryChildProcess(parentSSClosed, child, "StorageServerReceivedClosedMessage");
|
||||
}
|
||||
when(wait(success(onShutdown))) {
|
||||
ASSERT(false);
|
||||
// In prod, we use prctl to bind parent and child processes to die together
|
||||
// In simulation, we simply disable killing parent or child processes as we cannot use the same
|
||||
// mechanism here
|
||||
}
|
||||
when(wait(success(parentShutdown))) {
|
||||
ASSERT(false);
|
||||
// Parent process is not killed, see above
|
||||
}
|
||||
}
|
||||
} else {
|
||||
ASSERT(false);
|
||||
}
|
||||
} catch (Error& e) {
|
||||
TraceEvent(SevError, "RemoteIKVSDied").errorUnsuppressed(e);
|
||||
result = -1;
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
#if defined(_WIN32) || defined(__APPLE__) || defined(__INTEL_COMPILER)
|
||||
ACTOR Future<int> spawnProcess(std::string binPath,
|
||||
std::vector<std::string> paramList,
|
||||
double maxWaitTime,
|
||||
bool isSync,
|
||||
double maxSimDelayTime) {
|
||||
double maxSimDelayTime,
|
||||
IClosable* parent) {
|
||||
if (g_network->isSimulated() && getExecPath() == binPath) {
|
||||
int res = wait(spawnSimulated(paramList, maxWaitTime, isSync, maxSimDelayTime, parent));
|
||||
return res;
|
||||
}
|
||||
wait(delay(0.0));
|
||||
return 0;
|
||||
}
|
||||
@ -125,6 +264,9 @@ static auto fork_child(const std::string& path, std::vector<char*>& paramList) {
|
||||
}
|
||||
|
||||
static void setupTraceWithOutput(TraceEvent& event, size_t bytesRead, char* outputBuffer) {
|
||||
// get some errors printed for spawned process
|
||||
std::cout << "Output bytesRead: " << bytesRead << std::endl;
|
||||
std::cout << "output buffer: " << std::string(outputBuffer) << std::endl;
|
||||
if (bytesRead == 0)
|
||||
return;
|
||||
ASSERT(bytesRead <= SERVER_KNOBS->MAX_FORKED_PROCESS_OUTPUT);
|
||||
@ -139,7 +281,12 @@ ACTOR Future<int> spawnProcess(std::string path,
|
||||
std::vector<std::string> args,
|
||||
double maxWaitTime,
|
||||
bool isSync,
|
||||
double maxSimDelayTime) {
|
||||
double maxSimDelayTime,
|
||||
IClosable* parent) {
|
||||
if (g_network->isSimulated() && getExecPath() == path) {
|
||||
int res = wait(spawnSimulated(args, maxWaitTime, isSync, maxSimDelayTime, parent));
|
||||
return res;
|
||||
}
|
||||
// for async calls in simulator, always delay by a deterministic amount of time and then
|
||||
// do the call synchronously, otherwise the predictability of the simulator breaks
|
||||
if (!isSync && g_network->isSimulated()) {
|
||||
@ -182,7 +329,7 @@ ACTOR Future<int> spawnProcess(std::string path,
|
||||
int flags = fcntl(readFD.get(), F_GETFL, 0);
|
||||
fcntl(readFD.get(), F_SETFL, flags | O_NONBLOCK);
|
||||
while (true) {
|
||||
if (runTime > maxWaitTime) {
|
||||
if (maxWaitTime >= 0 && runTime > maxWaitTime) {
|
||||
// timing out
|
||||
|
||||
TraceEvent(SevWarnAlways, "SpawnProcessFailure")
|
||||
@ -203,7 +350,6 @@ ACTOR Future<int> spawnProcess(std::string path,
|
||||
break;
|
||||
bytesRead += bytes;
|
||||
}
|
||||
|
||||
if (err < 0) {
|
||||
TraceEvent event(SevWarnAlways, "SpawnProcessFailure");
|
||||
setupTraceWithOutput(event, bytesRead, outputBuffer);
|
||||
|
@ -63,16 +63,19 @@ private: // data
|
||||
StringRef binaryPath;
|
||||
};
|
||||
|
||||
class IClosable; // Forward declaration
|
||||
|
||||
// FIXME: move this function to a common location
|
||||
// spawns a process pointed by `binPath` and the arguments provided at `paramList`,
|
||||
// if the process spawned takes more than `maxWaitTime` then it will be killed
|
||||
// if isSync is set to true then the process will be synchronously executed
|
||||
// if async and in simulator then delay spawning the process to max of maxSimDelayTime
|
||||
// if the process spawned takes more than `maxWaitTime` then it will be killed, if `maxWaitTime` < 0, then there won't
|
||||
// be timeout if isSync is set to true then the process will be synchronously executed if async and in simulator then
|
||||
// delay spawning the process to max of maxSimDelayTime
|
||||
ACTOR Future<int> spawnProcess(std::string binPath,
|
||||
std::vector<std::string> paramList,
|
||||
double maxWaitTime,
|
||||
bool isSync,
|
||||
double maxSimDelayTime);
|
||||
double maxSimDelayTime,
|
||||
IClosable* parent = nullptr);
|
||||
|
||||
// helper to run all the work related to running the exec command
|
||||
ACTOR Future<int> execHelper(ExecCmdValueString* execArg, UID snapUID, std::string folder, std::string role);
|
||||
|
@ -159,12 +159,23 @@ extern IKeyValueStore* keyValueStoreLogSystem(class IDiskQueue* queue,
|
||||
bool replaceContent,
|
||||
bool exactRecovery);
|
||||
|
||||
extern IKeyValueStore* openRemoteKVStore(KeyValueStoreType storeType,
|
||||
std::string const& filename,
|
||||
UID logID,
|
||||
int64_t memoryLimit,
|
||||
bool checkChecksums = false,
|
||||
bool checkIntegrity = false);
|
||||
|
||||
inline IKeyValueStore* openKVStore(KeyValueStoreType storeType,
|
||||
std::string const& filename,
|
||||
UID logID,
|
||||
int64_t memoryLimit,
|
||||
bool checkChecksums = false,
|
||||
bool checkIntegrity = false) {
|
||||
bool checkIntegrity = false,
|
||||
bool openRemotely = false) {
|
||||
if (openRemotely) {
|
||||
return openRemoteKVStore(storeType, filename, logID, memoryLimit, checkChecksums, checkIntegrity);
|
||||
}
|
||||
switch (storeType) {
|
||||
case KeyValueStoreType::SSD_BTREE_V1:
|
||||
return keyValueStoreSQLite(filename, logID, KeyValueStoreType::SSD_BTREE_V1, false, checkIntegrity);
|
||||
|
246
fdbserver/RemoteIKeyValueStore.actor.cpp
Normal file
246
fdbserver/RemoteIKeyValueStore.actor.cpp
Normal file
@ -0,0 +1,246 @@
|
||||
/*
|
||||
* RemoteIKeyValueStore.actor.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include "flow/ActorCollection.h"
|
||||
#include "flow/Error.h"
|
||||
#include "flow/Platform.h"
|
||||
#include "flow/Trace.h"
|
||||
#include "fdbrpc/FlowProcess.actor.h"
|
||||
#include "fdbrpc/fdbrpc.h"
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "fdbserver/FDBExecHelper.actor.h"
|
||||
#include "fdbserver/Knobs.h"
|
||||
#include "fdbserver/RemoteIKeyValueStore.actor.h"
|
||||
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
StringRef KeyValueStoreProcess::_name = "KeyValueStoreProcess"_sr;
|
||||
|
||||
// A guard for guaranteed killing of machine after runIKVS returns
|
||||
struct AfterReturn {
|
||||
IKeyValueStore* kvStore;
|
||||
UID id;
|
||||
AfterReturn() : kvStore(nullptr) {}
|
||||
AfterReturn(IKeyValueStore* store, UID& uid) : kvStore(store), id(uid) {}
|
||||
~AfterReturn() {
|
||||
TraceEvent(SevDebug, "RemoteKVStoreAfterReturn")
|
||||
.detail("Valid", kvStore != nullptr ? "True" : "False")
|
||||
.detail("UID", id)
|
||||
.log();
|
||||
if (kvStore != nullptr) {
|
||||
kvStore->close();
|
||||
}
|
||||
}
|
||||
// called when we already explicitly closed the kv store
|
||||
void invalidate() { kvStore = nullptr; }
|
||||
};
|
||||
|
||||
ACTOR void sendCommitReply(IKVSCommitRequest commitReq, IKeyValueStore* kvStore, Future<Void> onClosed) {
|
||||
try {
|
||||
choose {
|
||||
when(wait(onClosed)) { commitReq.reply.sendError(remote_kvs_cancelled()); }
|
||||
when(wait(kvStore->commit(commitReq.sequential))) {
|
||||
StorageBytes storageBytes = kvStore->getStorageBytes();
|
||||
commitReq.reply.send(IKVSCommitReply(storageBytes));
|
||||
}
|
||||
}
|
||||
} catch (Error& e) {
|
||||
TraceEvent(SevDebug, "RemoteKVSCommitReplyError").errorUnsuppressed(e);
|
||||
commitReq.reply.sendError(e.code() == error_code_actor_cancelled ? remote_kvs_cancelled() : e);
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR template <class T>
|
||||
Future<Void> cancellableForwardPromise(ReplyPromise<T> output, Future<T> input) {
|
||||
try {
|
||||
T value = wait(input);
|
||||
output.send(value);
|
||||
} catch (Error& e) {
|
||||
TraceEvent(SevDebug, "CancellableForwardPromiseError").errorUnsuppressed(e).backtrace();
|
||||
output.sendError(e.code() == error_code_actor_cancelled ? remote_kvs_cancelled() : e);
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR Future<Void> runIKVS(OpenKVStoreRequest openReq, IKVSInterface ikvsInterface) {
|
||||
state IKeyValueStore* kvStore = openKVStore(openReq.storeType,
|
||||
openReq.filename,
|
||||
openReq.logID,
|
||||
openReq.memoryLimit,
|
||||
openReq.checkChecksums,
|
||||
openReq.checkIntegrity);
|
||||
state UID kvsId(ikvsInterface.id());
|
||||
state ActorCollection actors(false);
|
||||
state AfterReturn guard(kvStore, kvsId);
|
||||
state Promise<Void> onClosed;
|
||||
TraceEvent(SevDebug, "RemoteKVStoreInitializing").detail("UID", kvsId);
|
||||
wait(kvStore->init());
|
||||
openReq.reply.send(ikvsInterface);
|
||||
TraceEvent(SevInfo, "RemoteKVStoreInitialized").detail("IKVSInterfaceUID", kvsId);
|
||||
|
||||
loop {
|
||||
try {
|
||||
choose {
|
||||
when(IKVSGetValueRequest getReq = waitNext(ikvsInterface.getValue.getFuture())) {
|
||||
actors.add(cancellableForwardPromise(getReq.reply,
|
||||
kvStore->readValue(getReq.key, getReq.type, getReq.debugID)));
|
||||
}
|
||||
when(IKVSSetRequest req = waitNext(ikvsInterface.set.getFuture())) { kvStore->set(req.keyValue); }
|
||||
when(IKVSClearRequest req = waitNext(ikvsInterface.clear.getFuture())) { kvStore->clear(req.range); }
|
||||
when(IKVSCommitRequest commitReq = waitNext(ikvsInterface.commit.getFuture())) {
|
||||
sendCommitReply(commitReq, kvStore, onClosed.getFuture());
|
||||
}
|
||||
when(IKVSReadValuePrefixRequest readPrefixReq = waitNext(ikvsInterface.readValuePrefix.getFuture())) {
|
||||
actors.add(cancellableForwardPromise(
|
||||
readPrefixReq.reply,
|
||||
kvStore->readValuePrefix(
|
||||
readPrefixReq.key, readPrefixReq.maxLength, readPrefixReq.type, readPrefixReq.debugID)));
|
||||
}
|
||||
when(IKVSReadRangeRequest readRangeReq = waitNext(ikvsInterface.readRange.getFuture())) {
|
||||
actors.add(cancellableForwardPromise(
|
||||
readRangeReq.reply,
|
||||
fmap(
|
||||
[](const RangeResult& result) { return IKVSReadRangeReply(result); },
|
||||
kvStore->readRange(
|
||||
readRangeReq.keys, readRangeReq.rowLimit, readRangeReq.byteLimit, readRangeReq.type))));
|
||||
}
|
||||
when(IKVSGetStorageByteRequest req = waitNext(ikvsInterface.getStorageBytes.getFuture())) {
|
||||
StorageBytes storageBytes = kvStore->getStorageBytes();
|
||||
req.reply.send(storageBytes);
|
||||
}
|
||||
when(IKVSGetErrorRequest getFutureReq = waitNext(ikvsInterface.getError.getFuture())) {
|
||||
actors.add(cancellableForwardPromise(getFutureReq.reply, kvStore->getError()));
|
||||
}
|
||||
when(IKVSOnClosedRequest onClosedReq = waitNext(ikvsInterface.onClosed.getFuture())) {
|
||||
// onClosed request is not cancelled even this actor is cancelled
|
||||
forwardPromise(onClosedReq.reply, kvStore->onClosed());
|
||||
}
|
||||
when(IKVSDisposeRequest disposeReq = waitNext(ikvsInterface.dispose.getFuture())) {
|
||||
TraceEvent(SevDebug, "RemoteIKVSDisposeReceivedRequest").detail("UID", kvsId);
|
||||
kvStore->dispose();
|
||||
guard.invalidate();
|
||||
onClosed.send(Void());
|
||||
return Void();
|
||||
}
|
||||
when(IKVSCloseRequest closeReq = waitNext(ikvsInterface.close.getFuture())) {
|
||||
TraceEvent(SevDebug, "RemoteIKVSCloseReceivedRequest").detail("UID", kvsId);
|
||||
kvStore->close();
|
||||
guard.invalidate();
|
||||
onClosed.send(Void());
|
||||
return Void();
|
||||
}
|
||||
}
|
||||
} catch (Error& e) {
|
||||
if (e.code() == error_code_actor_cancelled) {
|
||||
TraceEvent(SevInfo, "RemoteKVStoreCancelled").detail("UID", kvsId).backtrace();
|
||||
onClosed.send(Void());
|
||||
return Void();
|
||||
} else {
|
||||
TraceEvent(SevError, "RemoteKVStoreError").error(e).detail("UID", kvsId).backtrace();
|
||||
throw;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR static Future<int> flowProcessRunner(RemoteIKeyValueStore* self, Promise<Void> ready) {
|
||||
state FlowProcessInterface processInterface;
|
||||
state Future<int> process;
|
||||
|
||||
auto path = abspath(getExecPath());
|
||||
auto endpoint = processInterface.registerProcess.getEndpoint();
|
||||
auto address = endpoint.addresses.address.toString();
|
||||
auto token = endpoint.token;
|
||||
|
||||
// port 0 means we will find a random available port number for it
|
||||
std::string flowProcessAddr = g_network->getLocalAddress().ip.toString().append(":0");
|
||||
std::vector<std::string> args = { "bin/fdbserver",
|
||||
"-r",
|
||||
"flowprocess",
|
||||
"-C",
|
||||
SERVER_KNOBS->CONN_FILE,
|
||||
"--logdir",
|
||||
SERVER_KNOBS->LOG_DIRECTORY,
|
||||
"-p",
|
||||
flowProcessAddr,
|
||||
"--process-name",
|
||||
KeyValueStoreProcess::_name.toString(),
|
||||
"--process-endpoint",
|
||||
format("%s,%lu,%lu", address.c_str(), token.first(), token.second()) };
|
||||
// For remote IKV store, we need to make sure the shutdown signal is sent back until we can destroy it in the
|
||||
// simulation
|
||||
process = spawnProcess(path, args, -1.0, false, 0.01 /*not used*/, self);
|
||||
choose {
|
||||
when(FlowProcessRegistrationRequest req = waitNext(processInterface.registerProcess.getFuture())) {
|
||||
self->consumeInterface(req.flowProcessInterface);
|
||||
ready.send(Void());
|
||||
}
|
||||
when(int res = wait(process)) {
|
||||
// 0 means process normally shut down; non-zero means errors
|
||||
// process should not shut down normally before not ready
|
||||
ASSERT(res);
|
||||
return res;
|
||||
}
|
||||
}
|
||||
int res = wait(process);
|
||||
return res;
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> initializeRemoteKVStore(RemoteIKeyValueStore* self, OpenKVStoreRequest openKVSReq) {
|
||||
TraceEvent(SevInfo, "WaitingOnFlowProcess").detail("StoreType", openKVSReq.storeType).log();
|
||||
Promise<Void> ready;
|
||||
self->returnCode = flowProcessRunner(self, ready);
|
||||
wait(ready.getFuture());
|
||||
IKVSInterface ikvsInterface = wait(self->kvsProcess.openKVStore.getReply(openKVSReq));
|
||||
TraceEvent(SevInfo, "IKVSInterfaceReceived").detail("UID", ikvsInterface.id());
|
||||
self->interf = ikvsInterface;
|
||||
self->interf.storeType = openKVSReq.storeType;
|
||||
return Void();
|
||||
}
|
||||
|
||||
IKeyValueStore* openRemoteKVStore(KeyValueStoreType storeType,
|
||||
std::string const& filename,
|
||||
UID logID,
|
||||
int64_t memoryLimit,
|
||||
bool checkChecksums,
|
||||
bool checkIntegrity) {
|
||||
RemoteIKeyValueStore* self = new RemoteIKeyValueStore();
|
||||
self->initialized = initializeRemoteKVStore(
|
||||
self, OpenKVStoreRequest(storeType, filename, logID, memoryLimit, checkChecksums, checkIntegrity));
|
||||
return self;
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> delayFlowProcessRunAction(FlowProcess* self, double time) {
|
||||
wait(delay(time));
|
||||
wait(self->run());
|
||||
return Void();
|
||||
}
|
||||
|
||||
Future<Void> runFlowProcess(std::string const& name, Endpoint endpoint) {
|
||||
TraceEvent(SevInfo, "RunFlowProcessStart").log();
|
||||
FlowProcess* self = IProcessFactory::create(name.c_str());
|
||||
self->registerEndpoint(endpoint);
|
||||
RequestStream<FlowProcessRegistrationRequest> registerProcess(endpoint);
|
||||
FlowProcessRegistrationRequest req;
|
||||
req.flowProcessInterface = self->serializedInterface();
|
||||
registerProcess.send(req);
|
||||
TraceEvent(SevDebug, "FlowProcessInitFinished").log();
|
||||
return delayFlowProcessRunAction(self, g_network->isSimulated() ? 0 : SERVER_KNOBS->REMOTE_KV_STORE_INIT_DELAY);
|
||||
}
|
504
fdbserver/RemoteIKeyValueStore.actor.h
Normal file
504
fdbserver/RemoteIKeyValueStore.actor.h
Normal file
@ -0,0 +1,504 @@
|
||||
/*
|
||||
* RemoteIKeyValueStore.actor.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#pragma once
|
||||
|
||||
#if defined(NO_INTELLISENSE) && !defined(FDBSERVER_REMOTE_IKEYVALUESTORE_ACTOR_G_H)
|
||||
#define FDBSERVER_REMOTE_IKEYVALUESTORE_ACTOR_G_H
|
||||
#include "fdbserver/RemoteIKeyValueStore.actor.g.h"
|
||||
#elif !defined(FDBSERVER_REMOTE_IKEYVALUESTORE_ACTOR_H)
|
||||
#define FDBSERVER_REMOTE_IKEYVALUESTORE_ACTOR_H
|
||||
|
||||
#include "flow/ActorCollection.h"
|
||||
#include "flow/IRandom.h"
|
||||
#include "flow/Knobs.h"
|
||||
#include "flow/Trace.h"
|
||||
#include "flow/flow.h"
|
||||
#include "flow/network.h"
|
||||
#include "fdbrpc/FlowProcess.actor.h"
|
||||
#include "fdbrpc/FlowTransport.h"
|
||||
#include "fdbrpc/fdbrpc.h"
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "fdbserver/FDBExecHelper.actor.h"
|
||||
#include "fdbserver/IKeyValueStore.h"
|
||||
#include "fdbserver/Knobs.h"
|
||||
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
struct IKVSCommitReply {
|
||||
constexpr static FileIdentifier file_identifier = 3958189;
|
||||
StorageBytes storeBytes;
|
||||
|
||||
IKVSCommitReply() : storeBytes(0, 0, 0, 0) {}
|
||||
IKVSCommitReply(const StorageBytes& sb) : storeBytes(sb) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, storeBytes);
|
||||
}
|
||||
};
|
||||
|
||||
struct RemoteKVSProcessInterface {
|
||||
|
||||
constexpr static FileIdentifier file_identifier = 3491838;
|
||||
RequestStream<struct GetRemoteKVSProcessInterfaceRequest> getProcessInterface;
|
||||
RequestStream<struct OpenKVStoreRequest> openKVStore;
|
||||
|
||||
UID uniqueID = deterministicRandom()->randomUniqueID();
|
||||
|
||||
UID id() const { return uniqueID; }
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, getProcessInterface, openKVStore);
|
||||
}
|
||||
};
|
||||
|
||||
struct IKVSInterface {
|
||||
constexpr static FileIdentifier file_identifier = 4929113;
|
||||
RequestStream<struct IKVSGetValueRequest> getValue;
|
||||
RequestStream<struct IKVSSetRequest> set;
|
||||
RequestStream<struct IKVSClearRequest> clear;
|
||||
RequestStream<struct IKVSCommitRequest> commit;
|
||||
RequestStream<struct IKVSReadValuePrefixRequest> readValuePrefix;
|
||||
RequestStream<struct IKVSReadRangeRequest> readRange;
|
||||
RequestStream<struct IKVSGetStorageByteRequest> getStorageBytes;
|
||||
RequestStream<struct IKVSGetErrorRequest> getError;
|
||||
RequestStream<struct IKVSOnClosedRequest> onClosed;
|
||||
RequestStream<struct IKVSDisposeRequest> dispose;
|
||||
RequestStream<struct IKVSCloseRequest> close;
|
||||
|
||||
UID uniqueID;
|
||||
|
||||
UID id() const { return uniqueID; }
|
||||
|
||||
KeyValueStoreType storeType;
|
||||
|
||||
KeyValueStoreType type() const { return storeType; }
|
||||
|
||||
IKVSInterface() {}
|
||||
|
||||
IKVSInterface(KeyValueStoreType type) : uniqueID(deterministicRandom()->randomUniqueID()), storeType(type) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar,
|
||||
getValue,
|
||||
set,
|
||||
clear,
|
||||
commit,
|
||||
readValuePrefix,
|
||||
readRange,
|
||||
getStorageBytes,
|
||||
getError,
|
||||
onClosed,
|
||||
dispose,
|
||||
close,
|
||||
uniqueID);
|
||||
}
|
||||
};
|
||||
|
||||
struct GetRemoteKVSProcessInterfaceRequest {
|
||||
constexpr static FileIdentifier file_identifier = 8382983;
|
||||
ReplyPromise<struct RemoteKVSProcessInterface> reply;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, reply);
|
||||
}
|
||||
};
|
||||
|
||||
struct OpenKVStoreRequest {
|
||||
constexpr static FileIdentifier file_identifier = 5918682;
|
||||
KeyValueStoreType storeType;
|
||||
std::string filename;
|
||||
UID logID;
|
||||
int64_t memoryLimit;
|
||||
bool checkChecksums;
|
||||
bool checkIntegrity;
|
||||
ReplyPromise<struct IKVSInterface> reply;
|
||||
|
||||
OpenKVStoreRequest(){};
|
||||
|
||||
OpenKVStoreRequest(KeyValueStoreType storeType,
|
||||
std::string filename,
|
||||
UID logID,
|
||||
int64_t memoryLimit,
|
||||
bool checkChecksums = false,
|
||||
bool checkIntegrity = false)
|
||||
: storeType(storeType), filename(filename), logID(logID), memoryLimit(memoryLimit),
|
||||
checkChecksums(checkChecksums), checkIntegrity(checkIntegrity) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, storeType, filename, logID, memoryLimit, checkChecksums, checkIntegrity, reply);
|
||||
}
|
||||
};
|
||||
|
||||
struct IKVSGetValueRequest {
|
||||
constexpr static FileIdentifier file_identifier = 1029439;
|
||||
KeyRef key;
|
||||
IKeyValueStore::ReadType type;
|
||||
Optional<UID> debugID = Optional<UID>();
|
||||
ReplyPromise<Optional<Value>> reply;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, key, type, debugID, reply);
|
||||
}
|
||||
};
|
||||
|
||||
struct IKVSSetRequest {
|
||||
constexpr static FileIdentifier file_identifier = 7283948;
|
||||
KeyValueRef keyValue;
|
||||
ReplyPromise<Void> reply;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, keyValue, reply);
|
||||
}
|
||||
};
|
||||
|
||||
struct IKVSClearRequest {
|
||||
constexpr static FileIdentifier file_identifier = 2838575;
|
||||
KeyRangeRef range;
|
||||
ReplyPromise<Void> reply;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, range, reply);
|
||||
}
|
||||
};
|
||||
|
||||
struct IKVSCommitRequest {
|
||||
constexpr static FileIdentifier file_identifier = 2985129;
|
||||
bool sequential;
|
||||
ReplyPromise<IKVSCommitReply> reply;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, sequential, reply);
|
||||
}
|
||||
};
|
||||
|
||||
struct IKVSReadValuePrefixRequest {
|
||||
constexpr static FileIdentifier file_identifier = 1928374;
|
||||
KeyRef key;
|
||||
int maxLength;
|
||||
IKeyValueStore::ReadType type;
|
||||
Optional<UID> debugID = Optional<UID>();
|
||||
ReplyPromise<Optional<Value>> reply;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, key, maxLength, type, debugID, reply);
|
||||
}
|
||||
};
|
||||
|
||||
// Use this instead of RangeResult as reply for better serialization performance
|
||||
struct IKVSReadRangeReply {
|
||||
constexpr static FileIdentifier file_identifier = 6682449;
|
||||
Arena arena;
|
||||
VectorRef<KeyValueRef, VecSerStrategy::String> data;
|
||||
bool more;
|
||||
Optional<KeyRef> readThrough;
|
||||
bool readToBegin;
|
||||
bool readThroughEnd;
|
||||
|
||||
IKVSReadRangeReply() = default;
|
||||
|
||||
explicit IKVSReadRangeReply(const RangeResult& res)
|
||||
: arena(res.arena()), data(static_cast<const VectorRef<KeyValueRef>&>(res)), more(res.more),
|
||||
readThrough(res.readThrough), readToBegin(res.readToBegin), readThroughEnd(res.readThroughEnd) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, data, more, readThrough, readToBegin, readThroughEnd, arena);
|
||||
}
|
||||
|
||||
RangeResult toRangeResult() const {
|
||||
RangeResult r(RangeResultRef(data, more, readThrough), arena);
|
||||
r.readToBegin = readToBegin;
|
||||
r.readThroughEnd = readThroughEnd;
|
||||
return r;
|
||||
}
|
||||
};
|
||||
|
||||
struct IKVSReadRangeRequest {
|
||||
constexpr static FileIdentifier file_identifier = 5918394;
|
||||
KeyRangeRef keys;
|
||||
int rowLimit;
|
||||
int byteLimit;
|
||||
IKeyValueStore::ReadType type;
|
||||
ReplyPromise<IKVSReadRangeReply> reply;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, keys, rowLimit, byteLimit, type, reply);
|
||||
}
|
||||
};
|
||||
|
||||
struct IKVSGetStorageByteRequest {
|
||||
constexpr static FileIdentifier file_identifier = 3512344;
|
||||
ReplyPromise<StorageBytes> reply;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, reply);
|
||||
}
|
||||
};
|
||||
|
||||
struct IKVSGetErrorRequest {
|
||||
constexpr static FileIdentifier file_identifier = 3942891;
|
||||
ReplyPromise<Void> reply;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, reply);
|
||||
}
|
||||
};
|
||||
|
||||
struct IKVSOnClosedRequest {
|
||||
constexpr static FileIdentifier file_identifier = 1923894;
|
||||
ReplyPromise<Void> reply;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, reply);
|
||||
}
|
||||
};
|
||||
|
||||
struct IKVSDisposeRequest {
|
||||
constexpr static FileIdentifier file_identifier = 1235952;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar);
|
||||
}
|
||||
};
|
||||
|
||||
struct IKVSCloseRequest {
|
||||
constexpr static FileIdentifier file_identifier = 13859172;
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar);
|
||||
}
|
||||
};
|
||||
|
||||
ACTOR Future<Void> runIKVS(OpenKVStoreRequest openReq, IKVSInterface ikvsInterface);
|
||||
|
||||
struct KeyValueStoreProcess : FlowProcess {
|
||||
RemoteKVSProcessInterface kvsIf;
|
||||
Standalone<StringRef> serializedIf;
|
||||
|
||||
Endpoint ssProcess; // endpoint for the storage process
|
||||
RequestStream<FlowProcessRegistrationRequest> ssRequestStream;
|
||||
|
||||
KeyValueStoreProcess() {
|
||||
TraceEvent(SevDebug, "InitKeyValueStoreProcess").log();
|
||||
ObjectWriter writer(IncludeVersion());
|
||||
writer.serialize(kvsIf);
|
||||
serializedIf = writer.toString();
|
||||
}
|
||||
|
||||
void registerEndpoint(Endpoint p) override {
|
||||
ssProcess = p;
|
||||
ssRequestStream = RequestStream<FlowProcessRegistrationRequest>(p);
|
||||
}
|
||||
|
||||
StringRef name() const override { return _name; }
|
||||
StringRef serializedInterface() const override { return serializedIf; }
|
||||
|
||||
ACTOR static Future<Void> _run(KeyValueStoreProcess* self) {
|
||||
state ActorCollection actors(true);
|
||||
TraceEvent("WaitingForOpenKVStoreRequest").log();
|
||||
loop {
|
||||
choose {
|
||||
when(OpenKVStoreRequest req = waitNext(self->kvsIf.openKVStore.getFuture())) {
|
||||
TraceEvent("OpenKVStoreRequestReceived").log();
|
||||
IKVSInterface interf;
|
||||
actors.add(runIKVS(req, interf));
|
||||
}
|
||||
when(ErrorOr<Void> e = wait(errorOr(actors.getResult()))) {
|
||||
if (e.isError()) {
|
||||
TraceEvent("KeyValueStoreProcessRunActorError").errorUnsuppressed(e.getError());
|
||||
throw e.getError();
|
||||
} else {
|
||||
TraceEvent("KeyValueStoreProcessFinished").log();
|
||||
return e.get();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Future<Void> run() override { return _run(this); }
|
||||
|
||||
static StringRef _name;
|
||||
};
|
||||
|
||||
struct RemoteIKeyValueStore : public IKeyValueStore {
|
||||
RemoteKVSProcessInterface kvsProcess;
|
||||
IKVSInterface interf;
|
||||
Future<Void> initialized;
|
||||
Future<int> returnCode;
|
||||
StorageBytes storageBytes;
|
||||
|
||||
RemoteIKeyValueStore() : storageBytes(0, 0, 0, 0) {}
|
||||
|
||||
Future<Void> init() override {
|
||||
TraceEvent(SevInfo, "RemoteIKeyValueStoreInit").log();
|
||||
return initialized;
|
||||
}
|
||||
|
||||
Future<Void> getError() const override { return getErrorImpl(this, returnCode); }
|
||||
Future<Void> onClosed() const override { return onCloseImpl(this); }
|
||||
|
||||
void dispose() override {
|
||||
TraceEvent(SevDebug, "RemoteIKVSDisposeRequest").backtrace();
|
||||
interf.dispose.send(IKVSDisposeRequest{});
|
||||
// hold the future to not cancel the spawned process
|
||||
uncancellable(returnCode);
|
||||
delete this;
|
||||
}
|
||||
void close() override {
|
||||
TraceEvent(SevDebug, "RemoteIKVSCloseRequest").backtrace();
|
||||
interf.close.send(IKVSCloseRequest{});
|
||||
// hold the future to not cancel the spawned process
|
||||
uncancellable(returnCode);
|
||||
delete this;
|
||||
}
|
||||
|
||||
KeyValueStoreType getType() const override { return interf.type(); }
|
||||
|
||||
void set(KeyValueRef keyValue, const Arena* arena = nullptr) override {
|
||||
interf.set.send(IKVSSetRequest{ keyValue, ReplyPromise<Void>() });
|
||||
}
|
||||
void clear(KeyRangeRef range, const Arena* arena = nullptr) override {
|
||||
interf.clear.send(IKVSClearRequest{ range, ReplyPromise<Void>() });
|
||||
}
|
||||
|
||||
Future<Void> commit(bool sequential = false) override {
|
||||
Future<IKVSCommitReply> commitReply =
|
||||
interf.commit.getReply(IKVSCommitRequest{ sequential, ReplyPromise<IKVSCommitReply>() });
|
||||
return commitAndGetStorageBytes(this, commitReply);
|
||||
}
|
||||
|
||||
Future<Optional<Value>> readValue(KeyRef key,
|
||||
ReadType type = ReadType::NORMAL,
|
||||
Optional<UID> debugID = Optional<UID>()) override {
|
||||
return readValueImpl(this, IKVSGetValueRequest{ key, type, debugID, ReplyPromise<Optional<Value>>() });
|
||||
}
|
||||
|
||||
Future<Optional<Value>> readValuePrefix(KeyRef key,
|
||||
int maxLength,
|
||||
ReadType type = ReadType::NORMAL,
|
||||
Optional<UID> debugID = Optional<UID>()) override {
|
||||
return interf.readValuePrefix.getReply(
|
||||
IKVSReadValuePrefixRequest{ key, maxLength, type, debugID, ReplyPromise<Optional<Value>>() });
|
||||
}
|
||||
|
||||
Future<RangeResult> readRange(KeyRangeRef keys,
|
||||
int rowLimit = 1 << 30,
|
||||
int byteLimit = 1 << 30,
|
||||
ReadType type = ReadType::NORMAL) override {
|
||||
IKVSReadRangeRequest req{ keys, rowLimit, byteLimit, type, ReplyPromise<IKVSReadRangeReply>() };
|
||||
return fmap([](const IKVSReadRangeReply& reply) { return reply.toRangeResult(); },
|
||||
interf.readRange.getReply(req));
|
||||
}
|
||||
|
||||
StorageBytes getStorageBytes() const override { return storageBytes; }
|
||||
|
||||
void consumeInterface(StringRef intf) {
|
||||
kvsProcess = ObjectReader::fromStringRef<RemoteKVSProcessInterface>(intf, IncludeVersion());
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> commitAndGetStorageBytes(RemoteIKeyValueStore* self,
|
||||
Future<IKVSCommitReply> commitReplyFuture) {
|
||||
IKVSCommitReply commitReply = wait(commitReplyFuture);
|
||||
self->storageBytes = commitReply.storeBytes;
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR static Future<Optional<Value>> readValueImpl(RemoteIKeyValueStore* self, IKVSGetValueRequest req) {
|
||||
Optional<Value> val = wait(self->interf.getValue.getReply(req));
|
||||
return val;
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> getErrorImpl(const RemoteIKeyValueStore* self, Future<int> returnCode) {
|
||||
choose {
|
||||
when(wait(self->initialized)) {}
|
||||
when(wait(delay(SERVER_KNOBS->REMOTE_KV_STORE_MAX_INIT_DURATION))) {
|
||||
TraceEvent(SevError, "RemoteIKVSInitTooLong")
|
||||
.detail("TimeLimit", SERVER_KNOBS->REMOTE_KV_STORE_MAX_INIT_DURATION);
|
||||
throw please_reboot_remote_kv_store();
|
||||
}
|
||||
}
|
||||
state Future<Void> connectionCheckingDelay = delay(FLOW_KNOBS->FAILURE_DETECTION_DELAY);
|
||||
state Future<ErrorOr<Void>> storeError = errorOr(self->interf.getError.getReply(IKVSGetErrorRequest{}));
|
||||
loop choose {
|
||||
when(ErrorOr<Void> e = wait(storeError)) {
|
||||
TraceEvent(SevDebug, "RemoteIKVSGetError")
|
||||
.errorUnsuppressed(e.isError() ? e.getError() : success())
|
||||
.backtrace();
|
||||
if (e.isError())
|
||||
throw e.getError();
|
||||
else
|
||||
return e.get();
|
||||
}
|
||||
when(int res = wait(returnCode)) {
|
||||
TraceEvent(res != 0 ? SevError : SevInfo, "SpawnedProcessDied").detail("Res", res);
|
||||
if (res)
|
||||
throw please_reboot_remote_kv_store(); // this will reboot the worker
|
||||
else
|
||||
return Void();
|
||||
}
|
||||
when(wait(connectionCheckingDelay)) {
|
||||
// for the corner case where the child process stuck and waitpid also does not give update on it
|
||||
// In this scenario, we need to manually reboot the storage engine process
|
||||
if (IFailureMonitor::failureMonitor()
|
||||
.getState(self->interf.getError.getEndpoint().getPrimaryAddress())
|
||||
.isFailed()) {
|
||||
TraceEvent(SevError, "RemoteKVStoreConnectionStuck").log();
|
||||
throw please_reboot_remote_kv_store(); // this will reboot the worker
|
||||
}
|
||||
connectionCheckingDelay = delay(FLOW_KNOBS->FAILURE_DETECTION_DELAY);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> onCloseImpl(const RemoteIKeyValueStore* self) {
|
||||
try {
|
||||
wait(self->initialized);
|
||||
wait(self->interf.onClosed.getReply(IKVSOnClosedRequest{}));
|
||||
TraceEvent(SevDebug, "RemoteIKVSOnCloseImplOnClosedFinished");
|
||||
} catch (Error& e) {
|
||||
TraceEvent(SevInfo, "RemoteIKVSOnCloseImplError").errorUnsuppressed(e).backtrace();
|
||||
throw;
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
};
|
||||
|
||||
Future<Void> runFlowProcess(std::string const& name, Endpoint endpoint);
|
||||
|
||||
#include "flow/unactorcompiler.h"
|
||||
#endif
|
@ -263,6 +263,9 @@ class TestConfig {
|
||||
if (attrib == "disableHostname") {
|
||||
disableHostname = strcmp(value.c_str(), "true") == 0;
|
||||
}
|
||||
if (attrib == "disableRemoteKVS") {
|
||||
disableRemoteKVS = strcmp(value.c_str(), "true") == 0;
|
||||
}
|
||||
if (attrib == "restartInfoLocation") {
|
||||
isFirstTestInRestart = true;
|
||||
}
|
||||
@ -298,6 +301,8 @@ public:
|
||||
bool disableTss = false;
|
||||
// 7.1 cannot be downgraded to 7.0 and below after enabling hostname, so disable hostname for 7.0 downgrade tests
|
||||
bool disableHostname = false;
|
||||
// remote key value store is a child process spawned by the SS process to run the storage engine
|
||||
bool disableRemoteKVS = false;
|
||||
// Storage Engine Types: Verify match with SimulationConfig::generateNormalConfig
|
||||
// 0 = "ssd"
|
||||
// 1 = "memory"
|
||||
@ -357,6 +362,7 @@ public:
|
||||
.add("maxTLogVersion", &maxTLogVersion)
|
||||
.add("disableTss", &disableTss)
|
||||
.add("disableHostname", &disableHostname)
|
||||
.add("disableRemoteKVS", &disableRemoteKVS)
|
||||
.add("simpleConfig", &simpleConfig)
|
||||
.add("generateFearless", &generateFearless)
|
||||
.add("datacenters", &datacenters)
|
||||
@ -1084,6 +1090,11 @@ ACTOR Future<Void> restartSimulatedSystem(std::vector<Future<Void>>* systemActor
|
||||
INetworkConnections::net()->parseMockDNSFromString(mockDNSStr);
|
||||
}
|
||||
}
|
||||
if (testConfig.disableRemoteKVS) {
|
||||
IKnobCollection::getMutableGlobalKnobCollection().setKnob("remote_kv_store",
|
||||
KnobValueRef::create(bool{ false }));
|
||||
TraceEvent(SevDebug, "DisaableRemoteKVS").log();
|
||||
}
|
||||
*pConnString = conn;
|
||||
*pTesterCount = testerCount;
|
||||
bool usingSSL = conn.toString().find(":tls") != std::string::npos || listenersPerProcess > 1;
|
||||
@ -1836,6 +1847,11 @@ void setupSimulatedSystem(std::vector<Future<Void>>* systemActors,
|
||||
if (testConfig.configureLocked) {
|
||||
startingConfigString += " locked";
|
||||
}
|
||||
if (testConfig.disableRemoteKVS) {
|
||||
IKnobCollection::getMutableGlobalKnobCollection().setKnob("remote_kv_store",
|
||||
KnobValueRef::create(bool{ false }));
|
||||
TraceEvent(SevDebug, "DisaableRemoteKVS").log();
|
||||
}
|
||||
auto configDBType = testConfig.getConfigDBType();
|
||||
for (auto kv : startingConfigJSON) {
|
||||
if ("tss_storage_engine" == kv.first) {
|
||||
|
@ -45,16 +45,20 @@
|
||||
#include "fdbclient/WellKnownEndpoints.h"
|
||||
#include "fdbclient/SimpleIni.h"
|
||||
#include "fdbrpc/AsyncFileCached.actor.h"
|
||||
#include "fdbrpc/FlowProcess.actor.h"
|
||||
#include "fdbrpc/Net2FileSystem.h"
|
||||
#include "fdbrpc/PerfMetric.h"
|
||||
#include "fdbrpc/fdbrpc.h"
|
||||
#include "fdbrpc/simulator.h"
|
||||
#include "fdbserver/ConflictSet.h"
|
||||
#include "fdbserver/CoordinationInterface.h"
|
||||
#include "fdbserver/CoroFlow.h"
|
||||
#include "fdbserver/DataDistribution.actor.h"
|
||||
#include "fdbserver/FDBExecHelper.actor.h"
|
||||
#include "fdbserver/IKeyValueStore.h"
|
||||
#include "fdbserver/MoveKeys.actor.h"
|
||||
#include "fdbserver/NetworkTest.h"
|
||||
#include "fdbserver/RemoteIKeyValueStore.actor.h"
|
||||
#include "fdbserver/RestoreWorkerInterface.actor.h"
|
||||
#include "fdbserver/ServerDBInfo.h"
|
||||
#include "fdbserver/SimulatedCluster.h"
|
||||
@ -74,10 +78,13 @@
|
||||
#include "flow/WriteOnlySet.h"
|
||||
#include "flow/UnitTest.h"
|
||||
#include "flow/FaultInjection.h"
|
||||
#include "flow/flow.h"
|
||||
#include "flow/network.h"
|
||||
|
||||
#if defined(__linux__) || defined(__FreeBSD__)
|
||||
#include <execinfo.h>
|
||||
#include <signal.h>
|
||||
#include <sys/prctl.h>
|
||||
#ifdef ALLOC_INSTRUMENTATION
|
||||
#include <cxxabi.h>
|
||||
#endif
|
||||
@ -100,7 +107,7 @@ enum {
|
||||
OPT_DCID, OPT_MACHINE_CLASS, OPT_BUGGIFY, OPT_VERSION, OPT_BUILD_FLAGS, OPT_CRASHONERROR, OPT_HELP, OPT_NETWORKIMPL, OPT_NOBUFSTDOUT, OPT_BUFSTDOUTERR,
|
||||
OPT_TRACECLOCK, OPT_NUMTESTERS, OPT_DEVHELP, OPT_ROLLSIZE, OPT_MAXLOGS, OPT_MAXLOGSSIZE, OPT_KNOB, OPT_UNITTESTPARAM, OPT_TESTSERVERS, OPT_TEST_ON_SERVERS, OPT_METRICSCONNFILE,
|
||||
OPT_METRICSPREFIX, OPT_LOGGROUP, OPT_LOCALITY, OPT_IO_TRUST_SECONDS, OPT_IO_TRUST_WARN_ONLY, OPT_FILESYSTEM, OPT_PROFILER_RSS_SIZE, OPT_KVFILE,
|
||||
OPT_TRACE_FORMAT, OPT_WHITELIST_BINPATH, OPT_BLOB_CREDENTIAL_FILE, OPT_CONFIG_PATH, OPT_USE_TEST_CONFIG_DB, OPT_FAULT_INJECTION, OPT_PROFILER, OPT_PRINT_SIMTIME,
|
||||
OPT_TRACE_FORMAT, OPT_WHITELIST_BINPATH, OPT_BLOB_CREDENTIAL_FILE, OPT_CONFIG_PATH, OPT_USE_TEST_CONFIG_DB, OPT_FAULT_INJECTION, OPT_PROFILER, OPT_PRINT_SIMTIME, OPT_FLOW_PROCESS_NAME, OPT_FLOW_PROCESS_ENDPOINT
|
||||
};
|
||||
|
||||
CSimpleOpt::SOption g_rgOptions[] = {
|
||||
@ -187,8 +194,10 @@ CSimpleOpt::SOption g_rgOptions[] = {
|
||||
{ OPT_USE_TEST_CONFIG_DB, "--use-test-config-db", SO_NONE },
|
||||
{ OPT_FAULT_INJECTION, "-fi", SO_REQ_SEP },
|
||||
{ OPT_FAULT_INJECTION, "--fault-injection", SO_REQ_SEP },
|
||||
{ OPT_PROFILER, "--profiler-", SO_REQ_SEP},
|
||||
{ OPT_PROFILER, "--profiler-", SO_REQ_SEP },
|
||||
{ OPT_PRINT_SIMTIME, "--print-sim-time", SO_NONE },
|
||||
{ OPT_FLOW_PROCESS_NAME, "--process-name", SO_REQ_SEP },
|
||||
{ OPT_FLOW_PROCESS_ENDPOINT, "--process-endpoint", SO_REQ_SEP },
|
||||
|
||||
#ifndef TLS_DISABLED
|
||||
TLS_OPTION_FLAGS
|
||||
@ -959,7 +968,8 @@ enum class ServerRole {
|
||||
SkipListTest,
|
||||
Test,
|
||||
VersionedMapTest,
|
||||
UnitTests
|
||||
UnitTests,
|
||||
FlowProcess
|
||||
};
|
||||
struct CLIOptions {
|
||||
std::string commandLine;
|
||||
@ -1015,6 +1025,8 @@ struct CLIOptions {
|
||||
UnitTestParameters testParams;
|
||||
|
||||
std::map<std::string, std::string> profilerConfig;
|
||||
std::string flowProcessName;
|
||||
Endpoint flowProcessEndpoint;
|
||||
bool printSimTime = false;
|
||||
|
||||
static CLIOptions parseArgs(int argc, char* argv[]) {
|
||||
@ -1193,6 +1205,8 @@ private:
|
||||
role = ServerRole::ConsistencyCheck;
|
||||
else if (!strcmp(sRole, "unittests"))
|
||||
role = ServerRole::UnitTests;
|
||||
else if (!strcmp(sRole, "flowprocess"))
|
||||
role = ServerRole::FlowProcess;
|
||||
else {
|
||||
fprintf(stderr, "ERROR: Unknown role `%s'\n", sRole);
|
||||
printHelpTeaser(argv[0]);
|
||||
@ -1517,6 +1531,42 @@ private:
|
||||
case OPT_USE_TEST_CONFIG_DB:
|
||||
configDBType = ConfigDBType::SIMPLE;
|
||||
break;
|
||||
case OPT_FLOW_PROCESS_NAME:
|
||||
flowProcessName = args.OptionArg();
|
||||
std::cout << flowProcessName << std::endl;
|
||||
break;
|
||||
case OPT_FLOW_PROCESS_ENDPOINT: {
|
||||
std::vector<std::string> strings;
|
||||
std::cout << args.OptionArg() << std::endl;
|
||||
boost::split(strings, args.OptionArg(), [](char c) { return c == ','; });
|
||||
for (auto& str : strings) {
|
||||
std::cout << str << " ";
|
||||
}
|
||||
std::cout << "\n";
|
||||
if (strings.size() != 3) {
|
||||
std::cerr << "Invalid argument, expected 3 elements in --process-endpoint got " << strings.size()
|
||||
<< std::endl;
|
||||
flushAndExit(FDB_EXIT_ERROR);
|
||||
}
|
||||
try {
|
||||
auto addr = NetworkAddress::parse(strings[0]);
|
||||
uint64_t fst = std::stoul(strings[1]);
|
||||
uint64_t snd = std::stoul(strings[2]);
|
||||
UID token(fst, snd);
|
||||
NetworkAddressList l;
|
||||
l.address = addr;
|
||||
flowProcessEndpoint = Endpoint(l, token);
|
||||
std::cout << "flowProcessEndpoint: " << flowProcessEndpoint.getPrimaryAddress().toString()
|
||||
<< ", token: " << flowProcessEndpoint.token.toString() << "\n";
|
||||
} catch (Error& e) {
|
||||
std::cerr << "Could not parse network address " << strings[0] << std::endl;
|
||||
flushAndExit(FDB_EXIT_ERROR);
|
||||
} catch (std::exception& e) {
|
||||
std::cerr << "Could not parse token " << strings[1] << "," << strings[2] << std::endl;
|
||||
flushAndExit(FDB_EXIT_ERROR);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case OPT_PRINT_SIMTIME:
|
||||
printSimTime = true;
|
||||
break;
|
||||
@ -1723,6 +1773,7 @@ int main(int argc, char* argv[]) {
|
||||
role == ServerRole::Simulation ? IsSimulated::True
|
||||
: IsSimulated::False);
|
||||
IKnobCollection::getMutableGlobalKnobCollection().setKnob("log_directory", KnobValue::create(opts.logFolder));
|
||||
IKnobCollection::getMutableGlobalKnobCollection().setKnob("conn_file", KnobValue::create(opts.connFile));
|
||||
if (role != ServerRole::Simulation) {
|
||||
IKnobCollection::getMutableGlobalKnobCollection().setKnob("commit_batches_mem_bytes_hard_limit",
|
||||
KnobValue::create(int64_t{ opts.memLimit }));
|
||||
@ -1802,8 +1853,8 @@ int main(int argc, char* argv[]) {
|
||||
FlowTransport::createInstance(false, 1, WLTOKEN_RESERVED_COUNT);
|
||||
opts.buildNetwork(argv[0]);
|
||||
|
||||
const bool expectsPublicAddress =
|
||||
(role == ServerRole::FDBD || role == ServerRole::NetworkTestServer || role == ServerRole::Restore);
|
||||
const bool expectsPublicAddress = (role == ServerRole::FDBD || role == ServerRole::NetworkTestServer ||
|
||||
role == ServerRole::Restore || role == ServerRole::FlowProcess);
|
||||
if (opts.publicAddressStrs.empty()) {
|
||||
if (expectsPublicAddress) {
|
||||
fprintf(stderr, "ERROR: The -p or --public-address option is required\n");
|
||||
@ -2139,6 +2190,19 @@ int main(int argc, char* argv[]) {
|
||||
}
|
||||
|
||||
f = result;
|
||||
} else if (role == ServerRole::FlowProcess) {
|
||||
TraceEvent(SevDebug, "StartingFlowProcess").detail("From", "fdbserver");
|
||||
#if defined(__linux__) || defined(__FreeBSD__)
|
||||
prctl(PR_SET_PDEATHSIG, SIGTERM);
|
||||
if (getppid() == 1) /* parent already died before prctl */
|
||||
flushAndExit(FDB_EXIT_SUCCESS);
|
||||
#endif
|
||||
|
||||
if (opts.flowProcessName == "KeyValueStoreProcess") {
|
||||
ProcessFactory<KeyValueStoreProcess>(opts.flowProcessName.c_str());
|
||||
}
|
||||
f = stopAfter(runFlowProcess(opts.flowProcessName, opts.flowProcessEndpoint));
|
||||
g_network->run();
|
||||
} else if (role == ServerRole::KVFileDump) {
|
||||
f = stopAfter(KVFileDump(opts.kvFile));
|
||||
g_network->run();
|
||||
|
@ -8421,7 +8421,8 @@ bool storageServerTerminated(StorageServer& self, IKeyValueStore* persistentData
|
||||
}
|
||||
|
||||
if (e.code() == error_code_worker_removed || e.code() == error_code_recruitment_failed ||
|
||||
e.code() == error_code_file_not_found || e.code() == error_code_actor_cancelled) {
|
||||
e.code() == error_code_file_not_found || e.code() == error_code_actor_cancelled ||
|
||||
e.code() == error_code_remote_kvs_cancelled) {
|
||||
TraceEvent("StorageServerTerminated", self.thisServerID).errorUnsuppressed(e);
|
||||
return true;
|
||||
} else
|
||||
|
@ -1092,7 +1092,8 @@ std::map<std::string, std::function<void(const std::string&)>> testSpecGlobalKey
|
||||
[](const std::string& value) { TraceEvent("TestParserTest").detail("ParsedMaxTLogVersion", ""); } },
|
||||
{ "disableTss", [](const std::string& value) { TraceEvent("TestParserTest").detail("ParsedDisableTSS", ""); } },
|
||||
{ "disableHostname",
|
||||
[](const std::string& value) { TraceEvent("TestParserTest").detail("ParsedDisableHostname", ""); } }
|
||||
[](const std::string& value) { TraceEvent("TestParserTest").detail("ParsedDisableHostname", ""); } },
|
||||
{ "disableRemoteKVS", [](const std::string& value) { TraceEvent("TestParserTest").detail("ParsedRemoteKVS", ""); } }
|
||||
};
|
||||
|
||||
std::map<std::string, std::function<void(const std::string& value, TestSpec* spec)>> testSpecTestKeys = {
|
||||
|
@ -18,6 +18,7 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include <cstdlib>
|
||||
#include <tuple>
|
||||
#include <boost/lexical_cast.hpp>
|
||||
|
||||
@ -49,6 +50,7 @@
|
||||
#include "fdbserver/CoordinationInterface.h"
|
||||
#include "fdbserver/ConfigNode.h"
|
||||
#include "fdbserver/LocalConfiguration.h"
|
||||
#include "fdbserver/RemoteIKeyValueStore.actor.h"
|
||||
#include "fdbclient/MonitorLeader.h"
|
||||
#include "fdbclient/ClientWorkerInterface.h"
|
||||
#include "flow/Profiler.h"
|
||||
@ -208,31 +210,44 @@ ACTOR Future<Void> handleIOErrors(Future<Void> actor, IClosable* store, UID id,
|
||||
state Future<ErrorOr<Void>> storeError = actor.isReady() ? Never() : errorOr(store->getError());
|
||||
choose {
|
||||
when(state ErrorOr<Void> e = wait(errorOr(actor))) {
|
||||
TraceEvent(SevDebug, "HandleIOErrorsActorIsReady")
|
||||
.detail("Error", e.isError() ? e.getError().code() : -1)
|
||||
.detail("UID", id);
|
||||
if (e.isError() && e.getError().code() == error_code_please_reboot) {
|
||||
// no need to wait.
|
||||
} else {
|
||||
TraceEvent(SevDebug, "HandleIOErrorsActorBeforeOnClosed").detail("IsClosed", onClosed.isReady());
|
||||
wait(onClosed);
|
||||
TraceEvent(SevDebug, "HandleIOErrorsActorOnClosedFinished")
|
||||
.detail("StoreError",
|
||||
storeError.isReady() ? (storeError.get().isError() ? storeError.get().getError().code() : 0)
|
||||
: -1);
|
||||
}
|
||||
if (e.isError() && e.getError().code() == error_code_broken_promise && !storeError.isReady()) {
|
||||
wait(delay(0.00001 + FLOW_KNOBS->MAX_BUGGIFIED_DELAY));
|
||||
}
|
||||
if (storeError.isReady())
|
||||
throw storeError.get().getError();
|
||||
if (e.isError())
|
||||
if (storeError.isReady() &&
|
||||
!((storeError.get().isError() && storeError.get().getError().code() == error_code_file_not_found))) {
|
||||
throw storeError.get().isError() ? storeError.get().getError() : actor_cancelled();
|
||||
}
|
||||
if (e.isError()) {
|
||||
throw e.getError();
|
||||
else
|
||||
} else
|
||||
return e.get();
|
||||
}
|
||||
when(ErrorOr<Void> e = wait(storeError)) {
|
||||
TraceEvent("WorkerTerminatingByIOError", id).errorUnsuppressed(e.getError());
|
||||
// for remote kv store, worker can terminate without an error, so throws actor_cancelled
|
||||
// (there's probably a better way tho)
|
||||
TraceEvent("WorkerTerminatingByIOError", id)
|
||||
.errorUnsuppressed(e.isError() ? e.getError() : actor_cancelled());
|
||||
actor.cancel();
|
||||
// file_not_found can occur due to attempting to open a partially deleted DiskQueue, which should not be
|
||||
// reported SevError.
|
||||
if (e.getError().code() == error_code_file_not_found) {
|
||||
if (e.isError() && e.getError().code() == error_code_file_not_found) {
|
||||
TEST(true); // Worker terminated with file_not_found error
|
||||
return Void();
|
||||
}
|
||||
throw e.getError();
|
||||
throw e.isError() ? e.getError() : actor_cancelled();
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -243,6 +258,7 @@ ACTOR Future<Void> workerHandleErrors(FutureStream<ErrorInfo> errors) {
|
||||
ErrorInfo err = _err;
|
||||
bool ok = err.error.code() == error_code_success || err.error.code() == error_code_please_reboot ||
|
||||
err.error.code() == error_code_actor_cancelled ||
|
||||
err.error.code() == error_code_remote_kvs_cancelled ||
|
||||
err.error.code() == error_code_coordinators_changed || // The worker server was cancelled
|
||||
err.error.code() == error_code_shutdown_in_progress;
|
||||
|
||||
@ -253,6 +269,7 @@ ACTOR Future<Void> workerHandleErrors(FutureStream<ErrorInfo> errors) {
|
||||
endRole(err.role, err.id, "Error", ok, err.error);
|
||||
|
||||
if (err.error.code() == error_code_please_reboot ||
|
||||
err.error.code() == error_code_please_reboot_remote_kv_store ||
|
||||
(err.role == Role::SHARED_TRANSACTION_LOG &&
|
||||
(err.error.code() == error_code_io_error || err.error.code() == error_code_io_timeout)))
|
||||
throw err.error;
|
||||
@ -1090,9 +1107,13 @@ struct TrackRunningStorage {
|
||||
KeyValueStoreType storeType,
|
||||
std::set<std::pair<UID, KeyValueStoreType>>* runningStorages)
|
||||
: self(self), storeType(storeType), runningStorages(runningStorages) {
|
||||
TraceEvent(SevDebug, "TrackingRunningStorageConstruction").detail("StorageID", self);
|
||||
runningStorages->emplace(self, storeType);
|
||||
}
|
||||
~TrackRunningStorage() { runningStorages->erase(std::make_pair(self, storeType)); };
|
||||
~TrackRunningStorage() {
|
||||
runningStorages->erase(std::make_pair(self, storeType));
|
||||
TraceEvent(SevDebug, "TrackingRunningStorageDesctruction").detail("StorageID", self);
|
||||
};
|
||||
};
|
||||
|
||||
ACTOR Future<Void> storageServerRollbackRebooter(std::set<std::pair<UID, KeyValueStoreType>>* runningStorages,
|
||||
@ -1523,8 +1544,15 @@ ACTOR Future<Void> workerServer(Reference<IClusterConnectionRecord> connRecord,
|
||||
if (s.storedComponent == DiskStore::Storage) {
|
||||
LocalLineage _;
|
||||
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::Storage;
|
||||
IKeyValueStore* kv =
|
||||
openKVStore(s.storeType, s.filename, s.storeID, memoryLimit, false, validateDataFiles);
|
||||
IKeyValueStore* kv = openKVStore(
|
||||
s.storeType,
|
||||
s.filename,
|
||||
s.storeID,
|
||||
memoryLimit,
|
||||
false,
|
||||
validateDataFiles,
|
||||
SERVER_KNOBS->REMOTE_KV_STORE && /* testing mixed mode in simulation if remote kvs enabled*/
|
||||
(g_network->isSimulated() ? deterministicRandom()->coinflip() : true));
|
||||
Future<Void> kvClosed = kv->onClosed();
|
||||
filesClosed.add(kvClosed);
|
||||
|
||||
@ -1598,6 +1626,7 @@ ACTOR Future<Void> workerServer(Reference<IClusterConnectionRecord> connRecord,
|
||||
logQueueBasename = fileLogQueuePrefix.toString() + optionsString.toString() + "-";
|
||||
}
|
||||
ASSERT_WE_THINK(abspath(parentDirectory(s.filename)) == folder);
|
||||
// TraceEvent(SevDebug, "openRemoteKVStore").detail("storeType", "TlogData");
|
||||
IKeyValueStore* kv = openKVStore(s.storeType, s.filename, s.storeID, memoryLimit, validateDataFiles);
|
||||
const DiskQueueVersion dqv = s.tLogOptions.getDiskQueueVersion();
|
||||
const int64_t diskQueueWarnSize =
|
||||
@ -2002,6 +2031,7 @@ ACTOR Future<Void> workerServer(Reference<IClusterConnectionRecord> connRecord,
|
||||
req.logVersion > TLogVersion::V2 ? fileVersionedLogDataPrefix : fileLogDataPrefix;
|
||||
std::string filename =
|
||||
filenameFromId(req.storeType, folder, prefix.toString() + tLogOptions.toPrefix(), logId);
|
||||
// TraceEvent(SevDebug, "openRemoteKVStore").detail("storeType", "3");
|
||||
IKeyValueStore* data = openKVStore(req.storeType, filename, logId, memoryLimit);
|
||||
const DiskQueueVersion dqv = tLogOptions.getDiskQueueVersion();
|
||||
IDiskQueue* queue = openDiskQueue(
|
||||
@ -2086,7 +2116,17 @@ ACTOR Future<Void> workerServer(Reference<IClusterConnectionRecord> connRecord,
|
||||
folder,
|
||||
isTss ? testingStoragePrefix.toString() : fileStoragePrefix.toString(),
|
||||
recruited.id());
|
||||
IKeyValueStore* data = openKVStore(req.storeType, filename, recruited.id(), memoryLimit);
|
||||
|
||||
IKeyValueStore* data = openKVStore(
|
||||
req.storeType,
|
||||
filename,
|
||||
recruited.id(),
|
||||
memoryLimit,
|
||||
false,
|
||||
false,
|
||||
SERVER_KNOBS->REMOTE_KV_STORE && /* testing mixed mode in simulation if remote kvs enabled*/
|
||||
(g_network->isSimulated() ? deterministicRandom()->coinflip() : true));
|
||||
|
||||
Future<Void> kvClosed = data->onClosed();
|
||||
filesClosed.add(kvClosed);
|
||||
ReplyPromise<InitializeStorageReply> storageReady = req.reply;
|
||||
@ -2333,20 +2373,26 @@ ACTOR Future<Void> workerServer(Reference<IClusterConnectionRecord> connRecord,
|
||||
when(wait(handleErrors)) {}
|
||||
}
|
||||
} catch (Error& err) {
|
||||
TraceEvent(SevDebug, "WorkerServer").detail("Error", err.code()).backtrace();
|
||||
// Make sure actors are cancelled before "recovery" promises are destructed.
|
||||
for (auto f : recoveries)
|
||||
f.cancel();
|
||||
state Error e = err;
|
||||
bool ok = e.code() == error_code_please_reboot || e.code() == error_code_actor_cancelled ||
|
||||
e.code() == error_code_please_reboot_delete;
|
||||
e.code() == error_code_please_reboot_delete || e.code() == error_code_please_reboot_remote_kv_store;
|
||||
|
||||
endRole(Role::WORKER, interf.id(), "WorkerError", ok, e);
|
||||
errorForwarders.clear(false);
|
||||
sharedLogs.clear();
|
||||
|
||||
if (e.code() !=
|
||||
error_code_actor_cancelled) { // We get cancelled e.g. when an entire simulation times out, but in that case
|
||||
// we won't be restarted and don't need to wait for shutdown
|
||||
if (e.code() != error_code_actor_cancelled && e.code() != error_code_please_reboot_remote_kv_store) {
|
||||
// actor_cancelled:
|
||||
// We get cancelled e.g. when an entire simulation times out, but in that case
|
||||
// we won't be restarted and don't need to wait for shutdown
|
||||
// reboot_remote_kv_store:
|
||||
// The child process running the storage engine died abnormally,
|
||||
// the current solution is to reboot the worker.
|
||||
// Some refactoring work in the future can make it only reboot the storage server
|
||||
stopping.send(Void());
|
||||
wait(filesClosed.getResult()); // Wait for complete shutdown of KV stores
|
||||
wait(delay(0.0)); // Unwind the callstack to make sure that IAsyncFile references are all gone
|
||||
|
@ -22,6 +22,7 @@
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
#include "fdbserver/workloads/workloads.actor.h"
|
||||
#include "fdbrpc/simulator.h"
|
||||
#include "boost/algorithm/string/predicate.hpp"
|
||||
|
||||
#undef state
|
||||
#include "fdbclient/SimpleIni.h"
|
||||
@ -70,12 +71,14 @@ struct SaveAndKillWorkload : TestWorkload {
|
||||
std::map<NetworkAddress, ISimulator::ProcessInfo*> rebootingProcesses = g_simulator.currentlyRebootingProcesses;
|
||||
std::map<std::string, ISimulator::ProcessInfo*> allProcessesMap;
|
||||
for (const auto& [_, process] : rebootingProcesses) {
|
||||
if (allProcessesMap.find(process->dataFolder) == allProcessesMap.end()) {
|
||||
if (allProcessesMap.find(process->dataFolder) == allProcessesMap.end() &&
|
||||
std::string(process->name) != "remote flow process") {
|
||||
allProcessesMap[process->dataFolder] = process;
|
||||
}
|
||||
}
|
||||
for (const auto& process : processes) {
|
||||
if (allProcessesMap.find(process->dataFolder) == allProcessesMap.end()) {
|
||||
if (allProcessesMap.find(process->dataFolder) == allProcessesMap.end() &&
|
||||
std::string(process->name) != "remote flow process") {
|
||||
allProcessesMap[process->dataFolder] = process;
|
||||
}
|
||||
}
|
||||
|
@ -743,6 +743,13 @@ class Listener final : public IListener, ReferenceCounted<Listener> {
|
||||
public:
|
||||
Listener(boost::asio::io_context& io_service, NetworkAddress listenAddress)
|
||||
: io_service(io_service), listenAddress(listenAddress), acceptor(io_service, tcpEndpoint(listenAddress)) {
|
||||
// when port 0 is passed in, a random port will be opened
|
||||
// set listenAddress as the address with the actual port opened instead of port 0
|
||||
if (listenAddress.port == 0) {
|
||||
this->listenAddress =
|
||||
NetworkAddress::parse(acceptor.local_endpoint().address().to_string().append(":").append(
|
||||
std::to_string(acceptor.local_endpoint().port())));
|
||||
}
|
||||
platform::setCloseOnExec(acceptor.native_handle());
|
||||
}
|
||||
|
||||
|
@ -3755,6 +3755,40 @@ void fdb_probe_actor_exit(const char* name, unsigned long id, int index) {
|
||||
}
|
||||
#endif
|
||||
|
||||
void throwExecPathError(Error e, char path[]) {
|
||||
Severity sev = e.code() == error_code_io_error ? SevError : SevWarnAlways;
|
||||
TraceEvent(sev, "GetPathError").error(e).detail("Path", path);
|
||||
throw e;
|
||||
}
|
||||
|
||||
std::string getExecPath() {
|
||||
char path[1024];
|
||||
uint32_t size = sizeof(path);
|
||||
#if defined(__APPLE__)
|
||||
if (_NSGetExecutablePath(path, &size) == 0) {
|
||||
return std::string(path);
|
||||
} else {
|
||||
throwExecPathError(platform_error(), path);
|
||||
}
|
||||
#elif defined(__linux__)
|
||||
ssize_t len = ::readlink("/proc/self/exe", path, size);
|
||||
if (len != -1) {
|
||||
path[len] = '\0';
|
||||
return std::string(path);
|
||||
} else {
|
||||
throwExecPathError(platform_error(), path);
|
||||
}
|
||||
#elif defined(_WIN32)
|
||||
auto len = GetModuleFileName(nullptr, path, size);
|
||||
if (len != 0) {
|
||||
return std::string(path);
|
||||
} else {
|
||||
throwExecPathError(platform_error(), path);
|
||||
}
|
||||
#endif
|
||||
return "unsupported OS";
|
||||
}
|
||||
|
||||
void setupRunLoopProfiler() {
|
||||
#ifdef __linux__
|
||||
if (!profileThread && FLOW_KNOBS->RUN_LOOP_PROFILING_INTERVAL > 0) {
|
||||
|
@ -703,6 +703,9 @@ void* loadFunction(void* lib, const char* func_name);
|
||||
|
||||
std::string exePath();
|
||||
|
||||
// get the absolute path
|
||||
std::string getExecPath();
|
||||
|
||||
#ifdef _WIN32
|
||||
inline static int ctzll(uint64_t value) {
|
||||
unsigned long count = 0;
|
||||
|
@ -87,6 +87,7 @@ ERROR( blob_granule_file_load_error, 1063, "Error loading a blob file during gra
|
||||
ERROR( blob_granule_transaction_too_old, 1064, "Read version is older than blob granule history supports" )
|
||||
ERROR( blob_manager_replaced, 1065, "This blob manager has been replaced." )
|
||||
ERROR( change_feed_popped, 1066, "Tried to read a version older than what has been popped from the change feed" )
|
||||
ERROR( remote_kvs_cancelled, 1067, "The remote key-value store is cancelled" )
|
||||
|
||||
ERROR( broken_promise, 1100, "Broken promise" )
|
||||
ERROR( operation_cancelled, 1101, "Asynchronous operation cancelled" )
|
||||
@ -113,6 +114,7 @@ ERROR( dd_tracker_cancelled, 1215, "The data distribution tracker has been cance
|
||||
ERROR( failed_to_progress, 1216, "Process has failed to make sufficient progress" )
|
||||
ERROR( invalid_cluster_id, 1217, "Attempted to join cluster with a different cluster ID" )
|
||||
ERROR( restart_cluster_controller, 1218, "Restart cluster controller process" )
|
||||
ERROR( please_reboot_remote_kv_store, 1219, "Need to reboot the storage engine process as it died abnormally")
|
||||
|
||||
// 15xx Platform errors
|
||||
ERROR( platform_error, 1500, "Platform error" )
|
||||
|
@ -80,7 +80,7 @@ Future<Optional<T>> stopAfter(Future<T> what) {
|
||||
ret = Optional<T>(_);
|
||||
} catch (Error& e) {
|
||||
bool ok = e.code() == error_code_please_reboot || e.code() == error_code_please_reboot_delete ||
|
||||
e.code() == error_code_actor_cancelled;
|
||||
e.code() == error_code_actor_cancelled || e.code() == error_code_please_reboot_remote_kv_store;
|
||||
TraceEvent(ok ? SevInfo : SevError, "StopAfterError").error(e);
|
||||
if (!ok) {
|
||||
fprintf(stderr, "Fatal Error: %s\n", e.what());
|
||||
|
@ -507,6 +507,10 @@ public:
|
||||
virtual NetworkAddress getPeerAddress() const = 0;
|
||||
|
||||
virtual UID getDebugID() const = 0;
|
||||
|
||||
// At present, implemented by Sim2Conn where we want to disable bits flip for connections between parent process and
|
||||
// child process, also reduce latency for this kind of connection
|
||||
virtual bool isStableConnection() const { throw unsupported_operation(); }
|
||||
};
|
||||
|
||||
class IListener {
|
||||
|
@ -4,6 +4,7 @@ storageEngineType = 4
|
||||
processesPerMachine = 1
|
||||
coordinators = 3
|
||||
machineCount = 15
|
||||
disableRemoteKVS = true
|
||||
|
||||
[[test]]
|
||||
testTitle = 'PhysicalShardMove'
|
||||
|
@ -4,6 +4,7 @@ minimumReplication = 3
|
||||
minimumRegions = 3
|
||||
logAntiQuorum = 0
|
||||
storageEngineExcludeTypes = [4]
|
||||
disableRemoteKVS = true
|
||||
|
||||
[[test]]
|
||||
testTitle = 'DiskFailureCycle'
|
||||
|
Loading…
x
Reference in New Issue
Block a user