mirror of
https://github.com/apple/foundationdb.git
synced 2025-05-14 01:42:37 +08:00
* 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”>
504 lines
15 KiB
C++
504 lines
15 KiB
C++
/*
|
|
* 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 |