foundationdb/fdbserver/RemoteIKeyValueStore.actor.h
Chaoguang Lin 7d365bd1bb
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”>
2022-03-31 17:08:59 -07:00

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