2017-05-26 04:48:44 +08:00
|
|
|
/*
|
|
|
|
* worker.actor.cpp
|
|
|
|
*
|
|
|
|
* This source file is part of the FoundationDB open source project
|
|
|
|
*
|
2022-03-22 04:36:23 +08:00
|
|
|
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
|
2018-02-22 02:25:11 +08:00
|
|
|
*
|
2017-05-26 04:48:44 +08:00
|
|
|
* 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
|
2018-02-22 02:25:11 +08:00
|
|
|
*
|
2017-05-26 04:48:44 +08:00
|
|
|
* http://www.apache.org/licenses/LICENSE-2.0
|
2018-02-22 02:25:11 +08:00
|
|
|
*
|
2017-05-26 04:48:44 +08:00
|
|
|
* 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.
|
|
|
|
*/
|
|
|
|
|
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-04-01 08:08:59 +08:00
|
|
|
#include <cstdlib>
|
2020-05-05 16:00:17 +08:00
|
|
|
#include <tuple>
|
2019-02-23 04:15:23 +08:00
|
|
|
#include <boost/lexical_cast.hpp>
|
|
|
|
|
2022-04-07 12:06:01 +08:00
|
|
|
#include "fdbclient/FDBTypes.h"
|
2022-09-03 10:21:52 +08:00
|
|
|
#include "fdbserver/BlobMigratorInterface.h"
|
2022-09-02 15:28:13 +08:00
|
|
|
#include "flow/ApiVersion.h"
|
2022-06-24 07:03:53 +08:00
|
|
|
#include "flow/IAsyncFile.h"
|
2020-02-26 07:00:18 +08:00
|
|
|
#include "fdbrpc/Locality.h"
|
2021-06-03 11:21:44 +08:00
|
|
|
#include "fdbclient/GlobalConfig.actor.h"
|
2021-06-04 06:10:04 +08:00
|
|
|
#include "fdbclient/ProcessInterface.h"
|
2020-02-26 07:00:18 +08:00
|
|
|
#include "fdbclient/StorageServerInterface.h"
|
2020-05-09 08:14:42 +08:00
|
|
|
#include "fdbserver/Knobs.h"
|
2017-05-26 04:48:44 +08:00
|
|
|
#include "flow/ActorCollection.h"
|
2022-03-29 07:49:17 +08:00
|
|
|
#include "flow/Error.h"
|
|
|
|
#include "flow/FileIdentifier.h"
|
|
|
|
#include "flow/ObjectSerializer.h"
|
|
|
|
#include "flow/Platform.h"
|
2020-10-06 01:07:51 +08:00
|
|
|
#include "flow/ProtocolVersion.h"
|
2017-05-26 04:48:44 +08:00
|
|
|
#include "flow/SystemMonitor.h"
|
|
|
|
#include "flow/TDMetric.actor.h"
|
|
|
|
#include "fdbrpc/simulator.h"
|
2019-02-18 07:41:16 +08:00
|
|
|
#include "fdbclient/NativeAPI.actor.h"
|
2021-01-26 08:09:32 +08:00
|
|
|
#include "fdbserver/MetricLogger.actor.h"
|
2019-04-25 06:12:37 +08:00
|
|
|
#include "fdbserver/BackupInterface.h"
|
2022-08-24 14:04:12 +08:00
|
|
|
#include "fdbclient/EncryptKeyProxyInterface.h"
|
2021-06-04 06:10:04 +08:00
|
|
|
#include "fdbserver/RoleLineage.actor.h"
|
2019-02-18 11:13:26 +08:00
|
|
|
#include "fdbserver/WorkerInterface.actor.h"
|
2018-10-20 01:30:13 +08:00
|
|
|
#include "fdbserver/IKeyValueStore.h"
|
|
|
|
#include "fdbserver/WaitFailure.h"
|
2019-02-18 11:25:16 +08:00
|
|
|
#include "fdbserver/TesterInterface.actor.h" // for poisson()
|
2018-10-20 01:30:13 +08:00
|
|
|
#include "fdbserver/IDiskQueue.h"
|
2017-05-26 04:48:44 +08:00
|
|
|
#include "fdbclient/DatabaseContext.h"
|
2018-12-14 05:31:37 +08:00
|
|
|
#include "fdbserver/DataDistributorInterface.h"
|
2021-09-15 23:35:58 +08:00
|
|
|
#include "fdbserver/BlobManagerInterface.h"
|
2018-10-20 01:30:13 +08:00
|
|
|
#include "fdbserver/ServerDBInfo.h"
|
2019-04-21 03:58:24 +08:00
|
|
|
#include "fdbserver/FDBExecHelper.actor.h"
|
2017-05-26 04:48:44 +08:00
|
|
|
#include "fdbserver/CoordinationInterface.h"
|
2022-02-02 14:27:12 +08:00
|
|
|
#include "fdbserver/ConfigNode.h"
|
2021-04-17 08:58:00 +08:00
|
|
|
#include "fdbserver/LocalConfiguration.h"
|
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-04-01 08:08:59 +08:00
|
|
|
#include "fdbserver/RemoteIKeyValueStore.actor.h"
|
2017-05-26 04:48:44 +08:00
|
|
|
#include "fdbclient/MonitorLeader.h"
|
2017-10-12 05:13:16 +08:00
|
|
|
#include "fdbclient/ClientWorkerInterface.h"
|
|
|
|
#include "flow/Profiler.h"
|
2020-03-13 05:34:19 +08:00
|
|
|
#include "flow/ThreadHelper.actor.h"
|
2020-05-09 07:27:57 +08:00
|
|
|
#include "flow/Trace.h"
|
2021-06-04 06:10:04 +08:00
|
|
|
#include "flow/flow.h"
|
2022-04-02 06:45:24 +08:00
|
|
|
#include "flow/genericactors.actor.h"
|
2020-09-29 01:58:49 +08:00
|
|
|
#include "flow/network.h"
|
2022-03-29 07:49:17 +08:00
|
|
|
#include "flow/serialize.h"
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
#ifdef __linux__
|
2019-04-06 05:45:58 +08:00
|
|
|
#include <fcntl.h>
|
|
|
|
#include <stdio.h>
|
|
|
|
#include <sys/stat.h>
|
|
|
|
#include <sys/types.h>
|
|
|
|
#include <unistd.h>
|
2020-02-02 02:00:06 +08:00
|
|
|
#endif
|
|
|
|
#if defined(__linux__) || defined(__FreeBSD__)
|
2017-05-26 04:48:44 +08:00
|
|
|
#ifdef USE_GPERFTOOLS
|
|
|
|
#include "gperftools/profiler.h"
|
2019-04-04 06:57:16 +08:00
|
|
|
#include "gperftools/heap-profiler.h"
|
2017-05-26 04:48:44 +08:00
|
|
|
#endif
|
|
|
|
#include <unistd.h>
|
|
|
|
#include <thread>
|
|
|
|
#include <execinfo.h>
|
|
|
|
#endif
|
2018-08-11 06:18:24 +08:00
|
|
|
#include "flow/actorcompiler.h" // This must be the last #include.
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
#if CENABLED(0, NOT_IN_CLEAN)
|
|
|
|
extern IKeyValueStore* keyValueStoreCompressTestData(IKeyValueStore* store);
|
|
|
|
#define KV_STORE(filename, uid) keyValueStoreCompressTestData(keyValueStoreSQLite(filename, uid))
|
|
|
|
#elif CENABLED(0, NOT_IN_CLEAN)
|
|
|
|
#define KV_STORE(filename, uid) keyValueStoreSQLite(filename, uid)
|
|
|
|
#else
|
|
|
|
#define KV_STORE(filename, uid) keyValueStoreMemory(filename, uid)
|
|
|
|
#endif
|
|
|
|
|
2021-06-04 06:10:04 +08:00
|
|
|
namespace {
|
|
|
|
RoleLineageCollector roleLineageCollector;
|
|
|
|
}
|
|
|
|
|
2020-04-06 14:09:36 +08:00
|
|
|
ACTOR Future<std::vector<Endpoint>> tryDBInfoBroadcast(RequestStream<UpdateServerDBInfoRequest> stream,
|
|
|
|
UpdateServerDBInfoRequest req) {
|
2020-04-12 11:54:17 +08:00
|
|
|
ErrorOr<std::vector<Endpoint>> rep =
|
|
|
|
wait(stream.getReplyUnlessFailedFor(req, SERVER_KNOBS->DBINFO_FAILED_DELAY, 0));
|
2020-04-11 08:02:11 +08:00
|
|
|
if (rep.present()) {
|
|
|
|
return rep.get();
|
2020-04-06 14:09:36 +08:00
|
|
|
}
|
2020-04-11 08:02:11 +08:00
|
|
|
req.broadcastInfo.push_back(stream.getEndpoint());
|
|
|
|
return req.broadcastInfo;
|
2020-04-06 14:09:36 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
ACTOR Future<std::vector<Endpoint>> broadcastDBInfoRequest(UpdateServerDBInfoRequest req,
|
|
|
|
int sendAmount,
|
|
|
|
Optional<Endpoint> sender,
|
|
|
|
bool sendReply) {
|
|
|
|
state std::vector<Future<std::vector<Endpoint>>> replies;
|
|
|
|
state ReplyPromise<std::vector<Endpoint>> reply = req.reply;
|
|
|
|
resetReply(req);
|
|
|
|
int currentStream = 0;
|
2020-04-11 04:45:16 +08:00
|
|
|
std::vector<Endpoint> broadcastEndpoints = req.broadcastInfo;
|
|
|
|
for (int i = 0; i < sendAmount && currentStream < broadcastEndpoints.size(); i++) {
|
2020-04-07 11:58:43 +08:00
|
|
|
std::vector<Endpoint> endpoints;
|
2020-04-11 04:45:16 +08:00
|
|
|
RequestStream<UpdateServerDBInfoRequest> cur(broadcastEndpoints[currentStream++]);
|
|
|
|
while (currentStream < broadcastEndpoints.size() * (i + 1) / sendAmount) {
|
|
|
|
endpoints.push_back(broadcastEndpoints[currentStream++]);
|
2020-04-06 14:09:36 +08:00
|
|
|
}
|
2020-04-07 11:58:43 +08:00
|
|
|
req.broadcastInfo = endpoints;
|
2020-04-06 14:09:36 +08:00
|
|
|
replies.push_back(tryDBInfoBroadcast(cur, req));
|
|
|
|
resetReply(req);
|
|
|
|
}
|
|
|
|
wait(waitForAll(replies));
|
|
|
|
std::vector<Endpoint> notUpdated;
|
|
|
|
if (sender.present()) {
|
|
|
|
notUpdated.push_back(sender.get());
|
|
|
|
}
|
|
|
|
for (auto& it : replies) {
|
|
|
|
notUpdated.insert(notUpdated.end(), it.get().begin(), it.get().end());
|
|
|
|
}
|
|
|
|
if (sendReply) {
|
|
|
|
reply.send(notUpdated);
|
|
|
|
}
|
|
|
|
return notUpdated;
|
|
|
|
}
|
2019-04-10 02:17:58 +08:00
|
|
|
|
2021-07-12 12:11:21 +08:00
|
|
|
ACTOR static Future<Void> extractClientInfo(Reference<AsyncVar<ServerDBInfo> const> db,
|
2017-05-26 04:48:44 +08:00
|
|
|
Reference<AsyncVar<ClientDBInfo>> info) {
|
2020-09-11 08:44:15 +08:00
|
|
|
state std::vector<UID> lastCommitProxyUIDs;
|
|
|
|
state std::vector<CommitProxyInterface> lastCommitProxies;
|
2020-07-15 15:37:41 +08:00
|
|
|
state std::vector<UID> lastGrvProxyUIDs;
|
|
|
|
state std::vector<GrvProxyInterface> lastGrvProxies;
|
2017-05-26 04:48:44 +08:00
|
|
|
loop {
|
2020-01-11 04:20:30 +08:00
|
|
|
ClientDBInfo ni = db->get().client;
|
2020-09-11 08:44:15 +08:00
|
|
|
shrinkProxyList(ni, lastCommitProxyUIDs, lastCommitProxies, lastGrvProxyUIDs, lastGrvProxies);
|
2022-04-16 12:48:12 +08:00
|
|
|
info->setUnconditional(ni);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(db->onChange());
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-07-12 12:11:21 +08:00
|
|
|
Database openDBOnServer(Reference<AsyncVar<ServerDBInfo> const> const& db,
|
2019-06-25 17:47:35 +08:00
|
|
|
TaskPriority taskID,
|
2021-07-03 06:04:42 +08:00
|
|
|
LockAware lockAware,
|
|
|
|
EnableLocalityLoadBalance enableLocalityLoadBalance) {
|
2020-11-07 15:50:55 +08:00
|
|
|
auto info = makeReference<AsyncVar<ClientDBInfo>>();
|
2021-06-03 11:21:44 +08:00
|
|
|
auto cx = DatabaseContext::create(info,
|
|
|
|
extractClientInfo(db, info),
|
|
|
|
enableLocalityLoadBalance ? db->get().myLocality : LocalityData(),
|
|
|
|
enableLocalityLoadBalance,
|
|
|
|
taskID,
|
|
|
|
lockAware);
|
2022-05-05 07:36:04 +08:00
|
|
|
cx->globalConfig->init(db, std::addressof(db->get().client));
|
|
|
|
cx->globalConfig->trigger(samplingFrequency, samplingProfilerUpdateFrequency);
|
|
|
|
cx->globalConfig->trigger(samplingWindow, samplingProfilerUpdateWindow);
|
2021-06-03 11:21:44 +08:00
|
|
|
return cx;
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
struct ErrorInfo {
|
|
|
|
Error error;
|
2018-09-06 06:06:14 +08:00
|
|
|
const Role& role;
|
2017-05-26 04:48:44 +08:00
|
|
|
UID id;
|
2018-09-06 06:06:14 +08:00
|
|
|
ErrorInfo(Error e, const Role& role, UID id) : error(e), role(role), id(id) {}
|
2017-05-26 04:48:44 +08:00
|
|
|
template <class Ar>
|
|
|
|
void serialize(Ar&) {
|
|
|
|
ASSERT(false);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2017-05-27 08:43:28 +08:00
|
|
|
Error checkIOTimeout(Error const& e) {
|
2017-06-01 08:03:15 +08:00
|
|
|
// Convert all_errors to io_timeout if global timeout bool was set
|
2017-06-16 08:40:19 +08:00
|
|
|
bool timeoutOccurred = (bool)g_network->global(INetwork::enASIOTimedOut);
|
|
|
|
// In simulation, have to check global timed out flag for both this process and the machine process on which IO is
|
|
|
|
// done
|
|
|
|
if (g_network->isSimulated() && !timeoutOccurred)
|
2022-09-15 08:10:49 +08:00
|
|
|
timeoutOccurred = g_simulator->getCurrentProcess()->machine->machineProcess->global(INetwork::enASIOTimedOut);
|
2017-06-16 08:40:19 +08:00
|
|
|
|
|
|
|
if (timeoutOccurred) {
|
2022-07-20 04:15:51 +08:00
|
|
|
CODE_PROBE(true, "Timeout occurred");
|
2017-05-27 08:43:28 +08:00
|
|
|
Error timeout = io_timeout();
|
2017-06-16 08:40:19 +08:00
|
|
|
// Preserve injectedness of error
|
|
|
|
if (e.isInjectedFault())
|
2017-05-27 08:43:28 +08:00
|
|
|
timeout = timeout.asInjectedFault();
|
|
|
|
return timeout;
|
|
|
|
}
|
|
|
|
return e;
|
|
|
|
}
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
ACTOR Future<Void> forwardError(PromiseStream<ErrorInfo> errors, Role role, UID id, Future<Void> process) {
|
|
|
|
try {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(process);
|
2018-09-06 06:06:14 +08:00
|
|
|
errors.send(ErrorInfo(success(), role, id));
|
2017-05-26 04:48:44 +08:00
|
|
|
return Void();
|
|
|
|
} catch (Error& e) {
|
2018-09-06 06:06:14 +08:00
|
|
|
errors.send(ErrorInfo(e, role, id));
|
2017-05-26 04:48:44 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
ACTOR Future<Void> handleIOErrors(Future<Void> actor, IClosable* store, UID id, Future<Void> onClosed = Void()) {
|
2018-07-07 05:41:36 +08:00
|
|
|
state Future<ErrorOr<Void>> storeError = actor.isReady() ? Never() : errorOr(store->getError());
|
2017-05-26 04:48:44 +08:00
|
|
|
choose {
|
|
|
|
when(state ErrorOr<Void> e = wait(errorOr(actor))) {
|
2018-10-20 09:55:35 +08:00
|
|
|
if (e.isError() && e.getError().code() == error_code_please_reboot) {
|
2018-10-09 08:26:10 +08:00
|
|
|
// no need to wait.
|
|
|
|
} else {
|
2018-10-24 08:05:42 +08:00
|
|
|
wait(onClosed);
|
2018-10-09 08:26:10 +08:00
|
|
|
}
|
2019-07-10 07:37:54 +08:00
|
|
|
if (e.isError() && e.getError().code() == error_code_broken_promise && !storeError.isReady()) {
|
2021-06-23 05:50:01 +08:00
|
|
|
wait(delay(0.00001 + FLOW_KNOBS->MAX_BUGGIFIED_DELAY));
|
2019-07-10 07:37:54 +08:00
|
|
|
}
|
2022-08-24 15:40:38 +08:00
|
|
|
if (storeError.isReady() && storeError.isError() &&
|
|
|
|
storeError.getError().code() != error_code_file_not_found) {
|
|
|
|
throw storeError.get().getError();
|
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-04-01 08:08:59 +08:00
|
|
|
}
|
|
|
|
if (e.isError()) {
|
2017-05-26 04:48:44 +08:00
|
|
|
throw e.getError();
|
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-04-01 08:08:59 +08:00
|
|
|
} else
|
2017-05-26 04:48:44 +08:00
|
|
|
return e.get();
|
|
|
|
}
|
2018-07-07 05:41:36 +08:00
|
|
|
when(ErrorOr<Void> e = wait(storeError)) {
|
2022-08-24 15:40:38 +08:00
|
|
|
TraceEvent("WorkerTerminatingByIOError", id).errorUnsuppressed(e.getError());
|
2017-05-26 04:48:44 +08:00
|
|
|
actor.cancel();
|
|
|
|
// file_not_found can occur due to attempting to open a partially deleted DiskQueue, which should not be
|
|
|
|
// reported SevError.
|
2022-08-24 15:40:38 +08:00
|
|
|
if (e.getError().code() == error_code_file_not_found) {
|
2022-07-20 04:15:51 +08:00
|
|
|
CODE_PROBE(true, "Worker terminated with file_not_found error");
|
2017-05-26 04:48:44 +08:00
|
|
|
return Void();
|
2022-08-24 15:40:38 +08:00
|
|
|
} else if (e.getError().code() == error_code_lock_file_failure) {
|
2022-09-26 06:28:32 +08:00
|
|
|
CODE_PROBE(true, "Unable to lock file", probe::decoration::rare);
|
2022-08-24 15:40:38 +08:00
|
|
|
throw please_reboot_kv_store();
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2022-08-24 15:40:38 +08:00
|
|
|
throw e.getError();
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-05-27 08:43:28 +08:00
|
|
|
ACTOR Future<Void> workerHandleErrors(FutureStream<ErrorInfo> errors) {
|
2017-05-26 04:48:44 +08:00
|
|
|
loop choose {
|
2017-05-27 08:43:28 +08:00
|
|
|
when(ErrorInfo _err = waitNext(errors)) {
|
|
|
|
ErrorInfo err = _err;
|
2017-05-26 04:48:44 +08:00
|
|
|
bool ok = err.error.code() == error_code_success || err.error.code() == error_code_please_reboot ||
|
|
|
|
err.error.code() == error_code_actor_cancelled ||
|
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-04-01 08:08:59 +08:00
|
|
|
err.error.code() == error_code_remote_kvs_cancelled ||
|
2018-10-15 18:43:43 +08:00
|
|
|
err.error.code() == error_code_coordinators_changed || // The worker server was cancelled
|
|
|
|
err.error.code() == error_code_shutdown_in_progress;
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2018-11-22 03:18:26 +08:00
|
|
|
if (!ok) {
|
2017-06-16 08:40:19 +08:00
|
|
|
err.error = checkIOTimeout(err.error); // Possibly convert error to io_timeout
|
2018-08-30 05:40:39 +08:00
|
|
|
}
|
2017-06-16 08:40:19 +08:00
|
|
|
|
2018-09-06 06:06:14 +08:00
|
|
|
endRole(err.role, err.id, "Error", ok, err.error);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-01-28 05:18:13 +08:00
|
|
|
if (err.error.code() == error_code_please_reboot ||
|
|
|
|
(err.role == Role::SHARED_TRANSACTION_LOG &&
|
2022-09-14 07:59:45 +08:00
|
|
|
(err.error.code() == error_code_io_error || err.error.code() == error_code_io_timeout)) ||
|
|
|
|
(SERVER_KNOBS->STORAGE_SERVER_REBOOT_ON_IO_TIMEOUT && err.role == Role::STORAGE_SERVER &&
|
|
|
|
err.error.code() == error_code_io_timeout))
|
2021-01-28 05:18:13 +08:00
|
|
|
throw err.error;
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Improve simulation code coverage by sometimes deferring the destruction of workerInterface (and therefore "endpoint
|
|
|
|
// not found" responses to clients
|
|
|
|
// for an extra second, so that clients are more likely to see broken_promise errors
|
|
|
|
ACTOR template <class T>
|
|
|
|
Future<Void> zombie(T workerInterface, Future<Void> worker) {
|
|
|
|
try {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(worker);
|
2017-05-26 04:48:44 +08:00
|
|
|
if (BUGGIFY)
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(delay(1.0));
|
2017-05-26 04:48:44 +08:00
|
|
|
return Void();
|
|
|
|
} catch (Error& e) {
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
ACTOR Future<Void> loadedPonger(FutureStream<LoadedPingRequest> pings) {
|
2019-02-18 10:46:59 +08:00
|
|
|
state Standalone<StringRef> payloadBack(std::string(20480, '.'));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
loop {
|
|
|
|
LoadedPingRequest pong = waitNext(pings);
|
|
|
|
LoadedReply rep;
|
2022-09-20 02:35:58 +08:00
|
|
|
rep.payload = (pong.loadReply ? payloadBack : ""_sr);
|
2017-05-26 04:48:44 +08:00
|
|
|
rep.id = pong.id;
|
|
|
|
pong.reply.send(rep);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-09-20 02:35:58 +08:00
|
|
|
StringRef fileStoragePrefix = "storage-"_sr;
|
|
|
|
StringRef testingStoragePrefix = "testingstorage-"_sr;
|
|
|
|
StringRef fileLogDataPrefix = "log-"_sr;
|
|
|
|
StringRef fileVersionedLogDataPrefix = "log2-"_sr;
|
|
|
|
StringRef fileLogQueuePrefix = "logqueue-"_sr;
|
|
|
|
StringRef tlogQueueExtension = "fdq"_sr;
|
2017-09-22 14:51:55 +08:00
|
|
|
|
2020-06-30 00:25:36 +08:00
|
|
|
enum class FilesystemCheck {
|
|
|
|
FILES_ONLY,
|
|
|
|
DIRECTORIES_ONLY,
|
|
|
|
FILES_AND_DIRECTORIES,
|
|
|
|
};
|
|
|
|
|
|
|
|
struct KeyValueStoreSuffix {
|
|
|
|
KeyValueStoreType type;
|
|
|
|
std::string suffix;
|
|
|
|
FilesystemCheck check;
|
|
|
|
};
|
|
|
|
|
|
|
|
KeyValueStoreSuffix bTreeV1Suffix = { KeyValueStoreType::SSD_BTREE_V1, ".fdb", FilesystemCheck::FILES_ONLY };
|
|
|
|
KeyValueStoreSuffix bTreeV2Suffix = { KeyValueStoreType::SSD_BTREE_V2, ".sqlite", FilesystemCheck::FILES_ONLY };
|
|
|
|
KeyValueStoreSuffix memorySuffix = { KeyValueStoreType::MEMORY, "-0.fdq", FilesystemCheck::FILES_ONLY };
|
|
|
|
KeyValueStoreSuffix memoryRTSuffix = { KeyValueStoreType::MEMORY_RADIXTREE, "-0.fdr", FilesystemCheck::FILES_ONLY };
|
2021-11-16 18:15:22 +08:00
|
|
|
KeyValueStoreSuffix redwoodSuffix = { KeyValueStoreType::SSD_REDWOOD_V1, ".redwood-v1", FilesystemCheck::FILES_ONLY };
|
2020-06-30 00:25:36 +08:00
|
|
|
KeyValueStoreSuffix rocksdbSuffix = { KeyValueStoreType::SSD_ROCKSDB_V1,
|
|
|
|
".rocksdb",
|
|
|
|
FilesystemCheck::DIRECTORIES_ONLY };
|
2022-04-22 10:53:14 +08:00
|
|
|
KeyValueStoreSuffix shardedRocksdbSuffix = { KeyValueStoreType::SSD_SHARDED_ROCKSDB,
|
|
|
|
".shardedrocksdb",
|
|
|
|
FilesystemCheck::DIRECTORIES_ONLY };
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
std::string validationFilename = "_validate";
|
|
|
|
|
|
|
|
std::string filenameFromSample(KeyValueStoreType storeType, std::string folder, std::string sample_filename) {
|
|
|
|
if (storeType == KeyValueStoreType::SSD_BTREE_V1)
|
|
|
|
return joinPath(folder, sample_filename);
|
|
|
|
else if (storeType == KeyValueStoreType::SSD_BTREE_V2)
|
2017-08-29 02:25:37 +08:00
|
|
|
return joinPath(folder, sample_filename);
|
2020-02-18 09:31:41 +08:00
|
|
|
else if (storeType == KeyValueStoreType::MEMORY || storeType == KeyValueStoreType::MEMORY_RADIXTREE)
|
2017-05-26 04:48:44 +08:00
|
|
|
return joinPath(folder, sample_filename.substr(0, sample_filename.size() - 5));
|
2017-09-22 14:51:55 +08:00
|
|
|
else if (storeType == KeyValueStoreType::SSD_REDWOOD_V1)
|
|
|
|
return joinPath(folder, sample_filename);
|
2020-06-16 00:45:36 +08:00
|
|
|
else if (storeType == KeyValueStoreType::SSD_ROCKSDB_V1)
|
|
|
|
return joinPath(folder, sample_filename);
|
2022-04-22 10:53:14 +08:00
|
|
|
else if (storeType == KeyValueStoreType::SSD_SHARDED_ROCKSDB)
|
|
|
|
return joinPath(folder, sample_filename);
|
2017-05-26 04:48:44 +08:00
|
|
|
UNREACHABLE();
|
|
|
|
}
|
|
|
|
|
|
|
|
std::string filenameFromId(KeyValueStoreType storeType, std::string folder, std::string prefix, UID id) {
|
2021-03-06 03:28:15 +08:00
|
|
|
|
2020-02-13 08:55:33 +08:00
|
|
|
if (storeType == KeyValueStoreType::SSD_BTREE_V1)
|
2017-05-26 04:48:44 +08:00
|
|
|
return joinPath(folder, prefix + id.toString() + ".fdb");
|
|
|
|
else if (storeType == KeyValueStoreType::SSD_BTREE_V2)
|
2017-08-29 02:25:37 +08:00
|
|
|
return joinPath(folder, prefix + id.toString() + ".sqlite");
|
2020-02-13 08:55:33 +08:00
|
|
|
else if (storeType == KeyValueStoreType::MEMORY || storeType == KeyValueStoreType::MEMORY_RADIXTREE)
|
2017-05-26 04:48:44 +08:00
|
|
|
return joinPath(folder, prefix + id.toString() + "-");
|
2017-09-22 14:51:55 +08:00
|
|
|
else if (storeType == KeyValueStoreType::SSD_REDWOOD_V1)
|
2021-11-16 18:15:22 +08:00
|
|
|
return joinPath(folder, prefix + id.toString() + ".redwood-v1");
|
2020-06-16 00:45:36 +08:00
|
|
|
else if (storeType == KeyValueStoreType::SSD_ROCKSDB_V1)
|
|
|
|
return joinPath(folder, prefix + id.toString() + ".rocksdb");
|
2022-04-22 10:53:14 +08:00
|
|
|
else if (storeType == KeyValueStoreType::SSD_SHARDED_ROCKSDB)
|
|
|
|
return joinPath(folder, prefix + id.toString() + ".shardedrocksdb");
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-05-13 02:53:20 +08:00
|
|
|
TraceEvent(SevError, "UnknownStoreType").detail("StoreType", storeType.toString());
|
2017-05-26 04:48:44 +08:00
|
|
|
UNREACHABLE();
|
|
|
|
}
|
|
|
|
|
2019-02-23 04:15:23 +08:00
|
|
|
struct TLogOptions {
|
|
|
|
TLogOptions() = default;
|
|
|
|
TLogOptions(TLogVersion v, TLogSpillType s) : version(v), spillType(s) {}
|
|
|
|
|
|
|
|
TLogVersion version = TLogVersion::DEFAULT;
|
2019-10-03 16:27:36 +08:00
|
|
|
TLogSpillType spillType = TLogSpillType::UNSET;
|
2019-02-23 04:15:23 +08:00
|
|
|
|
|
|
|
static ErrorOr<TLogOptions> FromStringRef(StringRef s) {
|
|
|
|
TLogOptions options;
|
2019-02-27 08:47:04 +08:00
|
|
|
for (StringRef key = s.eat("_"), value = s.eat("_"); s.size() != 0 || key.size();
|
|
|
|
key = s.eat("_"), value = s.eat("_")) {
|
|
|
|
if (key.size() != 0 && value.size() == 0)
|
|
|
|
return default_error_or();
|
2019-02-23 04:15:23 +08:00
|
|
|
|
2022-09-20 02:35:58 +08:00
|
|
|
if (key == "V"_sr) {
|
2019-02-23 04:15:23 +08:00
|
|
|
ErrorOr<TLogVersion> tLogVersion = TLogVersion::FromStringRef(value);
|
|
|
|
if (tLogVersion.isError())
|
|
|
|
return tLogVersion.getError();
|
|
|
|
options.version = tLogVersion.get();
|
2022-09-20 02:35:58 +08:00
|
|
|
} else if (key == "LS"_sr) {
|
2019-02-23 04:15:23 +08:00
|
|
|
ErrorOr<TLogSpillType> tLogSpillType = TLogSpillType::FromStringRef(value);
|
|
|
|
if (tLogSpillType.isError())
|
|
|
|
return tLogSpillType.getError();
|
|
|
|
options.spillType = tLogSpillType.get();
|
|
|
|
} else {
|
|
|
|
return default_error_or();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return options;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool operator==(const TLogOptions& o) {
|
2019-10-03 16:27:36 +08:00
|
|
|
return version == o.version && (spillType == o.spillType || version >= TLogVersion::V5);
|
2019-02-23 04:15:23 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
std::string toPrefix() const {
|
2019-10-03 16:27:36 +08:00
|
|
|
std::string toReturn = "";
|
|
|
|
switch (version) {
|
|
|
|
case TLogVersion::UNSET:
|
|
|
|
ASSERT(false);
|
|
|
|
case TLogVersion::V2:
|
|
|
|
return "";
|
|
|
|
case TLogVersion::V3:
|
|
|
|
case TLogVersion::V4:
|
|
|
|
toReturn =
|
|
|
|
"V_" + boost::lexical_cast<std::string>(version) + "_LS_" + boost::lexical_cast<std::string>(spillType);
|
|
|
|
break;
|
|
|
|
case TLogVersion::V5:
|
2020-09-05 07:57:36 +08:00
|
|
|
case TLogVersion::V6:
|
2022-02-04 06:26:10 +08:00
|
|
|
case TLogVersion::V7:
|
2019-10-03 16:27:36 +08:00
|
|
|
toReturn = "V_" + boost::lexical_cast<std::string>(version);
|
|
|
|
break;
|
|
|
|
}
|
2019-02-23 04:15:23 +08:00
|
|
|
ASSERT_WE_THINK(FromStringRef(toReturn).get() == *this);
|
|
|
|
return toReturn + "-";
|
|
|
|
}
|
2022-02-04 06:26:10 +08:00
|
|
|
|
|
|
|
DiskQueueVersion getDiskQueueVersion() const {
|
|
|
|
if (version < TLogVersion::V3)
|
|
|
|
return DiskQueueVersion::V0;
|
|
|
|
if (version < TLogVersion::V7)
|
|
|
|
return DiskQueueVersion::V1;
|
|
|
|
return DiskQueueVersion::V2;
|
|
|
|
}
|
2019-02-23 04:15:23 +08:00
|
|
|
};
|
|
|
|
|
|
|
|
TLogFn tLogFnForOptions(TLogOptions options) {
|
2019-10-03 16:25:42 +08:00
|
|
|
switch (options.version) {
|
2019-07-09 13:22:45 +08:00
|
|
|
case TLogVersion::V2:
|
2019-10-03 16:25:42 +08:00
|
|
|
if (options.spillType == TLogSpillType::REFERENCE)
|
2019-07-09 13:22:45 +08:00
|
|
|
ASSERT(false);
|
2019-10-03 16:25:42 +08:00
|
|
|
return oldTLog_6_0::tLog;
|
2019-07-09 13:22:45 +08:00
|
|
|
case TLogVersion::V3:
|
|
|
|
case TLogVersion::V4:
|
2019-10-03 16:25:42 +08:00
|
|
|
if (options.spillType == TLogSpillType::VALUE)
|
|
|
|
return oldTLog_6_0::tLog;
|
2021-03-11 02:06:03 +08:00
|
|
|
else
|
2019-10-03 16:25:42 +08:00
|
|
|
return oldTLog_6_2::tLog;
|
2019-10-03 08:00:24 +08:00
|
|
|
case TLogVersion::V5:
|
2020-09-05 07:57:36 +08:00
|
|
|
case TLogVersion::V6:
|
2022-02-04 06:26:10 +08:00
|
|
|
case TLogVersion::V7:
|
2019-10-03 08:00:24 +08:00
|
|
|
return tLog;
|
2021-03-11 02:06:03 +08:00
|
|
|
default:
|
2019-07-09 13:22:45 +08:00
|
|
|
ASSERT(false);
|
|
|
|
}
|
2019-10-03 08:00:24 +08:00
|
|
|
return tLog;
|
2019-02-23 04:15:23 +08:00
|
|
|
}
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
struct DiskStore {
|
2019-02-08 09:02:47 +08:00
|
|
|
enum COMPONENT { TLogData, Storage, UNSET };
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2019-02-08 09:02:47 +08:00
|
|
|
UID storeID = UID();
|
|
|
|
std::string filename = ""; // For KVStoreMemory just the base filename to be passed to IDiskQueue
|
|
|
|
COMPONENT storedComponent = UNSET;
|
|
|
|
KeyValueStoreType storeType = KeyValueStoreType::END;
|
2019-02-23 04:15:23 +08:00
|
|
|
TLogOptions tLogOptions;
|
2017-05-26 04:48:44 +08:00
|
|
|
};
|
|
|
|
|
2020-06-30 00:25:36 +08:00
|
|
|
std::vector<DiskStore> getDiskStores(std::string folder,
|
|
|
|
std::string suffix,
|
|
|
|
KeyValueStoreType type,
|
|
|
|
FilesystemCheck check) {
|
2017-05-26 04:48:44 +08:00
|
|
|
std::vector<DiskStore> result;
|
2021-09-17 08:42:34 +08:00
|
|
|
std::vector<std::string> files;
|
2020-06-30 00:25:36 +08:00
|
|
|
|
|
|
|
if (check == FilesystemCheck::FILES_ONLY || check == FilesystemCheck::FILES_AND_DIRECTORIES) {
|
|
|
|
files = platform::listFiles(folder, suffix);
|
|
|
|
}
|
|
|
|
if (check == FilesystemCheck::DIRECTORIES_ONLY || check == FilesystemCheck::FILES_AND_DIRECTORIES) {
|
|
|
|
for (const auto& directory : platform::listDirectories(folder)) {
|
|
|
|
if (StringRef(directory).endsWith(suffix)) {
|
|
|
|
files.push_back(directory);
|
|
|
|
}
|
2020-06-19 01:21:14 +08:00
|
|
|
}
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
for (int idx = 0; idx < files.size(); idx++) {
|
|
|
|
DiskStore store;
|
|
|
|
store.storeType = type;
|
|
|
|
|
2019-02-20 14:02:07 +08:00
|
|
|
StringRef filename = StringRef(files[idx]);
|
|
|
|
Standalone<StringRef> prefix;
|
|
|
|
if (filename.startsWith(fileStoragePrefix)) {
|
2017-05-26 04:48:44 +08:00
|
|
|
store.storedComponent = DiskStore::Storage;
|
|
|
|
prefix = fileStoragePrefix;
|
2021-03-06 03:28:15 +08:00
|
|
|
} else if (filename.startsWith(testingStoragePrefix)) {
|
|
|
|
store.storedComponent = DiskStore::Storage;
|
|
|
|
prefix = testingStoragePrefix;
|
2019-02-20 14:02:07 +08:00
|
|
|
} else if (filename.startsWith(fileVersionedLogDataPrefix)) {
|
2019-02-08 09:02:47 +08:00
|
|
|
store.storedComponent = DiskStore::TLogData;
|
2019-02-27 08:47:04 +08:00
|
|
|
// Use the option string that's in the file rather than tLogOptions.toPrefix(),
|
|
|
|
// because they might be different if a new option was introduced in this version.
|
2019-02-23 04:15:23 +08:00
|
|
|
StringRef optionsString = filename.removePrefix(fileVersionedLogDataPrefix).eat("-");
|
2019-03-19 06:03:43 +08:00
|
|
|
TraceEvent("DiskStoreVersioned").detail("Filename", filename);
|
2019-02-23 04:15:23 +08:00
|
|
|
ErrorOr<TLogOptions> tLogOptions = TLogOptions::FromStringRef(optionsString);
|
|
|
|
if (tLogOptions.isError()) {
|
2019-03-19 06:03:43 +08:00
|
|
|
TraceEvent(SevWarn, "DiskStoreMalformedFilename").detail("Filename", filename);
|
2019-02-23 04:15:23 +08:00
|
|
|
continue;
|
|
|
|
}
|
2019-03-19 06:03:43 +08:00
|
|
|
TraceEvent("DiskStoreVersionedSuccess").detail("Filename", filename);
|
2019-02-23 04:15:23 +08:00
|
|
|
store.tLogOptions = tLogOptions.get();
|
|
|
|
prefix = filename.substr(0, fileVersionedLogDataPrefix.size() + optionsString.size() + 1);
|
2019-02-20 14:02:07 +08:00
|
|
|
} else if (filename.startsWith(fileLogDataPrefix)) {
|
2019-03-19 06:03:43 +08:00
|
|
|
TraceEvent("DiskStoreUnversioned").detail("Filename", filename);
|
2017-05-26 04:48:44 +08:00
|
|
|
store.storedComponent = DiskStore::TLogData;
|
2019-02-23 04:15:23 +08:00
|
|
|
store.tLogOptions.version = TLogVersion::V2;
|
|
|
|
store.tLogOptions.spillType = TLogSpillType::VALUE;
|
2017-05-26 04:48:44 +08:00
|
|
|
prefix = fileLogDataPrefix;
|
2021-04-17 08:58:00 +08:00
|
|
|
} else {
|
2017-05-26 04:48:44 +08:00
|
|
|
continue;
|
2021-04-17 08:58:00 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
store.storeID = UID::fromString(files[idx].substr(prefix.size(), 32));
|
|
|
|
store.filename = filenameFromSample(type, folder, files[idx]);
|
|
|
|
result.push_back(store);
|
|
|
|
}
|
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
|
|
|
std::vector<DiskStore> getDiskStores(std::string folder) {
|
2020-06-30 00:25:36 +08:00
|
|
|
auto result = getDiskStores(folder, bTreeV1Suffix.suffix, bTreeV1Suffix.type, bTreeV1Suffix.check);
|
|
|
|
auto result1 = getDiskStores(folder, bTreeV2Suffix.suffix, bTreeV2Suffix.type, bTreeV2Suffix.check);
|
2017-05-26 04:48:44 +08:00
|
|
|
result.insert(result.end(), result1.begin(), result1.end());
|
2020-06-30 00:25:36 +08:00
|
|
|
auto result2 = getDiskStores(folder, memorySuffix.suffix, memorySuffix.type, memorySuffix.check);
|
2017-05-26 04:48:44 +08:00
|
|
|
result.insert(result.end(), result2.begin(), result2.end());
|
2020-06-30 00:25:36 +08:00
|
|
|
auto result3 = getDiskStores(folder, redwoodSuffix.suffix, redwoodSuffix.type, redwoodSuffix.check);
|
2017-09-22 14:51:55 +08:00
|
|
|
result.insert(result.end(), result3.begin(), result3.end());
|
2020-06-30 00:25:36 +08:00
|
|
|
auto result4 = getDiskStores(folder, memoryRTSuffix.suffix, memoryRTSuffix.type, memoryRTSuffix.check);
|
2020-06-16 00:45:36 +08:00
|
|
|
result.insert(result.end(), result4.begin(), result4.end());
|
2020-06-30 00:25:36 +08:00
|
|
|
auto result5 = getDiskStores(folder, rocksdbSuffix.suffix, rocksdbSuffix.type, rocksdbSuffix.check);
|
2020-06-16 00:48:19 +08:00
|
|
|
result.insert(result.end(), result5.begin(), result5.end());
|
2022-04-22 10:53:14 +08:00
|
|
|
auto result6 =
|
|
|
|
getDiskStores(folder, shardedRocksdbSuffix.suffix, shardedRocksdbSuffix.type, shardedRocksdbSuffix.check);
|
|
|
|
result.insert(result.end(), result6.begin(), result6.end());
|
2017-05-26 04:48:44 +08:00
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
2020-03-13 01:18:31 +08:00
|
|
|
// Register the worker interf to cluster controller (cc) and
|
|
|
|
// re-register the worker when key roles interface, e.g., cc, dd, ratekeeper, change.
|
2022-03-18 03:40:34 +08:00
|
|
|
ACTOR Future<Void> registrationClient(
|
|
|
|
Reference<AsyncVar<Optional<ClusterControllerFullInterface>> const> ccInterface,
|
|
|
|
WorkerInterface interf,
|
|
|
|
Reference<AsyncVar<ClusterControllerPriorityInfo>> asyncPriorityInfo,
|
|
|
|
ProcessClass initialClass,
|
|
|
|
Reference<AsyncVar<Optional<DataDistributorInterface>> const> ddInterf,
|
|
|
|
Reference<AsyncVar<Optional<RatekeeperInterface>> const> rkInterf,
|
|
|
|
Reference<AsyncVar<Optional<std::pair<int64_t, BlobManagerInterface>>> const> bmInterf,
|
2022-09-03 10:21:52 +08:00
|
|
|
Reference<AsyncVar<Optional<BlobMigratorInterface>> const> blobMigratorInterf,
|
2022-03-18 03:40:34 +08:00
|
|
|
Reference<AsyncVar<Optional<EncryptKeyProxyInterface>> const> ekpInterf,
|
2022-09-17 00:03:06 +08:00
|
|
|
Reference<AsyncVar<Optional<ConsistencyScanInterface>> const> csInterf,
|
2022-03-18 03:40:34 +08:00
|
|
|
Reference<AsyncVar<bool> const> degraded,
|
|
|
|
Reference<IClusterConnectionRecord> connRecord,
|
|
|
|
Reference<AsyncVar<std::set<std::string>> const> issues,
|
|
|
|
Reference<ConfigNode> configNode,
|
|
|
|
Reference<LocalConfiguration> localConfig,
|
2022-08-17 08:21:49 +08:00
|
|
|
ConfigBroadcastInterface configBroadcastInterface,
|
2022-07-13 01:19:32 +08:00
|
|
|
Reference<AsyncVar<ServerDBInfo>> dbInfo,
|
2022-10-14 08:30:40 +08:00
|
|
|
Promise<Void> recoveredDiskFiles,
|
|
|
|
Reference<AsyncVar<Optional<UID>>> clusterId) {
|
2017-05-26 04:48:44 +08:00
|
|
|
// Keeps the cluster controller (as it may be re-elected) informed that this worker exists
|
|
|
|
// The cluster controller uses waitFailureClient to find out if we die, and returns from registrationReply
|
2019-01-29 03:29:39 +08:00
|
|
|
// (requiring us to re-register) The registration request piggybacks optional distributor interface if it exists.
|
2017-05-26 04:48:44 +08:00
|
|
|
state Generation requestGeneration = 0;
|
2018-02-10 08:48:55 +08:00
|
|
|
state ProcessClass processClass = initialClass;
|
2019-11-13 05:01:29 +08:00
|
|
|
state Reference<AsyncVar<Optional<std::pair<uint16_t, StorageServerInterface>>>> scInterf(
|
|
|
|
new AsyncVar<Optional<std::pair<uint16_t, StorageServerInterface>>>());
|
|
|
|
state Future<Void> cacheProcessFuture;
|
|
|
|
state Future<Void> cacheErrorsFuture;
|
2020-04-06 14:09:36 +08:00
|
|
|
state Optional<double> incorrectTime;
|
2021-11-10 01:44:07 +08:00
|
|
|
state bool firstReg = true;
|
2017-05-26 04:48:44 +08:00
|
|
|
loop {
|
2021-10-11 11:44:56 +08:00
|
|
|
state ClusterConnectionString storedConnectionString;
|
|
|
|
state bool upToDate = true;
|
|
|
|
if (connRecord) {
|
|
|
|
bool upToDateResult = wait(connRecord->upToDate(storedConnectionString));
|
|
|
|
upToDate = upToDateResult;
|
|
|
|
}
|
|
|
|
if (upToDate) {
|
|
|
|
incorrectTime = Optional<double>();
|
|
|
|
}
|
|
|
|
|
2022-07-30 08:28:34 +08:00
|
|
|
RegisterWorkerRequest request(
|
|
|
|
interf,
|
|
|
|
initialClass,
|
|
|
|
processClass,
|
|
|
|
asyncPriorityInfo->get(),
|
|
|
|
requestGeneration++,
|
|
|
|
ddInterf->get(),
|
|
|
|
rkInterf->get(),
|
|
|
|
bmInterf->get().present() ? bmInterf->get().get().second : Optional<BlobManagerInterface>(),
|
2022-09-03 10:21:52 +08:00
|
|
|
blobMigratorInterf->get(),
|
2022-07-30 08:28:34 +08:00
|
|
|
ekpInterf->get(),
|
2022-09-17 00:03:06 +08:00
|
|
|
csInterf->get(),
|
2022-07-30 08:28:34 +08:00
|
|
|
degraded->get(),
|
|
|
|
localConfig.isValid() ? localConfig->lastSeenVersion() : Optional<Version>(),
|
|
|
|
localConfig.isValid() ? localConfig->configClassSet() : Optional<ConfigClassSet>(),
|
2022-08-17 08:21:49 +08:00
|
|
|
recoveredDiskFiles.isSet(),
|
2022-10-14 08:30:40 +08:00
|
|
|
configBroadcastInterface,
|
|
|
|
clusterId->get());
|
2021-10-11 11:44:56 +08:00
|
|
|
|
2020-04-07 12:09:44 +08:00
|
|
|
for (auto const& i : issues->get()) {
|
|
|
|
request.issues.push_back_deep(request.issues.arena(), i);
|
|
|
|
}
|
2021-10-11 11:44:56 +08:00
|
|
|
|
|
|
|
if (!upToDate) {
|
2022-09-20 02:35:58 +08:00
|
|
|
request.issues.push_back_deep(request.issues.arena(), "incorrect_cluster_file_contents"_sr);
|
2021-10-11 11:44:56 +08:00
|
|
|
std::string connectionString = connRecord->getConnectionString().toString();
|
2020-04-06 14:09:36 +08:00
|
|
|
if (!incorrectTime.present()) {
|
|
|
|
incorrectTime = now();
|
|
|
|
}
|
|
|
|
|
2021-10-11 11:44:56 +08:00
|
|
|
// Don't log a SevWarnAlways initially to account for transient issues (e.g. someone else changing
|
|
|
|
// the file right before us)
|
|
|
|
TraceEvent(now() - incorrectTime.get() > 300 ? SevWarnAlways : SevWarn, "IncorrectClusterFileContents")
|
|
|
|
.detail("ClusterFile", connRecord->toString())
|
|
|
|
.detail("StoredConnectionString", storedConnectionString.toString())
|
|
|
|
.detail("CurrentConnectionString", connectionString);
|
|
|
|
}
|
2020-04-06 14:09:36 +08:00
|
|
|
auto peers = FlowTransport::transport().getIncompatiblePeers();
|
|
|
|
for (auto it = peers->begin(); it != peers->end();) {
|
2020-04-11 08:02:11 +08:00
|
|
|
if (now() - it->second.second > FLOW_KNOBS->INCOMPATIBLE_PEER_DELAY_BEFORE_LOGGING) {
|
2020-04-07 11:58:43 +08:00
|
|
|
request.incompatiblePeers.push_back(it->first);
|
2020-04-06 14:09:36 +08:00
|
|
|
it = peers->erase(it);
|
|
|
|
} else {
|
|
|
|
it++;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-05-13 07:21:44 +08:00
|
|
|
state bool ccInterfacePresent = ccInterface->get().present();
|
2021-11-04 03:51:21 +08:00
|
|
|
if (ccInterfacePresent) {
|
2021-11-09 01:43:02 +08:00
|
|
|
request.requestDbInfo = (ccInterface->get().get().id() != dbInfo->get().clusterInterface.id());
|
2021-11-10 01:44:07 +08:00
|
|
|
if (firstReg) {
|
|
|
|
request.requestDbInfo = true;
|
|
|
|
firstReg = false;
|
|
|
|
}
|
2022-07-13 01:19:32 +08:00
|
|
|
TraceEvent("WorkerRegister")
|
|
|
|
.detail("CCID", ccInterface->get().get().id())
|
|
|
|
.detail("Generation", requestGeneration)
|
2022-10-14 08:30:40 +08:00
|
|
|
.detail("RecoveredDiskFiles", recoveredDiskFiles.isSet())
|
|
|
|
.detail("ClusterId", clusterId->get());
|
2021-11-04 03:51:21 +08:00
|
|
|
}
|
2021-04-16 03:40:39 +08:00
|
|
|
state Future<RegisterWorkerReply> registrationReply =
|
2021-05-13 07:21:44 +08:00
|
|
|
ccInterfacePresent ? brokenPromiseToNever(ccInterface->get().get().registerWorker.getReply(request))
|
|
|
|
: Never();
|
2022-07-13 01:19:32 +08:00
|
|
|
state Future<Void> recovered = recoveredDiskFiles.isSet() ? Never() : recoveredDiskFiles.getFuture();
|
2021-04-16 03:40:39 +08:00
|
|
|
state double startTime = now();
|
|
|
|
loop choose {
|
2017-11-15 05:57:37 +08:00
|
|
|
when(RegisterWorkerReply reply = wait(registrationReply)) {
|
2020-03-13 01:18:31 +08:00
|
|
|
processClass = reply.processClass;
|
2018-02-10 08:48:55 +08:00
|
|
|
asyncPriorityInfo->set(reply.priorityInfo);
|
2021-05-13 07:21:44 +08:00
|
|
|
TraceEvent("WorkerRegisterReply")
|
|
|
|
.detail("CCID", ccInterface->get().get().id())
|
|
|
|
.detail("ProcessClass", reply.processClass.toString());
|
2021-04-16 03:40:39 +08:00
|
|
|
break;
|
2017-09-26 01:36:03 +08:00
|
|
|
}
|
2021-05-13 07:21:44 +08:00
|
|
|
when(wait(delay(SERVER_KNOBS->UNKNOWN_CC_TIMEOUT))) {
|
|
|
|
if (!ccInterfacePresent) {
|
|
|
|
TraceEvent(SevWarn, "WorkerRegisterTimeout").detail("WaitTime", now() - startTime);
|
|
|
|
}
|
2017-09-26 01:36:03 +08:00
|
|
|
}
|
2021-04-16 03:40:39 +08:00
|
|
|
when(wait(ccInterface->onChange())) { break; }
|
|
|
|
when(wait(ddInterf->onChange())) { break; }
|
|
|
|
when(wait(rkInterf->onChange())) { break; }
|
2022-09-17 00:03:06 +08:00
|
|
|
when(wait(csInterf->onChange())) { break; }
|
2021-09-15 23:35:58 +08:00
|
|
|
when(wait(bmInterf->onChange())) { break; }
|
2022-09-03 10:21:52 +08:00
|
|
|
when(wait(blobMigratorInterf->onChange())) { break; }
|
2022-01-14 03:11:01 +08:00
|
|
|
when(wait(ekpInterf->onChange())) { break; }
|
2021-04-16 03:40:39 +08:00
|
|
|
when(wait(degraded->onChange())) { break; }
|
|
|
|
when(wait(FlowTransport::transport().onIncompatibleChanged())) { break; }
|
|
|
|
when(wait(issues->onChange())) { break; }
|
2022-07-13 01:19:32 +08:00
|
|
|
when(wait(recovered)) { break; }
|
2017-09-26 01:36:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-06-16 08:36:56 +08:00
|
|
|
// Returns true if `address` is used in the db (indicated by `dbInfo`) transaction system and in the db's primary DC.
|
2021-08-15 07:51:39 +08:00
|
|
|
bool addressInDbAndPrimaryDc(const NetworkAddress& address, Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
|
2021-06-16 08:36:56 +08:00
|
|
|
const auto& dbi = dbInfo->get();
|
|
|
|
|
2021-06-18 01:13:08 +08:00
|
|
|
if (dbi.master.addresses().contains(address)) {
|
2021-06-16 08:36:56 +08:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (dbi.distributor.present() && dbi.distributor.get().address() == address) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (dbi.ratekeeper.present() && dbi.ratekeeper.get().address() == address) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2022-09-17 00:03:06 +08:00
|
|
|
if (dbi.consistencyScan.present() && dbi.consistencyScan.get().address() == address) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2021-09-15 23:35:58 +08:00
|
|
|
if (dbi.blobManager.present() && dbi.blobManager.get().address() == address) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2022-09-03 10:21:52 +08:00
|
|
|
if (dbi.blobMigrator.present() && dbi.blobMigrator.get().address() == address) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2022-01-14 03:11:01 +08:00
|
|
|
if (dbi.encryptKeyProxy.present() && dbi.encryptKeyProxy.get().address() == address) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2021-06-16 08:36:56 +08:00
|
|
|
for (const auto& resolver : dbi.resolvers) {
|
|
|
|
if (resolver.address() == address) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-07-27 09:19:16 +08:00
|
|
|
for (const auto& grvProxy : dbi.client.grvProxies) {
|
|
|
|
if (grvProxy.addresses().contains(address)) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
for (const auto& commitProxy : dbi.client.commitProxies) {
|
|
|
|
if (commitProxy.addresses().contains(address)) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-06-16 08:36:56 +08:00
|
|
|
auto localityIsInPrimaryDc = [&dbInfo](const LocalityData& locality) {
|
|
|
|
return locality.dcId() == dbInfo->get().master.locality.dcId();
|
|
|
|
};
|
|
|
|
|
|
|
|
for (const auto& logSet : dbi.logSystemConfig.tLogs) {
|
|
|
|
for (const auto& tlog : logSet.tLogs) {
|
2021-07-31 12:48:30 +08:00
|
|
|
if (!tlog.present()) {
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
|
2021-06-16 08:36:56 +08:00
|
|
|
if (!localityIsInPrimaryDc(tlog.interf().filteredLocality)) {
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
|
2021-06-18 01:13:08 +08:00
|
|
|
if (tlog.interf().addresses().contains(address)) {
|
2021-06-16 08:36:56 +08:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2021-08-15 07:51:39 +08:00
|
|
|
bool addressesInDbAndPrimaryDc(const NetworkAddressList& addresses, Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
|
2021-06-16 08:36:56 +08:00
|
|
|
return addressInDbAndPrimaryDc(addresses.address, dbInfo) ||
|
|
|
|
(addresses.secondaryAddress.present() && addressInDbAndPrimaryDc(addresses.secondaryAddress.get(), dbInfo));
|
|
|
|
}
|
|
|
|
|
|
|
|
namespace {
|
|
|
|
|
|
|
|
TEST_CASE("/fdbserver/worker/addressInDbAndPrimaryDc") {
|
|
|
|
// Setup a ServerDBInfo for test.
|
|
|
|
ServerDBInfo testDbInfo;
|
|
|
|
LocalityData testLocal;
|
2022-09-20 02:35:58 +08:00
|
|
|
testLocal.set("dcid"_sr, StringRef(std::to_string(1)));
|
2021-06-16 08:36:56 +08:00
|
|
|
testDbInfo.master.locality = testLocal;
|
|
|
|
|
|
|
|
// Manually set up a master address.
|
|
|
|
NetworkAddress testAddress(IPAddress(0x13131313), 1);
|
2022-01-07 04:15:51 +08:00
|
|
|
testDbInfo.master.getCommitVersion =
|
|
|
|
RequestStream<struct GetCommitVersionRequest>(Endpoint({ testAddress }, UID(1, 2)));
|
2021-06-16 08:36:56 +08:00
|
|
|
|
2021-07-31 12:48:30 +08:00
|
|
|
// First, create an empty TLogInterface, and check that it shouldn't be considered as in primary DC.
|
|
|
|
testDbInfo.logSystemConfig.tLogs.push_back(TLogSet());
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().tLogs.push_back(OptionalInterface<TLogInterface>());
|
|
|
|
ASSERT(!addressInDbAndPrimaryDc(g_network->getLocalAddress(), makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
|
|
|
|
|
|
|
|
// Create a remote TLog. Although the remote TLog also uses the local address, it shouldn't be considered as
|
2021-06-16 08:36:56 +08:00
|
|
|
// in primary DC given the remote locality.
|
|
|
|
LocalityData fakeRemote;
|
2022-09-20 02:35:58 +08:00
|
|
|
fakeRemote.set("dcid"_sr, StringRef(std::to_string(2)));
|
2021-06-16 08:36:56 +08:00
|
|
|
TLogInterface remoteTlog(fakeRemote);
|
|
|
|
remoteTlog.initEndpoints();
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().tLogs.push_back(OptionalInterface(remoteTlog));
|
|
|
|
ASSERT(!addressInDbAndPrimaryDc(g_network->getLocalAddress(), makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
|
|
|
|
|
|
|
|
// Next, create a local TLog. Now, the local address should be considered as in local DC.
|
|
|
|
TLogInterface localTlog(testLocal);
|
|
|
|
localTlog.initEndpoints();
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().tLogs.push_back(OptionalInterface(localTlog));
|
|
|
|
ASSERT(addressInDbAndPrimaryDc(g_network->getLocalAddress(), makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
|
|
|
|
|
2021-07-27 09:19:16 +08:00
|
|
|
// Use the master's address to test, which should be considered as in local DC.
|
2021-06-16 08:36:56 +08:00
|
|
|
testDbInfo.logSystemConfig.tLogs.clear();
|
|
|
|
ASSERT(addressInDbAndPrimaryDc(testAddress, makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
|
|
|
|
|
2021-07-27 09:19:16 +08:00
|
|
|
// Last, tests that proxies included in the ClientDbInfo are considered as local.
|
|
|
|
NetworkAddress grvProxyAddress(IPAddress(0x26262626), 1);
|
|
|
|
GrvProxyInterface grvProxyInterf;
|
2021-07-30 16:32:43 +08:00
|
|
|
grvProxyInterf.getConsistentReadVersion =
|
2022-03-15 23:57:26 +08:00
|
|
|
PublicRequestStream<struct GetReadVersionRequest>(Endpoint({ grvProxyAddress }, UID(1, 2)));
|
2021-07-27 09:19:16 +08:00
|
|
|
testDbInfo.client.grvProxies.push_back(grvProxyInterf);
|
|
|
|
ASSERT(addressInDbAndPrimaryDc(grvProxyAddress, makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
|
|
|
|
|
|
|
|
NetworkAddress commitProxyAddress(IPAddress(0x37373737), 1);
|
|
|
|
CommitProxyInterface commitProxyInterf;
|
2021-07-30 16:32:43 +08:00
|
|
|
commitProxyInterf.commit =
|
2022-03-15 23:57:26 +08:00
|
|
|
PublicRequestStream<struct CommitTransactionRequest>(Endpoint({ commitProxyAddress }, UID(1, 2)));
|
2021-07-27 09:19:16 +08:00
|
|
|
testDbInfo.client.commitProxies.push_back(commitProxyInterf);
|
|
|
|
ASSERT(addressInDbAndPrimaryDc(commitProxyAddress, makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
|
|
|
|
|
2021-06-16 08:36:56 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
} // namespace
|
|
|
|
|
2022-04-07 14:57:34 +08:00
|
|
|
// Returns true if `address` is used in the db (indicated by `dbInfo`) transaction system and in the db's primary
|
|
|
|
// satellite DC.
|
|
|
|
bool addressInDbAndPrimarySatelliteDc(const NetworkAddress& address, Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
|
|
|
|
for (const auto& logSet : dbInfo->get().logSystemConfig.tLogs) {
|
|
|
|
if (logSet.isLocal && logSet.locality == tagLocalitySatellite) {
|
|
|
|
for (const auto& tlog : logSet.tLogs) {
|
|
|
|
if (tlog.present() && tlog.interf().addresses().contains(address)) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool addressesInDbAndPrimarySatelliteDc(const NetworkAddressList& addresses,
|
|
|
|
Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
|
|
|
|
return addressInDbAndPrimarySatelliteDc(addresses.address, dbInfo) ||
|
|
|
|
(addresses.secondaryAddress.present() &&
|
|
|
|
addressInDbAndPrimarySatelliteDc(addresses.secondaryAddress.get(), dbInfo));
|
|
|
|
}
|
|
|
|
|
|
|
|
namespace {
|
|
|
|
|
|
|
|
TEST_CASE("/fdbserver/worker/addressInDbAndPrimarySatelliteDc") {
|
|
|
|
// Setup a ServerDBInfo for test.
|
|
|
|
ServerDBInfo testDbInfo;
|
|
|
|
LocalityData testLocal;
|
2022-09-20 02:35:58 +08:00
|
|
|
testLocal.set("dcid"_sr, StringRef(std::to_string(1)));
|
2022-04-07 14:57:34 +08:00
|
|
|
testDbInfo.master.locality = testLocal;
|
|
|
|
|
|
|
|
// First, create an empty TLogInterface, and check that it shouldn't be considered as in satellite DC.
|
|
|
|
testDbInfo.logSystemConfig.tLogs.push_back(TLogSet());
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().isLocal = true;
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().locality = tagLocalitySatellite;
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().tLogs.push_back(OptionalInterface<TLogInterface>());
|
|
|
|
ASSERT(!addressInDbAndPrimarySatelliteDc(g_network->getLocalAddress(),
|
|
|
|
makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
|
|
|
|
|
|
|
|
// Create a satellite tlog, and it should be considered as in primary satellite DC.
|
|
|
|
NetworkAddress satelliteTLogAddress(IPAddress(0x13131313), 1);
|
|
|
|
TLogInterface satelliteTLog(testLocal);
|
|
|
|
satelliteTLog.initEndpoints();
|
|
|
|
satelliteTLog.peekMessages = RequestStream<struct TLogPeekRequest>(Endpoint({ satelliteTLogAddress }, UID(1, 2)));
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().tLogs.push_back(OptionalInterface(satelliteTLog));
|
|
|
|
ASSERT(addressInDbAndPrimarySatelliteDc(satelliteTLogAddress, makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
|
|
|
|
|
|
|
|
// Create a primary TLog, and it shouldn't be considered as in primary Satellite DC.
|
|
|
|
NetworkAddress primaryTLogAddress(IPAddress(0x26262626), 1);
|
|
|
|
testDbInfo.logSystemConfig.tLogs.push_back(TLogSet());
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().isLocal = true;
|
|
|
|
TLogInterface primaryTLog(testLocal);
|
|
|
|
primaryTLog.initEndpoints();
|
|
|
|
primaryTLog.peekMessages = RequestStream<struct TLogPeekRequest>(Endpoint({ primaryTLogAddress }, UID(1, 2)));
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().tLogs.push_back(OptionalInterface(primaryTLog));
|
|
|
|
ASSERT(!addressInDbAndPrimarySatelliteDc(primaryTLogAddress, makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
|
|
|
|
|
|
|
|
// Create a remote TLog, and it should be considered as in remote DC.
|
|
|
|
NetworkAddress remoteTLogAddress(IPAddress(0x37373737), 1);
|
|
|
|
LocalityData fakeRemote;
|
2022-09-20 02:35:58 +08:00
|
|
|
fakeRemote.set("dcid"_sr, StringRef(std::to_string(2)));
|
2022-04-07 14:57:34 +08:00
|
|
|
TLogInterface remoteTLog(fakeRemote);
|
|
|
|
remoteTLog.initEndpoints();
|
|
|
|
remoteTLog.peekMessages = RequestStream<struct TLogPeekRequest>(Endpoint({ remoteTLogAddress }, UID(1, 2)));
|
|
|
|
|
|
|
|
testDbInfo.logSystemConfig.tLogs.push_back(TLogSet());
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().isLocal = false;
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().tLogs.push_back(OptionalInterface(remoteTLog));
|
|
|
|
ASSERT(!addressInDbAndPrimarySatelliteDc(remoteTLogAddress, makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
|
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
} // namespace
|
|
|
|
|
2021-09-10 11:29:28 +08:00
|
|
|
bool addressInDbAndRemoteDc(const NetworkAddress& address, Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
|
|
|
|
const auto& dbi = dbInfo->get();
|
|
|
|
|
|
|
|
for (const auto& logSet : dbi.logSystemConfig.tLogs) {
|
|
|
|
if (logSet.isLocal || logSet.locality == tagLocalitySatellite) {
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
for (const auto& tlog : logSet.tLogs) {
|
|
|
|
if (tlog.present() && tlog.interf().addresses().contains(address)) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
for (const auto& logRouter : logSet.logRouters) {
|
|
|
|
if (logRouter.present() && logRouter.interf().addresses().contains(address)) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool addressesInDbAndRemoteDc(const NetworkAddressList& addresses, Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
|
|
|
|
return addressInDbAndRemoteDc(addresses.address, dbInfo) ||
|
|
|
|
(addresses.secondaryAddress.present() && addressInDbAndRemoteDc(addresses.secondaryAddress.get(), dbInfo));
|
|
|
|
}
|
|
|
|
|
|
|
|
namespace {
|
|
|
|
|
|
|
|
TEST_CASE("/fdbserver/worker/addressInDbAndRemoteDc") {
|
|
|
|
// Setup a ServerDBInfo for test.
|
|
|
|
ServerDBInfo testDbInfo;
|
|
|
|
LocalityData testLocal;
|
2022-09-20 02:35:58 +08:00
|
|
|
testLocal.set("dcid"_sr, StringRef(std::to_string(1)));
|
2021-09-10 11:29:28 +08:00
|
|
|
testDbInfo.master.locality = testLocal;
|
|
|
|
|
|
|
|
// First, create an empty TLogInterface, and check that it shouldn't be considered as in remote DC.
|
|
|
|
testDbInfo.logSystemConfig.tLogs.push_back(TLogSet());
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().isLocal = true;
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().tLogs.push_back(OptionalInterface<TLogInterface>());
|
|
|
|
ASSERT(!addressInDbAndRemoteDc(g_network->getLocalAddress(), makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
|
|
|
|
|
|
|
|
TLogInterface localTlog(testLocal);
|
|
|
|
localTlog.initEndpoints();
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().tLogs.push_back(OptionalInterface(localTlog));
|
|
|
|
ASSERT(!addressInDbAndRemoteDc(g_network->getLocalAddress(), makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
|
|
|
|
|
|
|
|
// Create a remote TLog, and it should be considered as in remote DC.
|
|
|
|
LocalityData fakeRemote;
|
2022-09-20 02:35:58 +08:00
|
|
|
fakeRemote.set("dcid"_sr, StringRef(std::to_string(2)));
|
2021-09-10 11:29:28 +08:00
|
|
|
TLogInterface remoteTlog(fakeRemote);
|
|
|
|
remoteTlog.initEndpoints();
|
|
|
|
|
|
|
|
testDbInfo.logSystemConfig.tLogs.push_back(TLogSet());
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().isLocal = false;
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().tLogs.push_back(OptionalInterface(remoteTlog));
|
|
|
|
ASSERT(addressInDbAndRemoteDc(g_network->getLocalAddress(), makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
|
|
|
|
|
|
|
|
// Create a remote log router, and it should be considered as in remote DC.
|
|
|
|
NetworkAddress logRouterAddress(IPAddress(0x26262626), 1);
|
|
|
|
TLogInterface remoteLogRouter(fakeRemote);
|
|
|
|
remoteLogRouter.initEndpoints();
|
|
|
|
remoteLogRouter.peekMessages = RequestStream<struct TLogPeekRequest>(Endpoint({ logRouterAddress }, UID(1, 2)));
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().logRouters.push_back(OptionalInterface(remoteLogRouter));
|
|
|
|
ASSERT(addressInDbAndRemoteDc(logRouterAddress, makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
|
|
|
|
|
|
|
|
// Create a satellite tlog, and it shouldn't be considered as in remote DC.
|
|
|
|
testDbInfo.logSystemConfig.tLogs.push_back(TLogSet());
|
2021-10-08 06:29:35 +08:00
|
|
|
testDbInfo.logSystemConfig.tLogs.back().locality = tagLocalitySatellite;
|
2021-09-10 11:29:28 +08:00
|
|
|
NetworkAddress satelliteTLogAddress(IPAddress(0x13131313), 1);
|
|
|
|
TLogInterface satelliteTLog(fakeRemote);
|
|
|
|
satelliteTLog.initEndpoints();
|
|
|
|
satelliteTLog.peekMessages = RequestStream<struct TLogPeekRequest>(Endpoint({ satelliteTLogAddress }, UID(1, 2)));
|
|
|
|
testDbInfo.logSystemConfig.tLogs.back().tLogs.push_back(OptionalInterface(satelliteTLog));
|
|
|
|
ASSERT(!addressInDbAndRemoteDc(satelliteTLogAddress, makeReference<AsyncVar<ServerDBInfo>>(testDbInfo)));
|
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
} // namespace
|
|
|
|
|
2021-06-16 08:36:56 +08:00
|
|
|
// The actor that actively monitors the health of local and peer servers, and reports anomaly to the cluster controller.
|
2021-08-15 07:51:39 +08:00
|
|
|
ACTOR Future<Void> healthMonitor(Reference<AsyncVar<Optional<ClusterControllerFullInterface>> const> ccInterface,
|
2021-06-16 08:36:56 +08:00
|
|
|
WorkerInterface interf,
|
|
|
|
LocalityData locality,
|
2021-08-15 07:51:39 +08:00
|
|
|
Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
|
2021-06-16 08:36:56 +08:00
|
|
|
loop {
|
|
|
|
Future<Void> nextHealthCheckDelay = Never();
|
2021-09-10 11:29:28 +08:00
|
|
|
if (dbInfo->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS && ccInterface->get().present()) {
|
2021-06-16 08:36:56 +08:00
|
|
|
nextHealthCheckDelay = delay(SERVER_KNOBS->WORKER_HEALTH_MONITOR_INTERVAL);
|
2021-06-18 01:13:08 +08:00
|
|
|
const auto& allPeers = FlowTransport::transport().getAllPeers();
|
2021-06-24 05:59:00 +08:00
|
|
|
UpdateWorkerHealthRequest req;
|
2021-06-16 08:36:56 +08:00
|
|
|
|
2022-04-08 05:45:54 +08:00
|
|
|
enum WorkerLocation { None, Primary, Remote };
|
2022-04-07 14:57:34 +08:00
|
|
|
WorkerLocation workerLocation = None;
|
2021-09-10 11:29:28 +08:00
|
|
|
if (addressesInDbAndPrimaryDc(interf.addresses(), dbInfo)) {
|
2022-04-07 14:57:34 +08:00
|
|
|
workerLocation = Primary;
|
2021-09-10 11:29:28 +08:00
|
|
|
} else if (addressesInDbAndRemoteDc(interf.addresses(), dbInfo)) {
|
2022-04-07 14:57:34 +08:00
|
|
|
workerLocation = Remote;
|
2021-09-10 11:29:28 +08:00
|
|
|
}
|
2021-06-16 08:36:56 +08:00
|
|
|
|
2022-04-07 14:57:34 +08:00
|
|
|
if (workerLocation != None) {
|
2021-09-10 11:29:28 +08:00
|
|
|
for (const auto& [address, peer] : allPeers) {
|
2022-04-05 13:49:25 +08:00
|
|
|
if (peer->connectFailedCount == 0 &&
|
|
|
|
peer->pingLatencies.getPopulationSize() < SERVER_KNOBS->PEER_LATENCY_CHECK_MIN_POPULATION) {
|
2021-09-10 11:29:28 +08:00
|
|
|
// Ignore peers that don't have enough samples.
|
|
|
|
// TODO(zhewu): Currently, FlowTransport latency monitor clears ping latency samples on a
|
|
|
|
// regular
|
|
|
|
// basis, which may affect the measurement count. Currently,
|
|
|
|
// WORKER_HEALTH_MONITOR_INTERVAL is much smaller than the ping clearance interval,
|
|
|
|
// so it may be ok. If this ends to be a problem, we need to consider keep track of
|
|
|
|
// last ping latencies logged.
|
|
|
|
continue;
|
|
|
|
}
|
2022-04-07 14:57:34 +08:00
|
|
|
bool degradedPeer = false;
|
2022-09-29 05:24:14 +08:00
|
|
|
bool disconnectedPeer = false;
|
2022-04-07 14:57:34 +08:00
|
|
|
if ((workerLocation == Primary && addressInDbAndPrimaryDc(address, dbInfo)) ||
|
|
|
|
(workerLocation == Remote && addressInDbAndRemoteDc(address, dbInfo))) {
|
|
|
|
// Monitors intra DC latencies between servers that in the primary or remote DC's transaction
|
|
|
|
// systems. Note that currently we are not monitor storage servers, since lagging in storage
|
|
|
|
// servers today already can trigger server exclusion by data distributor.
|
2021-09-10 11:29:28 +08:00
|
|
|
|
2022-09-29 05:24:14 +08:00
|
|
|
if (peer->connectFailedCount >= SERVER_KNOBS->PEER_DEGRADATION_CONNECTION_FAILURE_COUNT) {
|
|
|
|
disconnectedPeer = true;
|
|
|
|
} else if (peer->pingLatencies.percentile(SERVER_KNOBS->PEER_LATENCY_DEGRADATION_PERCENTILE) >
|
|
|
|
SERVER_KNOBS->PEER_LATENCY_DEGRADATION_THRESHOLD ||
|
|
|
|
peer->timeoutCount / (double)(peer->pingLatencies.getPopulationSize()) >
|
|
|
|
SERVER_KNOBS->PEER_TIMEOUT_PERCENTAGE_DEGRADATION_THRESHOLD) {
|
2022-04-07 14:57:34 +08:00
|
|
|
degradedPeer = true;
|
2021-09-10 11:29:28 +08:00
|
|
|
}
|
2022-09-29 05:24:14 +08:00
|
|
|
if (disconnectedPeer || degradedPeer) {
|
|
|
|
TraceEvent("HealthMonitorDetectDegradedPeer")
|
|
|
|
.detail("Peer", address)
|
|
|
|
.detail("Elapsed", now() - peer->lastLoggedTime)
|
|
|
|
.detail("Disconnected", disconnectedPeer)
|
|
|
|
.detail("MinLatency", peer->pingLatencies.min())
|
|
|
|
.detail("MaxLatency", peer->pingLatencies.max())
|
|
|
|
.detail("MeanLatency", peer->pingLatencies.mean())
|
|
|
|
.detail("MedianLatency", peer->pingLatencies.median())
|
|
|
|
.detail("CheckedPercentile", SERVER_KNOBS->PEER_LATENCY_DEGRADATION_PERCENTILE)
|
|
|
|
.detail(
|
|
|
|
"CheckedPercentileLatency",
|
|
|
|
peer->pingLatencies.percentile(SERVER_KNOBS->PEER_LATENCY_DEGRADATION_PERCENTILE))
|
|
|
|
.detail("PingCount", peer->pingLatencies.getPopulationSize())
|
|
|
|
.detail("PingTimeoutCount", peer->timeoutCount)
|
|
|
|
.detail("ConnectionFailureCount", peer->connectFailedCount);
|
|
|
|
}
|
2022-04-07 14:57:34 +08:00
|
|
|
} else if (workerLocation == Primary && addressInDbAndPrimarySatelliteDc(address, dbInfo)) {
|
|
|
|
// Monitors inter DC latencies between servers in primary and primary satellite DC. Note that
|
|
|
|
// TLog workers in primary satellite DC are on the critical path of serving a commit.
|
2022-09-29 05:24:14 +08:00
|
|
|
if (peer->connectFailedCount >= SERVER_KNOBS->PEER_DEGRADATION_CONNECTION_FAILURE_COUNT) {
|
|
|
|
disconnectedPeer = true;
|
|
|
|
} else if (peer->pingLatencies.percentile(
|
|
|
|
SERVER_KNOBS->PEER_LATENCY_DEGRADATION_PERCENTILE_SATELLITE) >
|
|
|
|
SERVER_KNOBS->PEER_LATENCY_DEGRADATION_THRESHOLD_SATELLITE ||
|
|
|
|
peer->timeoutCount / (double)(peer->pingLatencies.getPopulationSize()) >
|
|
|
|
SERVER_KNOBS->PEER_TIMEOUT_PERCENTAGE_DEGRADATION_THRESHOLD) {
|
2022-04-07 14:57:34 +08:00
|
|
|
degradedPeer = true;
|
|
|
|
}
|
|
|
|
|
2022-09-29 05:24:14 +08:00
|
|
|
if (disconnectedPeer || degradedPeer) {
|
|
|
|
TraceEvent("HealthMonitorDetectDegradedPeer")
|
|
|
|
.detail("Peer", address)
|
2022-10-05 14:09:30 +08:00
|
|
|
.detail("Satellite", true)
|
2022-09-29 05:24:14 +08:00
|
|
|
.detail("Elapsed", now() - peer->lastLoggedTime)
|
|
|
|
.detail("Disconnected", disconnectedPeer)
|
|
|
|
.detail("MinLatency", peer->pingLatencies.min())
|
|
|
|
.detail("MaxLatency", peer->pingLatencies.max())
|
|
|
|
.detail("MeanLatency", peer->pingLatencies.mean())
|
|
|
|
.detail("MedianLatency", peer->pingLatencies.median())
|
|
|
|
.detail("CheckedPercentile",
|
|
|
|
SERVER_KNOBS->PEER_LATENCY_DEGRADATION_PERCENTILE_SATELLITE)
|
|
|
|
.detail("CheckedPercentileLatency",
|
|
|
|
peer->pingLatencies.percentile(
|
|
|
|
SERVER_KNOBS->PEER_LATENCY_DEGRADATION_PERCENTILE_SATELLITE))
|
|
|
|
.detail("PingCount", peer->pingLatencies.getPopulationSize())
|
|
|
|
.detail("PingTimeoutCount", peer->timeoutCount)
|
|
|
|
.detail("ConnectionFailureCount", peer->connectFailedCount);
|
|
|
|
}
|
|
|
|
}
|
2022-04-07 14:57:34 +08:00
|
|
|
|
2022-09-29 05:24:14 +08:00
|
|
|
if (disconnectedPeer) {
|
|
|
|
req.disconnectedPeers.push_back(address);
|
|
|
|
} else if (degradedPeer) {
|
2022-04-07 14:57:34 +08:00
|
|
|
req.degradedPeers.push_back(address);
|
2021-09-10 11:29:28 +08:00
|
|
|
}
|
2021-06-16 08:36:56 +08:00
|
|
|
}
|
2022-04-06 13:22:06 +08:00
|
|
|
|
|
|
|
if (SERVER_KNOBS->WORKER_HEALTH_REPORT_RECENT_DESTROYED_PEER) {
|
|
|
|
// When the worker cannot connect to a remote peer, the peer maybe erased from the list returned
|
|
|
|
// from getAllPeers(). Therefore, we also look through all the recent closed peers in the flow
|
|
|
|
// transport's health monitor. Note that all the closed peers stored here are caused by connection
|
|
|
|
// failure, but not normal connection close. Therefore, we report all such peers if they are also
|
|
|
|
// part of the transaction sub system.
|
|
|
|
for (const auto& address : FlowTransport::transport().healthMonitor()->getRecentClosedPeers()) {
|
|
|
|
if (allPeers.find(address) != allPeers.end()) {
|
|
|
|
// We have checked this peer in the above for loop.
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
|
2022-04-07 14:57:34 +08:00
|
|
|
if ((workerLocation == Primary && addressInDbAndPrimaryDc(address, dbInfo)) ||
|
|
|
|
(workerLocation == Remote && addressInDbAndRemoteDc(address, dbInfo)) ||
|
|
|
|
(workerLocation == Primary && addressInDbAndPrimarySatelliteDc(address, dbInfo))) {
|
2022-04-06 13:22:06 +08:00
|
|
|
TraceEvent("HealthMonitorDetectRecentClosedPeer").suppressFor(30).detail("Peer", address);
|
2022-09-29 05:24:14 +08:00
|
|
|
req.disconnectedPeers.push_back(address);
|
2022-04-06 13:22:06 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-06-16 08:36:56 +08:00
|
|
|
}
|
2021-06-24 05:59:00 +08:00
|
|
|
|
2022-09-29 05:24:14 +08:00
|
|
|
if (!req.disconnectedPeers.empty() || !req.degradedPeers.empty()) {
|
2021-06-24 05:59:00 +08:00
|
|
|
req.address = FlowTransport::transport().getLocalAddress();
|
|
|
|
ccInterface->get().get().updateWorkerHealth.send(req);
|
|
|
|
}
|
2021-06-16 08:36:56 +08:00
|
|
|
}
|
|
|
|
choose {
|
|
|
|
when(wait(nextHealthCheckDelay)) {}
|
|
|
|
when(wait(ccInterface->onChange())) {}
|
|
|
|
when(wait(dbInfo->onChange())) {}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-02-02 02:00:06 +08:00
|
|
|
#if (defined(__linux__) || defined(__FreeBSD__)) && defined(USE_GPERFTOOLS)
|
2017-05-26 04:48:44 +08:00
|
|
|
// A set of threads that should be profiled
|
|
|
|
std::set<std::thread::id> profiledThreads;
|
|
|
|
|
|
|
|
// Returns whether or not a given thread should be profiled
|
|
|
|
int filter_in_thread(void* arg) {
|
|
|
|
return profiledThreads.count(std::this_thread::get_id()) > 0 ? 1 : 0;
|
|
|
|
}
|
|
|
|
#endif
|
|
|
|
|
|
|
|
// Enables the calling thread to be profiled
|
|
|
|
void registerThreadForProfiling() {
|
2020-02-02 02:00:06 +08:00
|
|
|
#if (defined(__linux__) || defined(__FreeBSD__)) && defined(USE_GPERFTOOLS)
|
2017-05-26 04:48:44 +08:00
|
|
|
// Not sure if this is actually needed, but a call to backtrace was advised here:
|
|
|
|
// http://groups.google.com/group/google-perftools/browse_thread/thread/0dfd74532e038eb8/2686d9f24ac4365f?pli=1
|
|
|
|
profiledThreads.insert(std::this_thread::get_id());
|
|
|
|
const int num_levels = 100;
|
|
|
|
void* pc[num_levels];
|
|
|
|
backtrace(pc, num_levels);
|
|
|
|
#endif
|
|
|
|
}
|
|
|
|
|
|
|
|
// Starts or stops the CPU profiler
|
|
|
|
void updateCpuProfiler(ProfilerRequest req) {
|
2017-10-12 05:13:16 +08:00
|
|
|
switch (req.type) {
|
|
|
|
case ProfilerRequest::Type::GPROF:
|
2020-02-02 02:00:06 +08:00
|
|
|
#if (defined(__linux__) || defined(__FreeBSD__)) && defined(USE_GPERFTOOLS) && !defined(VALGRIND)
|
2017-10-12 05:13:16 +08:00
|
|
|
switch (req.action) {
|
|
|
|
case ProfilerRequest::Action::ENABLE: {
|
2017-10-17 07:04:09 +08:00
|
|
|
const char* path = (const char*)req.outputFile.begin();
|
2017-10-12 05:13:16 +08:00
|
|
|
ProfilerOptions* options = new ProfilerOptions();
|
|
|
|
options->filter_in_thread = &filter_in_thread;
|
2020-08-19 05:18:50 +08:00
|
|
|
options->filter_in_thread_arg = nullptr;
|
2017-10-12 05:13:16 +08:00
|
|
|
ProfilerStartWithOptions(path, options);
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ProfilerRequest::Action::DISABLE:
|
|
|
|
ProfilerStop();
|
|
|
|
break;
|
|
|
|
case ProfilerRequest::Action::RUN:
|
|
|
|
ASSERT(false); // User should have called runProfiler.
|
|
|
|
break;
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
#endif
|
2017-10-12 05:13:16 +08:00
|
|
|
break;
|
|
|
|
case ProfilerRequest::Type::FLOW:
|
|
|
|
switch (req.action) {
|
|
|
|
case ProfilerRequest::Action::ENABLE:
|
|
|
|
startProfiling(g_network, {}, req.outputFile);
|
|
|
|
break;
|
|
|
|
case ProfilerRequest::Action::DISABLE:
|
|
|
|
stopProfiling();
|
|
|
|
break;
|
|
|
|
case ProfilerRequest::Action::RUN:
|
|
|
|
ASSERT(false); // User should have called runProfiler.
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
break;
|
2019-04-05 04:32:10 +08:00
|
|
|
default:
|
|
|
|
ASSERT(false);
|
2019-04-04 06:57:16 +08:00
|
|
|
break;
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2017-10-12 05:13:16 +08:00
|
|
|
}
|
|
|
|
|
2019-04-04 11:54:30 +08:00
|
|
|
ACTOR Future<Void> runCpuProfiler(ProfilerRequest req) {
|
2017-11-08 05:54:17 +08:00
|
|
|
if (req.action == ProfilerRequest::Action::RUN) {
|
|
|
|
req.action = ProfilerRequest::Action::ENABLE;
|
|
|
|
updateCpuProfiler(req);
|
2019-04-04 11:54:30 +08:00
|
|
|
wait(delay(req.duration));
|
|
|
|
req.action = ProfilerRequest::Action::DISABLE;
|
|
|
|
updateCpuProfiler(req);
|
2017-11-08 05:54:17 +08:00
|
|
|
return Void();
|
|
|
|
} else {
|
|
|
|
updateCpuProfiler(req);
|
|
|
|
return Void();
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-04-06 05:45:58 +08:00
|
|
|
void runHeapProfiler(const char* msg) {
|
2019-04-04 11:54:30 +08:00
|
|
|
#if defined(__linux__) && defined(USE_GPERFTOOLS) && !defined(VALGRIND)
|
2019-04-05 06:20:30 +08:00
|
|
|
if (IsHeapProfilerRunning()) {
|
2019-04-06 05:45:58 +08:00
|
|
|
HeapProfilerDump(msg);
|
2019-04-05 06:20:30 +08:00
|
|
|
} else {
|
2019-04-05 06:29:50 +08:00
|
|
|
TraceEvent("ProfilerError").detail("Message", "HeapProfiler not running");
|
2019-04-05 06:20:30 +08:00
|
|
|
}
|
|
|
|
#else
|
|
|
|
TraceEvent("ProfilerError").detail("Message", "HeapProfiler Unsupported");
|
2019-04-04 11:54:30 +08:00
|
|
|
#endif
|
|
|
|
}
|
|
|
|
|
|
|
|
ACTOR Future<Void> runProfiler(ProfilerRequest req) {
|
|
|
|
if (req.type == ProfilerRequest::Type::GPROF_HEAP) {
|
2019-04-06 05:45:58 +08:00
|
|
|
runHeapProfiler("User triggered heap dump");
|
2019-04-05 06:20:30 +08:00
|
|
|
} else {
|
|
|
|
wait(runCpuProfiler(req));
|
2019-04-04 11:54:30 +08:00
|
|
|
}
|
2019-04-05 06:20:30 +08:00
|
|
|
|
2019-04-04 11:54:30 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2019-04-06 07:06:30 +08:00
|
|
|
bool checkHighMemory(int64_t threshold, bool* error) {
|
2019-04-06 05:45:58 +08:00
|
|
|
#if defined(__linux__) && defined(USE_GPERFTOOLS) && !defined(VALGRIND)
|
|
|
|
*error = false;
|
|
|
|
uint64_t page_size = sysconf(_SC_PAGESIZE);
|
2019-05-15 14:30:58 +08:00
|
|
|
int fd = open("/proc/self/statm", O_RDONLY | O_CLOEXEC);
|
2019-04-06 05:45:58 +08:00
|
|
|
if (fd < 0) {
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("OpenStatmFileFailure").log();
|
2019-04-06 05:45:58 +08:00
|
|
|
*error = true;
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
const int buf_sz = 256;
|
|
|
|
char stat_buf[buf_sz];
|
|
|
|
ssize_t stat_nread = read(fd, stat_buf, buf_sz);
|
|
|
|
if (stat_nread < 0) {
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("ReadStatmFileFailure").log();
|
2019-04-06 05:45:58 +08:00
|
|
|
*error = true;
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
uint64_t vmsize, rss;
|
|
|
|
sscanf(stat_buf, "%lu %lu", &vmsize, &rss);
|
|
|
|
rss *= page_size;
|
2019-04-06 07:06:30 +08:00
|
|
|
if (rss >= threshold) {
|
2019-04-06 05:45:58 +08:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
#else
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("CheckHighMemoryUnsupported").log();
|
2019-04-06 05:45:58 +08:00
|
|
|
*error = true;
|
|
|
|
#endif
|
2019-04-06 07:06:30 +08:00
|
|
|
return false;
|
2019-04-06 05:45:58 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// Runs heap profiler when RSS memory usage is high.
|
2019-04-06 07:06:30 +08:00
|
|
|
ACTOR Future<Void> monitorHighMemory(int64_t threshold) {
|
|
|
|
if (threshold <= 0)
|
|
|
|
return Void();
|
|
|
|
|
2019-04-06 05:45:58 +08:00
|
|
|
loop {
|
|
|
|
bool err = false;
|
2019-04-06 07:06:30 +08:00
|
|
|
bool highmem = checkHighMemory(threshold, &err);
|
2019-04-06 05:45:58 +08:00
|
|
|
if (err)
|
|
|
|
break;
|
|
|
|
|
|
|
|
if (highmem)
|
|
|
|
runHeapProfiler("Highmem heap dump");
|
|
|
|
wait(delay(SERVER_KNOBS->HEAP_PROFILER_INTERVAL));
|
|
|
|
}
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-06-16 06:00:43 +08:00
|
|
|
struct TrackRunningStorage {
|
|
|
|
UID self;
|
|
|
|
KeyValueStoreType storeType;
|
|
|
|
std::set<std::pair<UID, KeyValueStoreType>>* runningStorages;
|
|
|
|
TrackRunningStorage(UID self,
|
|
|
|
KeyValueStoreType storeType,
|
|
|
|
std::set<std::pair<UID, KeyValueStoreType>>* runningStorages)
|
|
|
|
: self(self), storeType(storeType), runningStorages(runningStorages) {
|
|
|
|
runningStorages->emplace(self, storeType);
|
|
|
|
}
|
2022-08-13 15:10:20 +08:00
|
|
|
~TrackRunningStorage() { runningStorages->erase(std::make_pair(self, storeType)); };
|
2021-06-16 06:00:43 +08:00
|
|
|
};
|
|
|
|
|
|
|
|
ACTOR Future<Void> storageServerRollbackRebooter(std::set<std::pair<UID, KeyValueStoreType>>* runningStorages,
|
|
|
|
Future<Void> prevStorageServer,
|
2018-10-09 08:26:10 +08:00
|
|
|
KeyValueStoreType storeType,
|
|
|
|
std::string filename,
|
|
|
|
UID id,
|
|
|
|
LocalityData locality,
|
2021-03-06 03:28:15 +08:00
|
|
|
bool isTss,
|
2021-08-15 08:41:57 +08:00
|
|
|
Reference<AsyncVar<ServerDBInfo> const> db,
|
2018-10-09 08:26:10 +08:00
|
|
|
std::string folder,
|
|
|
|
ActorCollection* filesClosed,
|
|
|
|
int64_t memoryLimit,
|
2022-08-24 15:40:38 +08:00
|
|
|
IKeyValueStore* store,
|
|
|
|
bool validateDataFiles,
|
Support Redwood encryption (#7376)
A new knob `ENABLE_STORAGE_SERVER_ENCRYPTION` is added, which despite its name, currently only Redwood supports it. The knob is mean to be only used in tests to test encryption in individual components, and otherwise enabling encryption should be done through the general `ENABLE_ENCRYPTION` knob.
Under the hood, a new `Encryption` encoding type is added to `IPager`, which use AES-256 to encrypt a page. With this encoding, `BlobCipherEncryptHeader` is inserted into page header for encryption metadata. Moreover, since we compute and store an SHA-256 auth token with the encryption header, we rely on it to checksum the data (and the encryption header), and skip the standard xxhash checksum.
`EncryptionKeyProvider` implements the `IEncryptionKeyProvider` interface to provide encryption keys, which utilizes the existing `getLatestEncryptCipherKey` and `getEncryptCipherKey` actors to fetch encryption keys from either local cache or EKP server. If multi-tenancy is used, for writing a new page, `EncryptionKeyProvider` checks if a page contain only data for a single tenant, if so, fetches tenant specific encryption key; otherwise system encryption key is used. The tenant check is done by extracting tenant id from page bound key prefixes. `EncryptionKeyProvider` also holds a reference of the `tenantPrefixIndex` map maintained by storage server, which is used to check if a tenant do exists, and getting the tenant name in order to get the encryption key.
2022-09-01 03:19:55 +08:00
|
|
|
Promise<Void>* rebootKVStore,
|
2022-09-17 06:56:05 +08:00
|
|
|
Reference<IPageEncryptionKeyProvider> encryptionKeyProvider) {
|
2021-06-16 06:00:43 +08:00
|
|
|
state TrackRunningStorage _(id, storeType, runningStorages);
|
2017-05-26 04:48:44 +08:00
|
|
|
loop {
|
|
|
|
ErrorOr<Void> e = wait(errorOr(prevStorageServer));
|
|
|
|
if (!e.isError())
|
|
|
|
return Void();
|
2022-08-24 15:40:38 +08:00
|
|
|
else if (e.getError().code() != error_code_please_reboot &&
|
|
|
|
e.getError().code() != error_code_please_reboot_kv_store)
|
2017-05-26 04:48:44 +08:00
|
|
|
throw e.getError();
|
|
|
|
|
2022-08-24 15:40:38 +08:00
|
|
|
TraceEvent("StorageServerRequestedReboot", id)
|
|
|
|
.detail("RebootStorageEngine", e.getError().code() == error_code_please_reboot_kv_store)
|
|
|
|
.log();
|
|
|
|
|
|
|
|
if (e.getError().code() == error_code_please_reboot_kv_store) {
|
|
|
|
// Add the to actorcollection to make sure filesClosed not return
|
|
|
|
filesClosed->add(rebootKVStore->getFuture());
|
|
|
|
wait(delay(SERVER_KNOBS->REBOOT_KV_STORE_DELAY));
|
|
|
|
// reopen KV store
|
|
|
|
store = openKVStore(
|
|
|
|
storeType,
|
|
|
|
filename,
|
|
|
|
id,
|
|
|
|
memoryLimit,
|
|
|
|
false,
|
|
|
|
validateDataFiles,
|
|
|
|
SERVER_KNOBS->REMOTE_KV_STORE && /* testing mixed mode in simulation if remote kvs enabled */
|
|
|
|
(g_network->isSimulated()
|
|
|
|
? (/* Disable for RocksDB */ storeType != KeyValueStoreType::SSD_ROCKSDB_V1 &&
|
|
|
|
deterministicRandom()->coinflip())
|
|
|
|
: true));
|
|
|
|
Promise<Void> nextRebootKVStorePromise;
|
|
|
|
filesClosed->add(store->onClosed() ||
|
|
|
|
nextRebootKVStorePromise
|
|
|
|
.getFuture() /* clear the onClosed() Future in actorCollection when rebooting */);
|
|
|
|
// remove the original onClosed signal from the actorCollection
|
|
|
|
rebootKVStore->send(Void());
|
|
|
|
rebootKVStore->swap(nextRebootKVStorePromise);
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2018-11-28 01:50:39 +08:00
|
|
|
StorageServerInterface recruited;
|
|
|
|
recruited.uniqueID = id;
|
|
|
|
recruited.locality = locality;
|
2021-05-13 02:53:20 +08:00
|
|
|
recruited.tssPairID =
|
|
|
|
isTss ? Optional<UID>(UID()) : Optional<UID>(); // set this here since we use its presence to determine
|
|
|
|
// whether this server is a tss or not
|
2018-11-28 01:50:39 +08:00
|
|
|
recruited.initEndpoints();
|
|
|
|
|
|
|
|
DUMPTOKEN(recruited.getValue);
|
|
|
|
DUMPTOKEN(recruited.getKey);
|
|
|
|
DUMPTOKEN(recruited.getKeyValues);
|
2022-03-11 02:05:44 +08:00
|
|
|
DUMPTOKEN(recruited.getMappedKeyValues);
|
2018-11-28 01:50:39 +08:00
|
|
|
DUMPTOKEN(recruited.getShardState);
|
|
|
|
DUMPTOKEN(recruited.waitMetrics);
|
|
|
|
DUMPTOKEN(recruited.splitMetrics);
|
2020-01-13 07:30:36 +08:00
|
|
|
DUMPTOKEN(recruited.getReadHotRanges);
|
2020-07-01 05:28:15 +08:00
|
|
|
DUMPTOKEN(recruited.getRangeSplitPoints);
|
2019-07-26 07:27:32 +08:00
|
|
|
DUMPTOKEN(recruited.getStorageMetrics);
|
2018-11-28 01:50:39 +08:00
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
DUMPTOKEN(recruited.getQueuingMetrics);
|
|
|
|
DUMPTOKEN(recruited.getKeyValueStoreType);
|
|
|
|
DUMPTOKEN(recruited.watchValue);
|
2021-06-20 00:47:13 +08:00
|
|
|
DUMPTOKEN(recruited.getKeyValuesStream);
|
2022-03-18 07:45:42 +08:00
|
|
|
DUMPTOKEN(recruited.changeFeedStream);
|
|
|
|
DUMPTOKEN(recruited.changeFeedPop);
|
|
|
|
DUMPTOKEN(recruited.changeFeedVersionUpdate);
|
2018-11-28 01:50:39 +08:00
|
|
|
|
Support Redwood encryption (#7376)
A new knob `ENABLE_STORAGE_SERVER_ENCRYPTION` is added, which despite its name, currently only Redwood supports it. The knob is mean to be only used in tests to test encryption in individual components, and otherwise enabling encryption should be done through the general `ENABLE_ENCRYPTION` knob.
Under the hood, a new `Encryption` encoding type is added to `IPager`, which use AES-256 to encrypt a page. With this encoding, `BlobCipherEncryptHeader` is inserted into page header for encryption metadata. Moreover, since we compute and store an SHA-256 auth token with the encryption header, we rely on it to checksum the data (and the encryption header), and skip the standard xxhash checksum.
`EncryptionKeyProvider` implements the `IEncryptionKeyProvider` interface to provide encryption keys, which utilizes the existing `getLatestEncryptCipherKey` and `getEncryptCipherKey` actors to fetch encryption keys from either local cache or EKP server. If multi-tenancy is used, for writing a new page, `EncryptionKeyProvider` checks if a page contain only data for a single tenant, if so, fetches tenant specific encryption key; otherwise system encryption key is used. The tenant check is done by extracting tenant id from page bound key prefixes. `EncryptionKeyProvider` also holds a reference of the `tenantPrefixIndex` map maintained by storage server, which is used to check if a tenant do exists, and getting the tenant name in order to get the encryption key.
2022-09-01 03:19:55 +08:00
|
|
|
prevStorageServer = storageServer(store,
|
|
|
|
recruited,
|
|
|
|
db,
|
|
|
|
folder,
|
|
|
|
Promise<Void>(),
|
|
|
|
Reference<IClusterConnectionRecord>(nullptr),
|
|
|
|
encryptionKeyProvider);
|
2018-10-20 09:55:35 +08:00
|
|
|
prevStorageServer = handleIOErrors(prevStorageServer, store, id, store->onClosed());
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-03-03 09:11:23 +08:00
|
|
|
ACTOR Future<Void> storageCacheRollbackRebooter(Future<Void> prevStorageCache,
|
|
|
|
UID id,
|
|
|
|
LocalityData locality,
|
2021-08-15 08:41:57 +08:00
|
|
|
Reference<AsyncVar<ServerDBInfo> const> db) {
|
2020-03-03 09:11:23 +08:00
|
|
|
loop {
|
|
|
|
ErrorOr<Void> e = wait(errorOr(prevStorageCache));
|
|
|
|
if (!e.isError()) {
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("StorageCacheRequestedReboot1", id).log();
|
2020-03-03 09:11:23 +08:00
|
|
|
return Void();
|
|
|
|
} else if (e.getError().code() != error_code_please_reboot &&
|
|
|
|
e.getError().code() != error_code_worker_removed) {
|
|
|
|
TraceEvent("StorageCacheRequestedReboot2", id).detail("Code", e.getError().code());
|
|
|
|
throw e.getError();
|
|
|
|
}
|
|
|
|
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("StorageCacheRequestedReboot", id).log();
|
2020-03-03 09:11:23 +08:00
|
|
|
|
|
|
|
StorageServerInterface recruited;
|
|
|
|
recruited.uniqueID = deterministicRandom()->randomUniqueID(); // id;
|
|
|
|
recruited.locality = locality;
|
|
|
|
recruited.initEndpoints();
|
|
|
|
|
|
|
|
DUMPTOKEN(recruited.getValue);
|
|
|
|
DUMPTOKEN(recruited.getKey);
|
|
|
|
DUMPTOKEN(recruited.getKeyValues);
|
|
|
|
DUMPTOKEN(recruited.getShardState);
|
|
|
|
DUMPTOKEN(recruited.waitMetrics);
|
|
|
|
DUMPTOKEN(recruited.splitMetrics);
|
|
|
|
DUMPTOKEN(recruited.getStorageMetrics);
|
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
DUMPTOKEN(recruited.getQueuingMetrics);
|
|
|
|
DUMPTOKEN(recruited.getKeyValueStoreType);
|
|
|
|
DUMPTOKEN(recruited.watchValue);
|
|
|
|
|
|
|
|
prevStorageCache = storageCacheServer(recruited, 0, db);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
// FIXME: This will not work correctly in simulation as all workers would share the same roles map
|
|
|
|
std::set<std::pair<std::string, std::string>> g_roles;
|
|
|
|
|
|
|
|
Standalone<StringRef> roleString(std::set<std::pair<std::string, std::string>> roles, bool with_ids) {
|
|
|
|
std::string result;
|
|
|
|
for (auto& r : roles) {
|
|
|
|
if (!result.empty())
|
|
|
|
result.append(",");
|
|
|
|
result.append(r.first);
|
|
|
|
if (with_ids) {
|
|
|
|
result.append(":");
|
|
|
|
result.append(r.second);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return StringRef(result);
|
|
|
|
}
|
|
|
|
|
2020-02-15 04:33:43 +08:00
|
|
|
void startRole(const Role& role,
|
|
|
|
UID roleId,
|
|
|
|
UID workerId,
|
|
|
|
const std::map<std::string, std::string>& details,
|
|
|
|
const std::string& origination) {
|
2018-09-06 06:53:12 +08:00
|
|
|
if (role.includeInTraceRoles) {
|
2018-09-06 06:06:14 +08:00
|
|
|
addTraceRole(role.abbreviation);
|
|
|
|
}
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
TraceEvent ev("Role", roleId);
|
2018-09-06 06:06:14 +08:00
|
|
|
ev.detail("As", role.roleName)
|
2017-05-26 04:48:44 +08:00
|
|
|
.detail("Transition", "Begin")
|
|
|
|
.detail("Origination", origination)
|
|
|
|
.detail("OnWorker", workerId);
|
|
|
|
for (auto it = details.begin(); it != details.end(); it++)
|
|
|
|
ev.detail(it->first.c_str(), it->second);
|
|
|
|
|
2020-03-06 10:17:06 +08:00
|
|
|
ev.trackLatest(roleId.shortString() + ".Role");
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
// Update roles map, log Roles metrics
|
2018-09-06 06:06:14 +08:00
|
|
|
g_roles.insert({ role.roleName, roleId.shortString() });
|
2022-09-20 02:35:58 +08:00
|
|
|
StringMetricHandle("Roles"_sr) = roleString(g_roles, false);
|
|
|
|
StringMetricHandle("RolesWithIDs"_sr) = roleString(g_roles, true);
|
2018-09-06 06:06:14 +08:00
|
|
|
if (g_network->isSimulated())
|
2022-09-15 08:10:49 +08:00
|
|
|
g_simulator->addRole(g_network->getLocalAddress(), role.roleName);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
2018-09-06 06:06:14 +08:00
|
|
|
void endRole(const Role& role, UID id, std::string reason, bool ok, Error e) {
|
2017-05-26 04:48:44 +08:00
|
|
|
{
|
|
|
|
TraceEvent ev("Role", id);
|
|
|
|
if (e.code() != invalid_error_code)
|
2022-02-25 04:25:52 +08:00
|
|
|
ev.errorUnsuppressed(e);
|
2017-05-26 04:48:44 +08:00
|
|
|
ev.detail("Transition", "End").detail("As", role.roleName).detail("Reason", reason);
|
|
|
|
|
2020-03-06 10:17:06 +08:00
|
|
|
ev.trackLatest(id.shortString() + ".Role");
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
if (!ok) {
|
2018-09-06 06:06:14 +08:00
|
|
|
std::string type = role.roleName + "Failed";
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
TraceEvent err(SevError, type.c_str(), id);
|
|
|
|
if (e.code() != invalid_error_code) {
|
2022-02-25 04:25:52 +08:00
|
|
|
err.errorUnsuppressed(e);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2018-08-02 05:30:57 +08:00
|
|
|
err.detail("Reason", reason);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
latestEventCache.clear(id.shortString());
|
|
|
|
|
|
|
|
// Update roles map, log Roles metrics
|
2018-09-06 06:06:14 +08:00
|
|
|
g_roles.erase({ role.roleName, id.shortString() });
|
2022-09-20 02:35:58 +08:00
|
|
|
StringMetricHandle("Roles"_sr) = roleString(g_roles, false);
|
|
|
|
StringMetricHandle("RolesWithIDs"_sr) = roleString(g_roles, true);
|
2018-09-06 06:06:14 +08:00
|
|
|
if (g_network->isSimulated())
|
2022-09-15 08:10:49 +08:00
|
|
|
g_simulator->removeRole(g_network->getLocalAddress(), role.roleName);
|
2018-09-06 06:06:14 +08:00
|
|
|
|
2018-09-06 06:53:12 +08:00
|
|
|
if (role.includeInTraceRoles) {
|
2018-09-06 06:06:14 +08:00
|
|
|
removeTraceRole(role.abbreviation);
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
2020-05-15 04:48:19 +08:00
|
|
|
ACTOR Future<Void> traceRole(Role role, UID roleId) {
|
2020-05-09 07:27:57 +08:00
|
|
|
loop {
|
2020-05-09 08:14:42 +08:00
|
|
|
wait(delay(SERVER_KNOBS->WORKER_LOGGING_INTERVAL));
|
2020-05-09 07:27:57 +08:00
|
|
|
TraceEvent("Role", roleId).detail("Transition", "Refresh").detail("As", role.roleName);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-06-30 02:23:07 +08:00
|
|
|
ACTOR Future<Void> workerSnapCreate(
|
|
|
|
WorkerSnapRequest snapReq,
|
|
|
|
std::string snapFolder,
|
|
|
|
std::map<std::string, WorkerSnapRequest>* snapReqMap /* ongoing snapshot requests */,
|
|
|
|
std::map<std::string, ErrorOr<Void>>*
|
|
|
|
snapReqResultMap /* finished snapshot requests, expired in SNAP_MINIMUM_TIME_GAP seconds */) {
|
2019-06-20 02:20:44 +08:00
|
|
|
state ExecCmdValueString snapArg(snapReq.snapPayload);
|
2022-06-30 02:23:07 +08:00
|
|
|
state std::string snapReqKey = snapReq.snapUID.toString() + snapReq.role.toString();
|
2019-06-20 02:20:44 +08:00
|
|
|
try {
|
2022-06-30 02:23:07 +08:00
|
|
|
int err = wait(execHelper(&snapArg, snapReq.snapUID, snapFolder, snapReq.role.toString()));
|
2019-06-20 02:20:44 +08:00
|
|
|
std::string uidStr = snapReq.snapUID.toString();
|
|
|
|
TraceEvent("ExecTraceWorker")
|
|
|
|
.detail("Uid", uidStr)
|
|
|
|
.detail("Status", err)
|
|
|
|
.detail("Role", snapReq.role)
|
|
|
|
.detail("Value", snapFolder)
|
|
|
|
.detail("ExecPayload", snapReq.snapPayload);
|
2019-07-13 07:26:28 +08:00
|
|
|
if (err != 0) {
|
|
|
|
throw operation_failed();
|
|
|
|
}
|
2019-06-20 02:20:44 +08:00
|
|
|
if (snapReq.role.toString() == "storage") {
|
|
|
|
printStorageVersionInfo();
|
|
|
|
}
|
2022-06-30 02:23:07 +08:00
|
|
|
snapReqMap->at(snapReqKey).reply.send(Void());
|
|
|
|
snapReqMap->erase(snapReqKey);
|
|
|
|
(*snapReqResultMap)[snapReqKey] = ErrorOr<Void>(Void());
|
2019-06-20 02:20:44 +08:00
|
|
|
} catch (Error& e) {
|
2022-02-25 04:25:52 +08:00
|
|
|
TraceEvent("ExecHelperError").errorUnsuppressed(e);
|
2019-07-21 16:00:29 +08:00
|
|
|
if (e.code() != error_code_operation_cancelled) {
|
2022-06-30 02:23:07 +08:00
|
|
|
snapReqMap->at(snapReqKey).reply.sendError(e);
|
|
|
|
snapReqMap->erase(snapReqKey);
|
|
|
|
(*snapReqResultMap)[snapReqKey] = ErrorOr<Void>(e);
|
2019-07-21 16:00:29 +08:00
|
|
|
} else {
|
|
|
|
throw e;
|
2019-07-04 08:15:23 +08:00
|
|
|
}
|
2019-06-20 02:20:44 +08:00
|
|
|
}
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2020-04-07 12:09:44 +08:00
|
|
|
// TODO: `issues` is right now only updated by `monitorTraceLogIssues` and thus is being `set` on every update.
|
|
|
|
// It could be changed to `insert` and `trigger` later if we want to use it as a generic way for the caller of this
|
|
|
|
// function to report issues to cluster controller.
|
|
|
|
ACTOR Future<Void> monitorTraceLogIssues(Reference<AsyncVar<std::set<std::string>>> issues) {
|
2020-03-13 05:34:19 +08:00
|
|
|
state bool pingTimeout = false;
|
|
|
|
loop {
|
|
|
|
wait(delay(SERVER_KNOBS->TRACE_LOG_FLUSH_FAILURE_CHECK_INTERVAL_SECONDS));
|
2020-03-17 04:36:55 +08:00
|
|
|
Future<Void> pingAck = pingTraceLogWriterThread();
|
2020-03-13 05:34:19 +08:00
|
|
|
try {
|
2020-03-17 04:36:55 +08:00
|
|
|
wait(timeoutError(pingAck, SERVER_KNOBS->TRACE_LOG_PING_TIMEOUT_SECONDS));
|
2020-03-13 05:34:19 +08:00
|
|
|
} catch (Error& e) {
|
|
|
|
if (e.code() == error_code_timed_out) {
|
|
|
|
pingTimeout = true;
|
|
|
|
} else {
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
2020-04-07 12:09:44 +08:00
|
|
|
std::set<std::string> _issues;
|
2020-11-05 14:39:56 +08:00
|
|
|
retrieveTraceLogIssues(_issues);
|
2020-04-07 12:09:44 +08:00
|
|
|
if (pingTimeout) {
|
|
|
|
// Ping trace log writer thread timeout.
|
|
|
|
_issues.insert("trace_log_writer_thread_unresponsive");
|
|
|
|
pingTimeout = false;
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2020-04-07 12:09:44 +08:00
|
|
|
issues->set(_issues);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-10-03 16:27:36 +08:00
|
|
|
class SharedLogsKey {
|
|
|
|
TLogVersion logVersion;
|
|
|
|
TLogSpillType spillType;
|
|
|
|
KeyValueStoreType storeType;
|
|
|
|
|
|
|
|
public:
|
2019-10-05 04:35:52 +08:00
|
|
|
SharedLogsKey(const TLogOptions& options, KeyValueStoreType kvst)
|
|
|
|
: logVersion(options.version), spillType(options.spillType), storeType(kvst) {
|
2019-10-03 16:27:36 +08:00
|
|
|
if (logVersion >= TLogVersion::V5)
|
|
|
|
spillType = TLogSpillType::UNSET;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool operator<(const SharedLogsKey& other) const {
|
|
|
|
return std::tie(logVersion, spillType, storeType) <
|
|
|
|
std::tie(other.logVersion, other.spillType, other.storeType);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2019-07-30 14:40:28 +08:00
|
|
|
struct SharedLogsValue {
|
|
|
|
Future<Void> actor = Void();
|
|
|
|
UID uid = UID();
|
|
|
|
PromiseStream<InitializeTLogRequest> requests;
|
|
|
|
|
|
|
|
SharedLogsValue() = default;
|
|
|
|
SharedLogsValue(Future<Void> actor, UID uid, PromiseStream<InitializeTLogRequest> requests)
|
|
|
|
: actor(actor), uid(uid), requests(requests) {}
|
|
|
|
};
|
|
|
|
|
2021-07-29 07:03:37 +08:00
|
|
|
ACTOR Future<Void> chaosMetricsLogger() {
|
|
|
|
|
|
|
|
auto res = g_network->global(INetwork::enChaosMetrics);
|
|
|
|
if (!res)
|
|
|
|
return Void();
|
|
|
|
|
|
|
|
state ChaosMetrics* chaosMetrics = static_cast<ChaosMetrics*>(res);
|
|
|
|
chaosMetrics->clear();
|
|
|
|
|
|
|
|
loop {
|
|
|
|
wait(delay(FLOW_KNOBS->CHAOS_LOGGING_INTERVAL));
|
|
|
|
|
|
|
|
TraceEvent e("ChaosMetrics");
|
2021-07-30 16:32:43 +08:00
|
|
|
double elapsed = now() - chaosMetrics->startTime;
|
2021-07-29 07:03:37 +08:00
|
|
|
e.detail("Elapsed", elapsed);
|
|
|
|
chaosMetrics->getFields(&e);
|
|
|
|
e.trackLatest("ChaosMetrics");
|
|
|
|
chaosMetrics->clear();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-03-18 03:40:34 +08:00
|
|
|
// like genericactors setWhenDoneOrError, but we need to take into account the bm epoch. We don't want to reset it if
|
|
|
|
// this manager was replaced by a later manager (with a higher epoch) on this worker
|
|
|
|
ACTOR Future<Void> resetBlobManagerWhenDoneOrError(
|
|
|
|
Future<Void> blobManagerProcess,
|
|
|
|
Reference<AsyncVar<Optional<std::pair<int64_t, BlobManagerInterface>>>> var,
|
|
|
|
int64_t epoch) {
|
|
|
|
try {
|
|
|
|
wait(blobManagerProcess);
|
|
|
|
} catch (Error& e) {
|
|
|
|
if (e.code() == error_code_actor_cancelled)
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
if (var->get().present() && var->get().get().first == epoch) {
|
|
|
|
var->set(Optional<std::pair<int64_t, BlobManagerInterface>>());
|
|
|
|
}
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2022-10-14 08:30:40 +08:00
|
|
|
static const std::string clusterIdFilename = "clusterId";
|
|
|
|
|
|
|
|
ACTOR Future<Void> createClusterIdFile(std::string folder, UID clusterId) {
|
|
|
|
state std::string clusterIdPath = joinPath(folder, clusterIdFilename);
|
|
|
|
if (fileExists(clusterIdPath)) {
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
loop {
|
|
|
|
try {
|
|
|
|
state ErrorOr<Reference<IAsyncFile>> clusterIdFile =
|
|
|
|
wait(errorOr(IAsyncFileSystem::filesystem(g_network)->open(
|
|
|
|
clusterIdPath, IAsyncFile::OPEN_READWRITE | IAsyncFile::OPEN_LOCK, 0600)));
|
|
|
|
|
|
|
|
if (clusterIdFile.isError() && clusterIdFile.getError().code() == error_code_file_not_found &&
|
|
|
|
!fileExists(clusterIdPath)) {
|
|
|
|
Reference<IAsyncFile> _clusterIdFile = wait(IAsyncFileSystem::filesystem()->open(
|
|
|
|
clusterIdPath,
|
|
|
|
IAsyncFile::OPEN_ATOMIC_WRITE_AND_CREATE | IAsyncFile::OPEN_CREATE | IAsyncFile::OPEN_LOCK |
|
|
|
|
IAsyncFile::OPEN_READWRITE,
|
|
|
|
0600));
|
|
|
|
clusterIdFile = _clusterIdFile;
|
|
|
|
BinaryWriter wr(IncludeVersion());
|
|
|
|
wr << clusterId;
|
|
|
|
wait(clusterIdFile.get()->write(wr.getData(), wr.getLength(), 0));
|
|
|
|
wait(clusterIdFile.get()->sync());
|
|
|
|
return Void();
|
|
|
|
} else {
|
|
|
|
throw clusterIdFile.getError();
|
|
|
|
}
|
|
|
|
} catch (Error& e) {
|
|
|
|
if (e.code() == error_code_actor_cancelled) {
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
if (!e.isInjectedFault()) {
|
|
|
|
fprintf(stderr,
|
|
|
|
"ERROR: error creating or opening cluster id file `%s'.\n",
|
|
|
|
joinPath(folder, clusterIdFilename).c_str());
|
|
|
|
}
|
|
|
|
TraceEvent(SevError, "OpenClusterIdError").error(e);
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-10-11 11:44:56 +08:00
|
|
|
ACTOR Future<Void> workerServer(Reference<IClusterConnectionRecord> connRecord,
|
2021-08-15 07:51:39 +08:00
|
|
|
Reference<AsyncVar<Optional<ClusterControllerFullInterface>> const> ccInterface,
|
2019-04-06 07:06:30 +08:00
|
|
|
LocalityData locality,
|
|
|
|
Reference<AsyncVar<ClusterControllerPriorityInfo>> asyncPriorityInfo,
|
|
|
|
ProcessClass initialClass,
|
|
|
|
std::string folder,
|
|
|
|
int64_t memoryLimit,
|
|
|
|
std::string metricsConnFile,
|
|
|
|
std::string metricsPrefix,
|
2019-03-29 19:49:44 +08:00
|
|
|
Promise<Void> recoveredDiskFiles,
|
|
|
|
int64_t memoryProfileThreshold,
|
2020-05-11 05:20:50 +08:00
|
|
|
std::string _coordFolder,
|
|
|
|
std::string whitelistBinPaths,
|
2021-08-06 09:50:11 +08:00
|
|
|
Reference<AsyncVar<ServerDBInfo>> dbInfo,
|
2022-02-02 14:27:12 +08:00
|
|
|
ConfigBroadcastInterface configBroadcastInterface,
|
|
|
|
Reference<ConfigNode> configNode,
|
2022-10-14 08:30:40 +08:00
|
|
|
Reference<LocalConfiguration> localConfig,
|
|
|
|
Reference<AsyncVar<Optional<UID>>> clusterId) {
|
2017-05-26 04:48:44 +08:00
|
|
|
state PromiseStream<ErrorInfo> errors;
|
2019-01-29 03:29:39 +08:00
|
|
|
state Reference<AsyncVar<Optional<DataDistributorInterface>>> ddInterf(
|
|
|
|
new AsyncVar<Optional<DataDistributorInterface>>());
|
2019-02-15 08:24:46 +08:00
|
|
|
state Reference<AsyncVar<Optional<RatekeeperInterface>>> rkInterf(new AsyncVar<Optional<RatekeeperInterface>>());
|
2022-03-18 03:40:34 +08:00
|
|
|
state Reference<AsyncVar<Optional<std::pair<int64_t, BlobManagerInterface>>>> bmEpochAndInterf(
|
|
|
|
new AsyncVar<Optional<std::pair<int64_t, BlobManagerInterface>>>());
|
2022-09-03 10:21:52 +08:00
|
|
|
state Reference<AsyncVar<Optional<BlobMigratorInterface>>> blobMigratorInterf(
|
|
|
|
new AsyncVar<Optional<BlobMigratorInterface>>());
|
2022-08-19 01:42:53 +08:00
|
|
|
state UID lastBMRecruitRequestId;
|
2022-01-14 03:11:01 +08:00
|
|
|
state Reference<AsyncVar<Optional<EncryptKeyProxyInterface>>> ekpInterf(
|
|
|
|
new AsyncVar<Optional<EncryptKeyProxyInterface>>());
|
2022-09-17 00:03:06 +08:00
|
|
|
state Reference<AsyncVar<Optional<ConsistencyScanInterface>>> csInterf(
|
|
|
|
new AsyncVar<Optional<ConsistencyScanInterface>>());
|
2017-05-27 08:43:28 +08:00
|
|
|
state Future<Void> handleErrors = workerHandleErrors(errors.getFuture()); // Needs to be stopped last
|
2017-05-26 04:48:44 +08:00
|
|
|
state ActorCollection errorForwarders(false);
|
|
|
|
state Future<Void> loggingTrigger = Void();
|
|
|
|
state double loggingDelay = SERVER_KNOBS->WORKER_LOGGING_INTERVAL;
|
2022-08-24 15:40:38 +08:00
|
|
|
// These two promises are destroyed after the "filesClosed" below to avoid broken_promise
|
|
|
|
state Promise<Void> rebootKVSPromise;
|
|
|
|
state Promise<Void> rebootKVSPromise2;
|
2017-05-26 04:48:44 +08:00
|
|
|
state ActorCollection filesClosed(true);
|
|
|
|
state Promise<Void> stopping;
|
2018-05-06 09:16:28 +08:00
|
|
|
state WorkerCache<InitializeStorageReply> storageCache;
|
2017-05-26 04:48:44 +08:00
|
|
|
state Future<Void> metricsLogger;
|
2021-07-29 07:03:37 +08:00
|
|
|
state Future<Void> chaosMetricsActor;
|
2019-04-05 05:11:12 +08:00
|
|
|
state Reference<AsyncVar<bool>> degraded = FlowTransport::transport().getDegraded();
|
2019-02-23 04:15:23 +08:00
|
|
|
// tLogFnForOptions() can return a function that doesn't correspond with the FDB version that the
|
|
|
|
// TLogVersion represents. This can be done if the newer TLog doesn't support a requested option.
|
|
|
|
// As (store type, spill type) can map to the same TLogFn across multiple TLogVersions, we need to
|
|
|
|
// decide if we should collapse them into the same SharedTLog instance as well. The answer
|
|
|
|
// here is no, so that when running with log_version==3, all files should say V=3.
|
2022-06-15 05:41:47 +08:00
|
|
|
state std::map<SharedLogsKey, std::vector<SharedLogsValue>> sharedLogs;
|
2019-10-08 09:06:49 +08:00
|
|
|
state Reference<AsyncVar<UID>> activeSharedTLog(new AsyncVar<UID>());
|
2020-03-24 01:22:24 +08:00
|
|
|
state WorkerCache<InitializeBackupReply> backupWorkerCache;
|
2021-12-04 02:29:22 +08:00
|
|
|
state WorkerCache<InitializeBlobWorkerReply> blobWorkerCache;
|
2019-10-08 09:06:49 +08:00
|
|
|
|
2022-06-30 02:23:07 +08:00
|
|
|
state WorkerSnapRequest lastSnapReq;
|
|
|
|
// Here the key is UID+role, as we still send duplicate requests to a process which is both storage and tlog
|
|
|
|
state std::map<std::string, WorkerSnapRequest> snapReqMap;
|
|
|
|
state std::map<std::string, ErrorOr<Void>> snapReqResultMap;
|
|
|
|
state double lastSnapTime = -SERVER_KNOBS->SNAP_MINIMUM_TIME_GAP; // always successful for the first Snap Request
|
2019-04-23 21:55:55 +08:00
|
|
|
state std::string coordFolder = abspath(_coordFolder);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
state WorkerInterface interf(locality);
|
2021-06-16 06:00:43 +08:00
|
|
|
state std::set<std::pair<UID, KeyValueStoreType>> runningStorages;
|
2019-11-12 07:17:11 +08:00
|
|
|
interf.initEndpoints();
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-03-13 05:34:19 +08:00
|
|
|
state Reference<AsyncVar<std::set<std::string>>> issues(new AsyncVar<std::set<std::string>>());
|
|
|
|
|
2021-07-01 08:05:04 +08:00
|
|
|
if (FLOW_KNOBS->ENABLE_CHAOS_FEATURES) {
|
2021-12-04 01:44:52 +08:00
|
|
|
TraceEvent(SevInfo, "ChaosFeaturesEnabled");
|
2021-07-29 07:03:37 +08:00
|
|
|
chaosMetricsActor = chaosMetricsLogger();
|
2021-07-01 08:05:04 +08:00
|
|
|
}
|
|
|
|
|
2019-03-16 14:54:33 +08:00
|
|
|
folder = abspath(folder);
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
if (metricsPrefix.size() > 0) {
|
|
|
|
if (metricsConnFile.size() > 0) {
|
|
|
|
try {
|
2019-07-09 05:01:04 +08:00
|
|
|
state Database db =
|
2022-09-02 15:28:13 +08:00
|
|
|
Database::createDatabase(metricsConnFile, ApiVersion::LATEST_VERSION, IsInternal::True, locality);
|
2018-09-22 06:58:14 +08:00
|
|
|
metricsLogger = runMetrics(db, KeyRef(metricsPrefix));
|
2022-05-05 07:36:04 +08:00
|
|
|
db->globalConfig->trigger(samplingFrequency, samplingProfilerUpdateFrequency);
|
2017-05-26 04:48:44 +08:00
|
|
|
} catch (Error& e) {
|
|
|
|
TraceEvent(SevWarnAlways, "TDMetricsBadClusterFile").error(e).detail("ConnFile", metricsConnFile);
|
|
|
|
}
|
|
|
|
} else {
|
2021-07-17 15:11:40 +08:00
|
|
|
auto lockAware = metricsPrefix.size() && metricsPrefix[0] == '\xff' ? LockAware::True : LockAware::False;
|
2022-05-04 08:14:38 +08:00
|
|
|
auto database = openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, lockAware);
|
|
|
|
metricsLogger = runMetrics(database, KeyRef(metricsPrefix));
|
2022-05-05 07:36:04 +08:00
|
|
|
database->globalConfig->trigger(samplingFrequency, samplingProfilerUpdateFrequency);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-04-08 14:00:58 +08:00
|
|
|
errorForwarders.add(resetAfter(degraded,
|
|
|
|
SERVER_KNOBS->DEGRADED_RESET_INTERVAL,
|
|
|
|
false,
|
|
|
|
SERVER_KNOBS->DEGRADED_WARNING_LIMIT,
|
|
|
|
SERVER_KNOBS->DEGRADED_WARNING_RESET_DELAY,
|
|
|
|
"DegradedReset"));
|
2017-05-26 04:48:44 +08:00
|
|
|
errorForwarders.add(loadedPonger(interf.debugPing.getFuture()));
|
|
|
|
errorForwarders.add(waitFailureServer(interf.waitFailure.getFuture()));
|
2020-04-07 12:09:44 +08:00
|
|
|
errorForwarders.add(monitorTraceLogIssues(issues));
|
2021-10-11 11:44:56 +08:00
|
|
|
errorForwarders.add(testerServerCore(interf.testerInterface, connRecord, dbInfo, locality));
|
2019-04-06 07:06:30 +08:00
|
|
|
errorForwarders.add(monitorHighMemory(memoryProfileThreshold));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
filesClosed.add(stopping.getFuture());
|
|
|
|
|
2020-10-31 02:20:40 +08:00
|
|
|
initializeSystemMonitorMachineState(SystemMonitorMachineState(
|
|
|
|
folder, locality.dcId(), locality.zoneId(), locality.machineId(), g_network->getLocalAddress().ip));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
{
|
2020-12-03 11:52:57 +08:00
|
|
|
auto recruited = interf;
|
2017-05-26 04:48:44 +08:00
|
|
|
DUMPTOKEN(recruited.clientInterface.reboot);
|
2017-10-12 05:13:16 +08:00
|
|
|
DUMPTOKEN(recruited.clientInterface.profiler);
|
2017-05-26 04:48:44 +08:00
|
|
|
DUMPTOKEN(recruited.tLog);
|
|
|
|
DUMPTOKEN(recruited.master);
|
2020-09-11 08:44:15 +08:00
|
|
|
DUMPTOKEN(recruited.commitProxy);
|
2020-07-15 15:37:41 +08:00
|
|
|
DUMPTOKEN(recruited.grvProxy);
|
2017-05-26 04:48:44 +08:00
|
|
|
DUMPTOKEN(recruited.resolver);
|
|
|
|
DUMPTOKEN(recruited.storage);
|
|
|
|
DUMPTOKEN(recruited.debugPing);
|
|
|
|
DUMPTOKEN(recruited.coordinationPing);
|
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
DUMPTOKEN(recruited.setMetricsRate);
|
|
|
|
DUMPTOKEN(recruited.eventLogRequest);
|
|
|
|
DUMPTOKEN(recruited.traceBatchDumpRequest);
|
2020-04-11 04:45:16 +08:00
|
|
|
DUMPTOKEN(recruited.updateServerDBInfo);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
2019-08-23 02:02:14 +08:00
|
|
|
state std::vector<Future<Void>> recoveries;
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
try {
|
|
|
|
std::vector<DiskStore> stores = getDiskStores(folder);
|
|
|
|
bool validateDataFiles = deleteFile(joinPath(folder, validationFilename));
|
|
|
|
for (int f = 0; f < stores.size(); f++) {
|
|
|
|
DiskStore s = stores[f];
|
|
|
|
// FIXME: Error handling
|
|
|
|
if (s.storedComponent == DiskStore::Storage) {
|
2021-06-14 09:23:59 +08:00
|
|
|
LocalLineage _;
|
2021-05-28 02:25:00 +08:00
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::Storage;
|
2022-09-17 06:56:05 +08:00
|
|
|
Reference<IPageEncryptionKeyProvider> encryptionKeyProvider =
|
Support Redwood encryption (#7376)
A new knob `ENABLE_STORAGE_SERVER_ENCRYPTION` is added, which despite its name, currently only Redwood supports it. The knob is mean to be only used in tests to test encryption in individual components, and otherwise enabling encryption should be done through the general `ENABLE_ENCRYPTION` knob.
Under the hood, a new `Encryption` encoding type is added to `IPager`, which use AES-256 to encrypt a page. With this encoding, `BlobCipherEncryptHeader` is inserted into page header for encryption metadata. Moreover, since we compute and store an SHA-256 auth token with the encryption header, we rely on it to checksum the data (and the encryption header), and skip the standard xxhash checksum.
`EncryptionKeyProvider` implements the `IEncryptionKeyProvider` interface to provide encryption keys, which utilizes the existing `getLatestEncryptCipherKey` and `getEncryptCipherKey` actors to fetch encryption keys from either local cache or EKP server. If multi-tenancy is used, for writing a new page, `EncryptionKeyProvider` checks if a page contain only data for a single tenant, if so, fetches tenant specific encryption key; otherwise system encryption key is used. The tenant check is done by extracting tenant id from page bound key prefixes. `EncryptionKeyProvider` also holds a reference of the `tenantPrefixIndex` map maintained by storage server, which is used to check if a tenant do exists, and getting the tenant name in order to get the encryption key.
2022-09-01 03:19:55 +08:00
|
|
|
makeReference<TenantAwareEncryptionKeyProvider>(dbInfo);
|
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-04-01 08:08:59 +08:00
|
|
|
IKeyValueStore* kv = openKVStore(
|
|
|
|
s.storeType,
|
|
|
|
s.filename,
|
|
|
|
s.storeID,
|
|
|
|
memoryLimit,
|
|
|
|
false,
|
|
|
|
validateDataFiles,
|
2022-04-07 06:09:22 +08:00
|
|
|
SERVER_KNOBS->REMOTE_KV_STORE && /* testing mixed mode in simulation if remote kvs enabled */
|
|
|
|
(g_network->isSimulated()
|
|
|
|
? (/* Disable for RocksDB */ s.storeType != KeyValueStoreType::SSD_ROCKSDB_V1 &&
|
2022-04-22 10:53:14 +08:00
|
|
|
s.storeType != KeyValueStoreType::SSD_SHARDED_ROCKSDB &&
|
2022-04-07 06:09:22 +08:00
|
|
|
deterministicRandom()->coinflip())
|
Support Redwood encryption (#7376)
A new knob `ENABLE_STORAGE_SERVER_ENCRYPTION` is added, which despite its name, currently only Redwood supports it. The knob is mean to be only used in tests to test encryption in individual components, and otherwise enabling encryption should be done through the general `ENABLE_ENCRYPTION` knob.
Under the hood, a new `Encryption` encoding type is added to `IPager`, which use AES-256 to encrypt a page. With this encoding, `BlobCipherEncryptHeader` is inserted into page header for encryption metadata. Moreover, since we compute and store an SHA-256 auth token with the encryption header, we rely on it to checksum the data (and the encryption header), and skip the standard xxhash checksum.
`EncryptionKeyProvider` implements the `IEncryptionKeyProvider` interface to provide encryption keys, which utilizes the existing `getLatestEncryptCipherKey` and `getEncryptCipherKey` actors to fetch encryption keys from either local cache or EKP server. If multi-tenancy is used, for writing a new page, `EncryptionKeyProvider` checks if a page contain only data for a single tenant, if so, fetches tenant specific encryption key; otherwise system encryption key is used. The tenant check is done by extracting tenant id from page bound key prefixes. `EncryptionKeyProvider` also holds a reference of the `tenantPrefixIndex` map maintained by storage server, which is used to check if a tenant do exists, and getting the tenant name in order to get the encryption key.
2022-09-01 03:19:55 +08:00
|
|
|
: true),
|
|
|
|
encryptionKeyProvider);
|
2022-08-24 15:40:38 +08:00
|
|
|
Future<Void> kvClosed =
|
|
|
|
kv->onClosed() ||
|
|
|
|
rebootKVSPromise.getFuture() /* clear the onClosed() Future in actorCollection when rebooting */;
|
2017-05-26 04:48:44 +08:00
|
|
|
filesClosed.add(kvClosed);
|
|
|
|
|
2021-03-06 03:28:15 +08:00
|
|
|
// std::string doesn't have startsWith
|
|
|
|
std::string tssPrefix = testingStoragePrefix.toString();
|
|
|
|
// TODO might be more efficient to mark a boolean on DiskStore in getDiskStores, but that kind of breaks
|
|
|
|
// the abstraction since DiskStore also applies to storage cache + tlog
|
|
|
|
bool isTss = s.filename.find(tssPrefix) != std::string::npos;
|
|
|
|
Role ssRole = isTss ? Role::TESTING_STORAGE_SERVER : Role::STORAGE_SERVER;
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
StorageServerInterface recruited;
|
|
|
|
recruited.uniqueID = s.storeID;
|
|
|
|
recruited.locality = locality;
|
2021-05-13 02:53:20 +08:00
|
|
|
recruited.tssPairID =
|
|
|
|
isTss ? Optional<UID>(UID())
|
|
|
|
: Optional<UID>(); // presence of optional is used as source of truth for tss vs not. Value
|
|
|
|
// gets overridden later in restoreDurableState
|
2017-05-26 04:48:44 +08:00
|
|
|
recruited.initEndpoints();
|
|
|
|
|
|
|
|
std::map<std::string, std::string> details;
|
|
|
|
details["StorageEngine"] = s.storeType.toString();
|
2021-03-06 03:28:15 +08:00
|
|
|
details["IsTSS"] = isTss ? "Yes" : "No";
|
|
|
|
|
|
|
|
startRole(ssRole, recruited.id(), interf.id(), details, "Restored");
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
DUMPTOKEN(recruited.getValue);
|
|
|
|
DUMPTOKEN(recruited.getKey);
|
|
|
|
DUMPTOKEN(recruited.getKeyValues);
|
2022-03-18 07:45:42 +08:00
|
|
|
DUMPTOKEN(recruited.getMappedKeyValues);
|
2017-05-26 04:48:44 +08:00
|
|
|
DUMPTOKEN(recruited.getShardState);
|
|
|
|
DUMPTOKEN(recruited.waitMetrics);
|
|
|
|
DUMPTOKEN(recruited.splitMetrics);
|
2020-01-13 07:30:36 +08:00
|
|
|
DUMPTOKEN(recruited.getReadHotRanges);
|
2020-07-01 05:28:15 +08:00
|
|
|
DUMPTOKEN(recruited.getRangeSplitPoints);
|
2019-07-26 07:27:32 +08:00
|
|
|
DUMPTOKEN(recruited.getStorageMetrics);
|
2017-05-26 04:48:44 +08:00
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
DUMPTOKEN(recruited.getQueuingMetrics);
|
|
|
|
DUMPTOKEN(recruited.getKeyValueStoreType);
|
|
|
|
DUMPTOKEN(recruited.watchValue);
|
2021-06-20 00:47:13 +08:00
|
|
|
DUMPTOKEN(recruited.getKeyValuesStream);
|
2022-01-12 05:35:20 +08:00
|
|
|
DUMPTOKEN(recruited.changeFeedStream);
|
|
|
|
DUMPTOKEN(recruited.changeFeedPop);
|
|
|
|
DUMPTOKEN(recruited.changeFeedVersionUpdate);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2017-09-16 01:57:58 +08:00
|
|
|
Promise<Void> recovery;
|
Support Redwood encryption (#7376)
A new knob `ENABLE_STORAGE_SERVER_ENCRYPTION` is added, which despite its name, currently only Redwood supports it. The knob is mean to be only used in tests to test encryption in individual components, and otherwise enabling encryption should be done through the general `ENABLE_ENCRYPTION` knob.
Under the hood, a new `Encryption` encoding type is added to `IPager`, which use AES-256 to encrypt a page. With this encoding, `BlobCipherEncryptHeader` is inserted into page header for encryption metadata. Moreover, since we compute and store an SHA-256 auth token with the encryption header, we rely on it to checksum the data (and the encryption header), and skip the standard xxhash checksum.
`EncryptionKeyProvider` implements the `IEncryptionKeyProvider` interface to provide encryption keys, which utilizes the existing `getLatestEncryptCipherKey` and `getEncryptCipherKey` actors to fetch encryption keys from either local cache or EKP server. If multi-tenancy is used, for writing a new page, `EncryptionKeyProvider` checks if a page contain only data for a single tenant, if so, fetches tenant specific encryption key; otherwise system encryption key is used. The tenant check is done by extracting tenant id from page bound key prefixes. `EncryptionKeyProvider` also holds a reference of the `tenantPrefixIndex` map maintained by storage server, which is used to check if a tenant do exists, and getting the tenant name in order to get the encryption key.
2022-09-01 03:19:55 +08:00
|
|
|
Future<Void> f =
|
|
|
|
storageServer(kv, recruited, dbInfo, folder, recovery, connRecord, encryptionKeyProvider);
|
2017-09-16 01:57:58 +08:00
|
|
|
recoveries.push_back(recovery.getFuture());
|
2018-10-21 11:48:46 +08:00
|
|
|
f = handleIOErrors(f, kv, s.storeID, kvClosed);
|
2021-06-16 06:00:43 +08:00
|
|
|
f = storageServerRollbackRebooter(&runningStorages,
|
|
|
|
f,
|
2018-10-09 08:26:10 +08:00
|
|
|
s.storeType,
|
|
|
|
s.filename,
|
|
|
|
recruited.id(),
|
|
|
|
recruited.locality,
|
2021-03-06 03:28:15 +08:00
|
|
|
isTss,
|
2018-10-09 08:26:10 +08:00
|
|
|
dbInfo,
|
|
|
|
folder,
|
|
|
|
&filesClosed,
|
|
|
|
memoryLimit,
|
2022-08-24 15:40:38 +08:00
|
|
|
kv,
|
|
|
|
validateDataFiles,
|
Support Redwood encryption (#7376)
A new knob `ENABLE_STORAGE_SERVER_ENCRYPTION` is added, which despite its name, currently only Redwood supports it. The knob is mean to be only used in tests to test encryption in individual components, and otherwise enabling encryption should be done through the general `ENABLE_ENCRYPTION` knob.
Under the hood, a new `Encryption` encoding type is added to `IPager`, which use AES-256 to encrypt a page. With this encoding, `BlobCipherEncryptHeader` is inserted into page header for encryption metadata. Moreover, since we compute and store an SHA-256 auth token with the encryption header, we rely on it to checksum the data (and the encryption header), and skip the standard xxhash checksum.
`EncryptionKeyProvider` implements the `IEncryptionKeyProvider` interface to provide encryption keys, which utilizes the existing `getLatestEncryptCipherKey` and `getEncryptCipherKey` actors to fetch encryption keys from either local cache or EKP server. If multi-tenancy is used, for writing a new page, `EncryptionKeyProvider` checks if a page contain only data for a single tenant, if so, fetches tenant specific encryption key; otherwise system encryption key is used. The tenant check is done by extracting tenant id from page bound key prefixes. `EncryptionKeyProvider` also holds a reference of the `tenantPrefixIndex` map maintained by storage server, which is used to check if a tenant do exists, and getting the tenant name in order to get the encryption key.
2022-09-01 03:19:55 +08:00
|
|
|
&rebootKVSPromise,
|
|
|
|
encryptionKeyProvider);
|
2021-03-06 03:28:15 +08:00
|
|
|
errorForwarders.add(forwardError(errors, ssRole, recruited.id(), f));
|
2017-05-26 04:48:44 +08:00
|
|
|
} else if (s.storedComponent == DiskStore::TLogData) {
|
2021-06-14 09:23:59 +08:00
|
|
|
LocalLineage _;
|
2021-05-28 02:25:00 +08:00
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::TLog;
|
2019-02-23 04:15:23 +08:00
|
|
|
std::string logQueueBasename;
|
|
|
|
const std::string filename = basename(s.filename);
|
|
|
|
if (StringRef(filename).startsWith(fileLogDataPrefix)) {
|
|
|
|
logQueueBasename = fileLogQueuePrefix.toString();
|
|
|
|
} else {
|
|
|
|
StringRef optionsString = StringRef(filename).removePrefix(fileVersionedLogDataPrefix).eat("-");
|
|
|
|
logQueueBasename = fileLogQueuePrefix.toString() + optionsString.toString() + "-";
|
|
|
|
}
|
2019-03-16 14:54:33 +08:00
|
|
|
ASSERT_WE_THINK(abspath(parentDirectory(s.filename)) == folder);
|
2017-05-26 04:48:44 +08:00
|
|
|
IKeyValueStore* kv = openKVStore(s.storeType, s.filename, s.storeID, memoryLimit, validateDataFiles);
|
2022-02-04 06:26:10 +08:00
|
|
|
const DiskQueueVersion dqv = s.tLogOptions.getDiskQueueVersion();
|
2019-03-16 12:01:20 +08:00
|
|
|
const int64_t diskQueueWarnSize =
|
|
|
|
s.tLogOptions.spillType == TLogSpillType::VALUE ? 10 * SERVER_KNOBS->TARGET_BYTES_PER_TLOG : -1;
|
|
|
|
IDiskQueue* queue = openDiskQueue(joinPath(folder, logQueueBasename + s.storeID.toString() + "-"),
|
|
|
|
tlogQueueExtension.toString(),
|
|
|
|
s.storeID,
|
|
|
|
dqv,
|
|
|
|
diskQueueWarnSize);
|
2017-05-26 04:48:44 +08:00
|
|
|
filesClosed.add(kv->onClosed());
|
|
|
|
filesClosed.add(queue->onClosed());
|
|
|
|
|
|
|
|
std::map<std::string, std::string> details;
|
|
|
|
details["StorageEngine"] = s.storeType.toString();
|
2018-09-06 06:06:14 +08:00
|
|
|
startRole(Role::SHARED_TRANSACTION_LOG, s.storeID, interf.id(), details, "Restored");
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
Promise<Void> oldLog;
|
2017-09-16 01:57:58 +08:00
|
|
|
Promise<Void> recovery;
|
2019-02-23 04:15:23 +08:00
|
|
|
TLogFn tLogFn = tLogFnForOptions(s.tLogOptions);
|
2019-10-03 16:27:36 +08:00
|
|
|
auto& logData = sharedLogs[SharedLogsKey(s.tLogOptions, s.storeType)];
|
2022-06-15 05:41:47 +08:00
|
|
|
logData.push_back(SharedLogsValue());
|
2019-02-08 09:02:47 +08:00
|
|
|
// FIXME: Shouldn't if logData.first isValid && !isReady, shouldn't we
|
|
|
|
// be sending a fake InitializeTLogRequest rather than calling tLog() ?
|
2022-06-15 05:41:47 +08:00
|
|
|
Future<Void> tl = tLogFn(kv,
|
|
|
|
queue,
|
|
|
|
dbInfo,
|
|
|
|
locality,
|
|
|
|
logData.back().requests,
|
|
|
|
s.storeID,
|
|
|
|
interf.id(),
|
|
|
|
true,
|
|
|
|
oldLog,
|
|
|
|
recovery,
|
|
|
|
folder,
|
|
|
|
degraded,
|
|
|
|
activeSharedTLog);
|
2017-09-16 01:57:58 +08:00
|
|
|
recoveries.push_back(recovery.getFuture());
|
2019-07-30 14:40:28 +08:00
|
|
|
activeSharedTLog->set(s.storeID);
|
2017-10-06 08:09:44 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
tl = handleIOErrors(tl, kv, s.storeID);
|
|
|
|
tl = handleIOErrors(tl, queue, s.storeID);
|
2022-06-15 05:41:47 +08:00
|
|
|
logData.back().actor = oldLog.getFuture() || tl;
|
|
|
|
logData.back().uid = s.storeID;
|
2018-09-06 06:06:14 +08:00
|
|
|
errorForwarders.add(forwardError(errors, Role::SHARED_TRANSACTION_LOG, s.storeID, tl));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-12-07 05:28:44 +08:00
|
|
|
bool hasCache = false;
|
|
|
|
// start cache role if we have the right process class
|
|
|
|
if (initialClass.classType() == ProcessClass::StorageCacheClass) {
|
|
|
|
hasCache = true;
|
|
|
|
StorageServerInterface recruited;
|
|
|
|
recruited.locality = locality;
|
|
|
|
recruited.initEndpoints();
|
|
|
|
|
|
|
|
std::map<std::string, std::string> details;
|
|
|
|
startRole(Role::STORAGE_CACHE, recruited.id(), interf.id(), details);
|
|
|
|
|
|
|
|
// DUMPTOKEN(recruited.getVersion);
|
|
|
|
DUMPTOKEN(recruited.getValue);
|
|
|
|
DUMPTOKEN(recruited.getKey);
|
|
|
|
DUMPTOKEN(recruited.getKeyValues);
|
2022-03-11 02:05:44 +08:00
|
|
|
DUMPTOKEN(recruited.getMappedKeyValues);
|
2019-12-07 05:28:44 +08:00
|
|
|
DUMPTOKEN(recruited.getShardState);
|
|
|
|
DUMPTOKEN(recruited.waitMetrics);
|
|
|
|
DUMPTOKEN(recruited.splitMetrics);
|
|
|
|
DUMPTOKEN(recruited.getStorageMetrics);
|
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
DUMPTOKEN(recruited.getQueuingMetrics);
|
|
|
|
DUMPTOKEN(recruited.getKeyValueStoreType);
|
|
|
|
DUMPTOKEN(recruited.watchValue);
|
|
|
|
|
2020-01-25 03:00:50 +08:00
|
|
|
auto f = storageCacheServer(recruited, 0, dbInfo);
|
2020-03-03 09:11:23 +08:00
|
|
|
f = storageCacheRollbackRebooter(f, recruited.id(), recruited.locality, dbInfo);
|
2019-12-07 05:28:44 +08:00
|
|
|
errorForwarders.add(forwardError(errors, Role::STORAGE_CACHE, recruited.id(), f));
|
|
|
|
}
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
std::map<std::string, std::string> details;
|
|
|
|
details["Locality"] = locality.toString();
|
|
|
|
details["DataFolder"] = folder;
|
|
|
|
details["StoresPresent"] = format("%d", stores.size());
|
2019-12-07 05:28:44 +08:00
|
|
|
details["CachePresent"] = hasCache ? "true" : "false";
|
2018-09-06 06:06:14 +08:00
|
|
|
startRole(Role::WORKER, interf.id(), interf.id(), details);
|
2020-05-09 07:27:57 +08:00
|
|
|
errorForwarders.add(traceRole(Role::WORKER, interf.id()));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2022-07-13 01:19:32 +08:00
|
|
|
// We want to avoid the worker being recruited as storage or TLog before recoverying it is local files,
|
|
|
|
// to make sure:
|
|
|
|
// (1) the worker can start serving requests once it is recruited as storage or TLog server, and
|
|
|
|
// (2) a slow recovering worker server wouldn't been recruited as TLog and make recovery slow.
|
|
|
|
// However, the worker server can still serve stateless roles, and if encryption is on, it is crucial to have
|
|
|
|
// some worker available to serve the EncryptKeyProxy role, before opening encrypted storage files.
|
|
|
|
//
|
|
|
|
// When encryption-at-rest is enabled, the follow code allows a worker to first register with the
|
|
|
|
// cluster controller to be recruited only as a stateless process i.e. it can't be recruited as a SS or TLog
|
|
|
|
// process; once the local disk recovery is complete (if applicable), the process re-registers with cluster
|
|
|
|
// controller as a stateful process role.
|
|
|
|
//
|
|
|
|
// TODO(yiwu): Unify behavior for encryption and non-encryption once the change is stable.
|
|
|
|
Future<Void> recoverDiskFiles = trigger(
|
|
|
|
[=]() {
|
|
|
|
TraceEvent("RecoveriesComplete", interf.id());
|
|
|
|
recoveredDiskFiles.send(Void());
|
|
|
|
return Void();
|
|
|
|
},
|
|
|
|
waitForAll(recoveries));
|
|
|
|
if (!SERVER_KNOBS->ENABLE_ENCRYPTION) {
|
|
|
|
wait(recoverDiskFiles);
|
|
|
|
} else {
|
|
|
|
errorForwarders.add(recoverDiskFiles);
|
|
|
|
}
|
2018-05-02 03:05:43 +08:00
|
|
|
|
2020-04-07 12:09:44 +08:00
|
|
|
errorForwarders.add(registrationClient(ccInterface,
|
|
|
|
interf,
|
|
|
|
asyncPriorityInfo,
|
|
|
|
initialClass,
|
|
|
|
ddInterf,
|
|
|
|
rkInterf,
|
2022-03-18 03:40:34 +08:00
|
|
|
bmEpochAndInterf,
|
2022-09-03 10:21:52 +08:00
|
|
|
blobMigratorInterf,
|
2022-01-14 03:11:01 +08:00
|
|
|
ekpInterf,
|
2022-09-17 00:03:06 +08:00
|
|
|
csInterf,
|
2020-04-07 12:09:44 +08:00
|
|
|
degraded,
|
2021-10-11 11:44:56 +08:00
|
|
|
connRecord,
|
2021-08-06 09:50:11 +08:00
|
|
|
issues,
|
2022-02-02 14:27:12 +08:00
|
|
|
configNode,
|
2021-11-09 01:43:02 +08:00
|
|
|
localConfig,
|
2022-08-17 08:21:49 +08:00
|
|
|
configBroadcastInterface,
|
2022-07-13 01:19:32 +08:00
|
|
|
dbInfo,
|
2022-10-14 08:30:40 +08:00
|
|
|
recoveredDiskFiles,
|
|
|
|
clusterId));
|
2021-08-06 09:50:11 +08:00
|
|
|
|
2022-02-02 14:27:12 +08:00
|
|
|
if (configNode.isValid()) {
|
2022-08-17 08:21:49 +08:00
|
|
|
errorForwarders.add(brokenPromiseToNever(localConfig->consume(configBroadcastInterface)));
|
2021-08-10 02:49:57 +08:00
|
|
|
}
|
2017-09-16 01:57:58 +08:00
|
|
|
|
2021-06-16 08:36:56 +08:00
|
|
|
if (SERVER_KNOBS->ENABLE_WORKER_HEALTH_MONITOR) {
|
|
|
|
errorForwarders.add(healthMonitor(ccInterface, interf, locality, dbInfo));
|
|
|
|
}
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
loop choose {
|
2022-10-14 08:30:40 +08:00
|
|
|
when(state UpdateServerDBInfoRequest req = waitNext(interf.updateServerDBInfo.getFuture())) {
|
2020-09-29 01:58:49 +08:00
|
|
|
ServerDBInfo localInfo = BinaryReader::fromStringRef<ServerDBInfo>(
|
|
|
|
req.serializedDbInfo, AssumeVersion(g_network->protocolVersion()));
|
2020-04-12 10:30:05 +08:00
|
|
|
localInfo.myLocality = locality;
|
|
|
|
|
2020-04-14 04:09:21 +08:00
|
|
|
if (localInfo.infoGeneration < dbInfo->get().infoGeneration &&
|
|
|
|
localInfo.clusterInterface == dbInfo->get().clusterInterface) {
|
2020-04-12 11:05:03 +08:00
|
|
|
std::vector<Endpoint> rep = req.broadcastInfo;
|
|
|
|
rep.push_back(interf.updateServerDBInfo.getEndpoint());
|
|
|
|
req.reply.send(rep);
|
|
|
|
} else {
|
|
|
|
Optional<Endpoint> notUpdated;
|
|
|
|
if (!ccInterface->get().present() || localInfo.clusterInterface != ccInterface->get().get()) {
|
|
|
|
notUpdated = interf.updateServerDBInfo.getEndpoint();
|
2020-04-18 05:44:58 +08:00
|
|
|
} else if (localInfo.infoGeneration > dbInfo->get().infoGeneration ||
|
|
|
|
dbInfo->get().clusterInterface != ccInterface->get().get()) {
|
2020-04-12 11:05:03 +08:00
|
|
|
TraceEvent("GotServerDBInfoChange")
|
|
|
|
.detail("ChangeID", localInfo.id)
|
2022-01-14 03:11:01 +08:00
|
|
|
.detail("InfoGeneration", localInfo.infoGeneration)
|
2020-04-12 11:05:03 +08:00
|
|
|
.detail("MasterID", localInfo.master.id())
|
|
|
|
.detail("RatekeeperID",
|
|
|
|
localInfo.ratekeeper.present() ? localInfo.ratekeeper.get().id() : UID())
|
|
|
|
.detail("DataDistributorID",
|
2021-09-15 23:35:58 +08:00
|
|
|
localInfo.distributor.present() ? localInfo.distributor.get().id() : UID())
|
|
|
|
.detail("BlobManagerID",
|
2022-01-14 03:11:01 +08:00
|
|
|
localInfo.blobManager.present() ? localInfo.blobManager.get().id() : UID())
|
2022-09-03 10:21:52 +08:00
|
|
|
.detail("BlobMigratorID",
|
|
|
|
localInfo.blobMigrator.present() ? localInfo.blobMigrator.get().id() : UID())
|
2022-01-14 03:11:01 +08:00
|
|
|
.detail("EncryptKeyProxyID",
|
2022-10-13 05:18:56 +08:00
|
|
|
localInfo.encryptKeyProxy.present() ? localInfo.encryptKeyProxy.get().id() : UID())
|
|
|
|
.detail("IsEncryptionEnabled", localInfo.client.isEncryptionEnabled);
|
2022-01-14 03:11:01 +08:00
|
|
|
|
2020-04-12 11:05:03 +08:00
|
|
|
dbInfo->set(localInfo);
|
|
|
|
}
|
|
|
|
errorForwarders.add(
|
|
|
|
success(broadcastDBInfoRequest(req, SERVER_KNOBS->DBINFO_SEND_AMOUNT, notUpdated, true)));
|
2022-10-14 08:30:40 +08:00
|
|
|
|
|
|
|
if (!clusterId->get().present() && localInfo.recoveryState >= RecoveryState::ACCEPTING_COMMITS &&
|
|
|
|
localInfo.client.clusterId.isValid()) {
|
|
|
|
// Persist the cluster ID as a file in the data
|
|
|
|
// directory once recovery has made the transaction
|
|
|
|
// state store durable. The txnStateStore also stores
|
|
|
|
// the cluster ID.
|
|
|
|
// TODO: Does the txnStateStore need to store the cluster ID?
|
|
|
|
state UID tmpClusterId = localInfo.client.clusterId;
|
|
|
|
wait(createClusterIdFile(folder, tmpClusterId));
|
|
|
|
clusterId->set(tmpClusterId);
|
|
|
|
}
|
2020-04-06 14:09:36 +08:00
|
|
|
}
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
when(RebootRequest req = waitNext(interf.clientInterface.reboot.getFuture())) {
|
|
|
|
state RebootRequest rebootReq = req;
|
2019-10-24 05:19:17 +08:00
|
|
|
// If suspendDuration is INT_MAX, the trace will not be logged if it was inside the next block
|
|
|
|
// Also a useful trace to have even if suspendDuration is 0
|
|
|
|
TraceEvent("RebootRequestSuspendingProcess").detail("Duration", req.waitForDuration);
|
2019-06-15 02:35:38 +08:00
|
|
|
if (req.waitForDuration) {
|
|
|
|
flushTraceFileVoid();
|
|
|
|
setProfilingEnabled(0);
|
|
|
|
g_network->stop();
|
|
|
|
threadSleep(req.waitForDuration);
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
if (rebootReq.checkData) {
|
|
|
|
Reference<IAsyncFile> checkFile =
|
|
|
|
wait(IAsyncFileSystem::filesystem()->open(joinPath(folder, validationFilename),
|
|
|
|
IAsyncFile::OPEN_CREATE | IAsyncFile::OPEN_READWRITE,
|
|
|
|
0600));
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(checkFile->sync());
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2017-08-29 02:25:37 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
if (g_network->isSimulated()) {
|
|
|
|
TraceEvent("SimulatedReboot").detail("Deletion", rebootReq.deleteData);
|
|
|
|
if (rebootReq.deleteData) {
|
|
|
|
throw please_reboot_delete();
|
|
|
|
}
|
|
|
|
throw please_reboot();
|
|
|
|
} else {
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("ProcessReboot").log();
|
2017-05-26 04:48:44 +08:00
|
|
|
ASSERT(!rebootReq.deleteData);
|
|
|
|
flushAndExit(0);
|
|
|
|
}
|
|
|
|
}
|
2021-07-01 08:05:04 +08:00
|
|
|
when(SetFailureInjection req = waitNext(interf.clientInterface.setFailureInjection.getFuture())) {
|
|
|
|
if (FLOW_KNOBS->ENABLE_CHAOS_FEATURES) {
|
2021-07-29 07:03:37 +08:00
|
|
|
if (req.diskFailure.present()) {
|
2021-07-13 08:51:01 +08:00
|
|
|
auto diskFailureInjector = DiskFailureInjector::injector();
|
2021-07-29 07:03:37 +08:00
|
|
|
diskFailureInjector->setDiskFailure(req.diskFailure.get().stallInterval,
|
|
|
|
req.diskFailure.get().stallPeriod,
|
|
|
|
req.diskFailure.get().throttlePeriod);
|
2021-07-19 08:35:05 +08:00
|
|
|
} else if (req.flipBits.present()) {
|
|
|
|
auto bitFlipper = BitFlipper::flipper();
|
2021-07-29 07:03:37 +08:00
|
|
|
bitFlipper->setBitFlipPercentage(req.flipBits.get().percentBitFlips);
|
2021-07-01 08:05:04 +08:00
|
|
|
}
|
|
|
|
req.reply.send(Void());
|
|
|
|
} else {
|
|
|
|
req.reply.sendError(client_invalid_operation());
|
|
|
|
}
|
|
|
|
}
|
2017-10-12 05:13:16 +08:00
|
|
|
when(ProfilerRequest req = waitNext(interf.clientInterface.profiler.getFuture())) {
|
2017-10-17 07:04:09 +08:00
|
|
|
state ProfilerRequest profilerReq = req;
|
|
|
|
// There really isn't a great "filepath sanitizer" or "filepath escape" function available,
|
|
|
|
// thus we instead enforce a different requirement. One can only write to a file that's
|
|
|
|
// beneath the working directory, and we remove the ability to do any symlink or ../..
|
|
|
|
// tricks by resolving all paths through `abspath` first.
|
|
|
|
try {
|
2017-10-13 08:49:41 +08:00
|
|
|
std::string realLogDir = abspath(SERVER_KNOBS->LOG_DIRECTORY);
|
|
|
|
std::string realOutPath = abspath(realLogDir + "/" + profilerReq.outputFile.toString());
|
|
|
|
if (realLogDir.size() < realOutPath.size() &&
|
|
|
|
strncmp(realLogDir.c_str(), realOutPath.c_str(), realLogDir.size()) == 0) {
|
|
|
|
profilerReq.outputFile = realOutPath;
|
|
|
|
uncancellable(runProfiler(profilerReq));
|
2017-10-17 07:04:09 +08:00
|
|
|
profilerReq.reply.send(Void());
|
|
|
|
} else {
|
|
|
|
profilerReq.reply.sendError(client_invalid_operation());
|
|
|
|
}
|
|
|
|
} catch (Error& e) {
|
|
|
|
profilerReq.reply.sendError(e);
|
|
|
|
}
|
2017-07-13 15:27:56 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
when(RecruitMasterRequest req = waitNext(interf.master.getFuture())) {
|
2021-06-14 09:23:59 +08:00
|
|
|
LocalLineage _;
|
2021-05-28 02:25:00 +08:00
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::Master;
|
2017-05-26 04:48:44 +08:00
|
|
|
MasterInterface recruited;
|
|
|
|
recruited.locality = locality;
|
|
|
|
recruited.initEndpoints();
|
|
|
|
|
2018-09-06 06:06:14 +08:00
|
|
|
startRole(Role::MASTER, recruited.id(), interf.id());
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
DUMPTOKEN(recruited.getCommitVersion);
|
2020-06-11 06:55:23 +08:00
|
|
|
DUMPTOKEN(recruited.getLiveCommittedVersion);
|
|
|
|
DUMPTOKEN(recruited.reportLiveCommittedVersion);
|
2022-01-07 04:15:51 +08:00
|
|
|
DUMPTOKEN(recruited.updateRecoveryData);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
// printf("Recruited as masterServer\n");
|
2020-07-18 05:59:38 +08:00
|
|
|
Future<Void> masterProcess = masterServer(
|
2021-10-11 11:44:56 +08:00
|
|
|
recruited, dbInfo, ccInterface, ServerCoordinators(connRecord), req.lifetime, req.forceRecovery);
|
2018-09-06 06:06:14 +08:00
|
|
|
errorForwarders.add(
|
|
|
|
zombie(recruited, forwardError(errors, Role::MASTER, recruited.id(), masterProcess)));
|
2017-05-26 04:48:44 +08:00
|
|
|
req.reply.send(recruited);
|
|
|
|
}
|
2018-12-14 05:31:37 +08:00
|
|
|
when(InitializeDataDistributorRequest req = waitNext(interf.dataDistributor.getFuture())) {
|
2021-06-14 09:23:59 +08:00
|
|
|
LocalLineage _;
|
2021-05-28 02:25:00 +08:00
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::DataDistributor;
|
2021-09-13 21:58:38 +08:00
|
|
|
DataDistributorInterface recruited(locality, req.reqId);
|
2019-02-01 02:10:41 +08:00
|
|
|
recruited.initEndpoints();
|
|
|
|
|
2019-01-29 03:29:39 +08:00
|
|
|
if (ddInterf->get().present()) {
|
|
|
|
recruited = ddInterf->get().get();
|
2022-07-20 04:15:51 +08:00
|
|
|
CODE_PROBE(true, "Recruited while already a data distributor.");
|
2019-01-29 03:29:39 +08:00
|
|
|
} else {
|
|
|
|
startRole(Role::DATA_DISTRIBUTOR, recruited.id(), interf.id());
|
2019-03-13 02:34:16 +08:00
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
2019-01-29 03:29:39 +08:00
|
|
|
|
|
|
|
Future<Void> dataDistributorProcess = dataDistributor(recruited, dbInfo);
|
2019-02-01 02:10:41 +08:00
|
|
|
errorForwarders.add(forwardError(
|
|
|
|
errors,
|
|
|
|
Role::DATA_DISTRIBUTOR,
|
|
|
|
recruited.id(),
|
|
|
|
setWhenDoneOrError(dataDistributorProcess, ddInterf, Optional<DataDistributorInterface>())));
|
2019-01-29 03:29:39 +08:00
|
|
|
ddInterf->set(Optional<DataDistributorInterface>(recruited));
|
|
|
|
}
|
2019-01-29 01:25:15 +08:00
|
|
|
TraceEvent("DataDistributorReceived", req.reqId).detail("DataDistributorId", recruited.id());
|
2018-12-14 05:31:37 +08:00
|
|
|
req.reply.send(recruited);
|
|
|
|
}
|
2019-02-15 08:24:46 +08:00
|
|
|
when(InitializeRatekeeperRequest req = waitNext(interf.ratekeeper.getFuture())) {
|
2021-06-14 09:23:59 +08:00
|
|
|
LocalLineage _;
|
2021-05-28 02:25:00 +08:00
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::Ratekeeper;
|
2019-03-20 02:29:19 +08:00
|
|
|
RatekeeperInterface recruited(locality, req.reqId);
|
2019-02-15 08:24:46 +08:00
|
|
|
recruited.initEndpoints();
|
|
|
|
|
|
|
|
if (rkInterf->get().present()) {
|
|
|
|
recruited = rkInterf->get().get();
|
2022-07-20 04:15:51 +08:00
|
|
|
CODE_PROBE(true, "Recruited while already a ratekeeper.");
|
2019-02-15 08:24:46 +08:00
|
|
|
} else {
|
2019-03-27 23:15:19 +08:00
|
|
|
startRole(Role::RATEKEEPER, recruited.id(), interf.id());
|
2019-03-13 02:34:16 +08:00
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
DUMPTOKEN(recruited.getRateInfo);
|
2019-03-15 06:00:57 +08:00
|
|
|
DUMPTOKEN(recruited.haltRatekeeper);
|
2020-08-30 03:35:31 +08:00
|
|
|
DUMPTOKEN(recruited.reportCommitCostEstimation);
|
2019-03-13 02:34:16 +08:00
|
|
|
|
2019-03-27 23:41:19 +08:00
|
|
|
Future<Void> ratekeeperProcess = ratekeeper(recruited, dbInfo);
|
2019-03-27 23:15:19 +08:00
|
|
|
errorForwarders.add(
|
|
|
|
forwardError(errors,
|
|
|
|
Role::RATEKEEPER,
|
|
|
|
recruited.id(),
|
2019-03-27 23:41:19 +08:00
|
|
|
setWhenDoneOrError(ratekeeperProcess, rkInterf, Optional<RatekeeperInterface>())));
|
2019-02-15 08:24:46 +08:00
|
|
|
rkInterf->set(Optional<RatekeeperInterface>(recruited));
|
|
|
|
}
|
|
|
|
TraceEvent("Ratekeeper_InitRequest", req.reqId).detail("RatekeeperId", recruited.id());
|
|
|
|
req.reply.send(recruited);
|
|
|
|
}
|
2022-09-17 00:03:06 +08:00
|
|
|
when(InitializeConsistencyScanRequest req = waitNext(interf.consistencyScan.getFuture())) {
|
|
|
|
LocalLineage _;
|
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::ConsistencyScan;
|
|
|
|
ConsistencyScanInterface recruited(locality, req.reqId);
|
|
|
|
recruited.initEndpoints();
|
|
|
|
|
|
|
|
if (csInterf->get().present()) {
|
|
|
|
recruited = csInterf->get().get();
|
|
|
|
CODE_PROBE(true, "Recovered while already a consistencyscan");
|
|
|
|
} else {
|
|
|
|
startRole(Role::CONSISTENCYSCAN, recruited.id(), interf.id());
|
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
DUMPTOKEN(recruited.haltConsistencyScan);
|
|
|
|
|
|
|
|
Future<Void> consistencyScanProcess = consistencyScan(recruited, dbInfo);
|
|
|
|
errorForwarders.add(forwardError(
|
|
|
|
errors,
|
|
|
|
Role::CONSISTENCYSCAN,
|
|
|
|
recruited.id(),
|
|
|
|
setWhenDoneOrError(consistencyScanProcess, csInterf, Optional<ConsistencyScanInterface>())));
|
|
|
|
csInterf->set(Optional<ConsistencyScanInterface>(recruited));
|
|
|
|
}
|
|
|
|
TraceEvent("ConsistencyScanReceived", req.reqId).detail("ConsistencyScanId", recruited.id());
|
|
|
|
req.reply.send(recruited);
|
|
|
|
}
|
2021-09-15 23:35:58 +08:00
|
|
|
when(InitializeBlobManagerRequest req = waitNext(interf.blobManager.getFuture())) {
|
2021-09-23 12:12:14 +08:00
|
|
|
LocalLineage _;
|
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::BlobManager;
|
2022-08-24 02:33:46 +08:00
|
|
|
BlobManagerInterface recruited(locality, req.reqId, req.epoch);
|
2021-09-15 23:35:58 +08:00
|
|
|
recruited.initEndpoints();
|
|
|
|
|
2022-03-18 03:40:34 +08:00
|
|
|
if (bmEpochAndInterf->get().present() && bmEpochAndInterf->get().get().first == req.epoch) {
|
2022-08-19 01:42:53 +08:00
|
|
|
ASSERT(req.reqId == lastBMRecruitRequestId);
|
2022-03-18 03:40:34 +08:00
|
|
|
recruited = bmEpochAndInterf->get().get().second;
|
2022-01-24 23:46:14 +08:00
|
|
|
|
2022-07-20 04:15:51 +08:00
|
|
|
CODE_PROBE(true, "Recruited while already a blob manager.");
|
2022-08-19 01:42:53 +08:00
|
|
|
} else if (lastBMRecruitRequestId == req.reqId && !bmEpochAndInterf->get().present()) {
|
|
|
|
// The previous blob manager WAS present, like the above case, but it died before the CC got the
|
|
|
|
// response to the recruitment request, so the CC retried to recruit the same blob manager id/epoch
|
|
|
|
// from the same reqId. To keep epoch safety between different managers, instead of restarting the
|
|
|
|
// same manager id at the same epoch, we should just tell it the original request succeeded, and let
|
|
|
|
// it realize this manager died via failure detection and start a new one.
|
2022-09-26 06:28:32 +08:00
|
|
|
CODE_PROBE(true, "Recruited while formerly the same blob manager.", probe::decoration::rare);
|
2021-09-15 23:35:58 +08:00
|
|
|
} else {
|
2022-03-09 23:49:49 +08:00
|
|
|
// TODO: it'd be more optimal to halt the last manager if present here, but it will figure it out
|
|
|
|
// via the epoch check
|
|
|
|
// Also, not halting lets us handle the case here where the last BM had a higher
|
|
|
|
// epoch and somehow the epochs got out of order by a delayed initialize request. The one we start
|
|
|
|
// here will just halt on the lock check.
|
2021-09-15 23:35:58 +08:00
|
|
|
startRole(Role::BLOB_MANAGER, recruited.id(), interf.id());
|
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
2021-09-23 12:12:14 +08:00
|
|
|
DUMPTOKEN(recruited.haltBlobManager);
|
2022-03-09 23:49:49 +08:00
|
|
|
DUMPTOKEN(recruited.haltBlobGranules);
|
|
|
|
DUMPTOKEN(recruited.blobManagerExclCheckReq);
|
2021-09-15 23:35:58 +08:00
|
|
|
|
2022-08-19 01:42:53 +08:00
|
|
|
lastBMRecruitRequestId = req.reqId;
|
|
|
|
|
2021-09-15 23:35:58 +08:00
|
|
|
Future<Void> blobManagerProcess = blobManager(recruited, dbInfo, req.epoch);
|
2022-03-18 03:40:34 +08:00
|
|
|
errorForwarders.add(
|
|
|
|
forwardError(errors,
|
|
|
|
Role::BLOB_MANAGER,
|
|
|
|
recruited.id(),
|
|
|
|
resetBlobManagerWhenDoneOrError(blobManagerProcess, bmEpochAndInterf, req.epoch)));
|
|
|
|
bmEpochAndInterf->set(
|
|
|
|
Optional<std::pair<int64_t, BlobManagerInterface>>(std::pair(req.epoch, recruited)));
|
2021-09-15 23:35:58 +08:00
|
|
|
}
|
|
|
|
TraceEvent("BlobManagerReceived", req.reqId).detail("BlobManagerId", recruited.id());
|
|
|
|
req.reply.send(recruited);
|
|
|
|
}
|
2022-09-03 10:21:52 +08:00
|
|
|
when(InitializeBlobMigratorRequest req = waitNext(interf.blobMigrator.getFuture())) {
|
|
|
|
LocalLineage _;
|
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::BlobMigrator;
|
|
|
|
|
|
|
|
BlobMigratorInterface recruited(locality, req.reqId);
|
|
|
|
recruited.initEndpoints();
|
|
|
|
if (blobMigratorInterf->get().present()) {
|
|
|
|
recruited = blobMigratorInterf->get().get();
|
|
|
|
CODE_PROBE(true, "Recruited while already a blob migrator.");
|
|
|
|
} else {
|
|
|
|
startRole(Role::BLOB_MIGRATOR, recruited.id(), interf.id());
|
2022-10-13 06:25:37 +08:00
|
|
|
DUMPTOKEN(recruited.haltBlobMigrator);
|
|
|
|
DUMPTOKEN(recruited.ssi.getValue);
|
|
|
|
DUMPTOKEN(recruited.ssi.getKey);
|
|
|
|
DUMPTOKEN(recruited.ssi.getKeyValues);
|
|
|
|
DUMPTOKEN(recruited.ssi.getMappedKeyValues);
|
|
|
|
DUMPTOKEN(recruited.ssi.getShardState);
|
|
|
|
DUMPTOKEN(recruited.ssi.waitMetrics);
|
|
|
|
DUMPTOKEN(recruited.ssi.splitMetrics);
|
|
|
|
DUMPTOKEN(recruited.ssi.getReadHotRanges);
|
|
|
|
DUMPTOKEN(recruited.ssi.getRangeSplitPoints);
|
|
|
|
DUMPTOKEN(recruited.ssi.getStorageMetrics);
|
|
|
|
DUMPTOKEN(recruited.ssi.waitFailure);
|
|
|
|
DUMPTOKEN(recruited.ssi.getQueuingMetrics);
|
|
|
|
DUMPTOKEN(recruited.ssi.getKeyValueStoreType);
|
|
|
|
DUMPTOKEN(recruited.ssi.watchValue);
|
|
|
|
DUMPTOKEN(recruited.ssi.getKeyValuesStream);
|
|
|
|
DUMPTOKEN(recruited.ssi.changeFeedStream);
|
|
|
|
DUMPTOKEN(recruited.ssi.changeFeedPop);
|
|
|
|
DUMPTOKEN(recruited.ssi.changeFeedVersionUpdate);
|
2022-09-03 10:21:52 +08:00
|
|
|
|
|
|
|
Future<Void> blobMigratorProcess = blobMigrator(recruited, dbInfo);
|
|
|
|
errorForwarders.add(forwardError(errors,
|
|
|
|
Role::BLOB_MIGRATOR,
|
|
|
|
recruited.id(),
|
|
|
|
setWhenDoneOrError(blobMigratorProcess,
|
|
|
|
blobMigratorInterf,
|
|
|
|
Optional<BlobMigratorInterface>())));
|
|
|
|
blobMigratorInterf->set(Optional<BlobMigratorInterface>(recruited));
|
|
|
|
}
|
|
|
|
TraceEvent("BlobMigrator_InitRequest", req.reqId).detail("BlobMigratorId", recruited.id());
|
|
|
|
req.reply.send(recruited);
|
|
|
|
}
|
2019-04-25 06:12:37 +08:00
|
|
|
when(InitializeBackupRequest req = waitNext(interf.backup.getFuture())) {
|
2020-03-24 01:22:24 +08:00
|
|
|
if (!backupWorkerCache.exists(req.reqId)) {
|
2021-06-14 09:23:59 +08:00
|
|
|
LocalLineage _;
|
2021-05-28 02:25:00 +08:00
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::Backup;
|
2020-03-24 01:22:24 +08:00
|
|
|
BackupInterface recruited(locality);
|
|
|
|
recruited.initEndpoints();
|
2019-04-25 06:12:37 +08:00
|
|
|
|
2020-03-24 01:22:24 +08:00
|
|
|
startRole(Role::BACKUP, recruited.id(), interf.id());
|
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
2019-04-25 06:12:37 +08:00
|
|
|
|
2020-03-24 01:22:24 +08:00
|
|
|
ReplyPromise<InitializeBackupReply> backupReady = req.reply;
|
|
|
|
backupWorkerCache.set(req.reqId, backupReady.getFuture());
|
|
|
|
Future<Void> backupProcess = backupWorker(recruited, req, dbInfo);
|
2020-03-24 03:47:42 +08:00
|
|
|
backupProcess = storageCache.removeOnReady(req.reqId, backupProcess);
|
2020-03-24 01:22:24 +08:00
|
|
|
errorForwarders.add(forwardError(errors, Role::BACKUP, recruited.id(), backupProcess));
|
|
|
|
TraceEvent("BackupInitRequest", req.reqId).detail("BackupId", recruited.id());
|
|
|
|
InitializeBackupReply reply(recruited, req.backupEpoch);
|
|
|
|
backupReady.send(reply);
|
|
|
|
} else {
|
|
|
|
forwardPromise(req.reply, backupWorkerCache.get(req.reqId));
|
|
|
|
}
|
2019-04-25 06:12:37 +08:00
|
|
|
}
|
2022-01-14 03:11:01 +08:00
|
|
|
when(InitializeEncryptKeyProxyRequest req = waitNext(interf.encryptKeyProxy.getFuture())) {
|
|
|
|
LocalLineage _;
|
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::EncryptKeyProxy;
|
|
|
|
EncryptKeyProxyInterface recruited(locality, req.reqId);
|
|
|
|
recruited.initEndpoints();
|
|
|
|
|
|
|
|
if (ekpInterf->get().present()) {
|
|
|
|
recruited = ekpInterf->get().get();
|
2022-07-20 04:15:51 +08:00
|
|
|
CODE_PROBE(true, "Recruited while already a encryptKeyProxy server.");
|
2022-01-14 03:11:01 +08:00
|
|
|
} else {
|
|
|
|
startRole(Role::ENCRYPT_KEY_PROXY, recruited.id(), interf.id());
|
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
|
|
|
|
Future<Void> encryptKeyProxyProcess = encryptKeyProxyServer(recruited, dbInfo);
|
|
|
|
errorForwarders.add(forwardError(
|
|
|
|
errors,
|
|
|
|
Role::ENCRYPT_KEY_PROXY,
|
|
|
|
recruited.id(),
|
|
|
|
setWhenDoneOrError(encryptKeyProxyProcess, ekpInterf, Optional<EncryptKeyProxyInterface>())));
|
|
|
|
ekpInterf->set(Optional<EncryptKeyProxyInterface>(recruited));
|
|
|
|
}
|
|
|
|
TraceEvent("EncryptKeyProxyReceived", req.reqId).detail("EncryptKeyProxyId", recruited.id());
|
|
|
|
req.reply.send(recruited);
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
when(InitializeTLogRequest req = waitNext(interf.tLog.getFuture())) {
|
2019-02-20 14:02:07 +08:00
|
|
|
// For now, there's a one-to-one mapping of spill type to TLogVersion.
|
|
|
|
// With future work, a particular version of the TLog can support multiple
|
|
|
|
// different spilling strategies, at which point SpillType will need to be
|
|
|
|
// plumbed down into tLogFn.
|
2019-02-23 04:15:23 +08:00
|
|
|
if (req.logVersion < TLogVersion::MIN_RECRUITABLE) {
|
|
|
|
TraceEvent(SevError, "InitializeTLogInvalidLogVersion")
|
|
|
|
.detail("Version", req.logVersion)
|
|
|
|
.detail("MinRecruitable", TLogVersion::MIN_RECRUITABLE);
|
|
|
|
req.reply.sendError(internal_error());
|
2019-02-20 14:02:07 +08:00
|
|
|
}
|
2021-06-14 09:23:59 +08:00
|
|
|
LocalLineage _;
|
2021-05-28 02:25:00 +08:00
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::TLog;
|
2019-02-23 04:15:23 +08:00
|
|
|
TLogOptions tLogOptions(req.logVersion, req.spillType);
|
|
|
|
TLogFn tLogFn = tLogFnForOptions(tLogOptions);
|
2019-10-03 16:27:36 +08:00
|
|
|
auto& logData = sharedLogs[SharedLogsKey(tLogOptions, req.storeType)];
|
2022-06-15 05:41:47 +08:00
|
|
|
while (!logData.empty() && (!logData.back().actor.isValid() || logData.back().actor.isReady())) {
|
|
|
|
logData.pop_back();
|
|
|
|
}
|
|
|
|
if (logData.empty()) {
|
2019-05-11 05:01:52 +08:00
|
|
|
UID logId = deterministicRandom()->randomUniqueID();
|
2017-05-26 04:48:44 +08:00
|
|
|
std::map<std::string, std::string> details;
|
|
|
|
details["ForMaster"] = req.recruitmentID.shortString();
|
|
|
|
details["StorageEngine"] = req.storeType.toString();
|
2017-08-29 02:25:37 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
// FIXME: start role for every tlog instance, rather that just for the shared actor, also use a
|
|
|
|
// different role type for the shared actor
|
2018-09-06 06:06:14 +08:00
|
|
|
startRole(Role::SHARED_TRANSACTION_LOG, logId, interf.id(), details);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2019-02-23 04:15:23 +08:00
|
|
|
const StringRef prefix =
|
|
|
|
req.logVersion > TLogVersion::V2 ? fileVersionedLogDataPrefix : fileLogDataPrefix;
|
|
|
|
std::string filename =
|
|
|
|
filenameFromId(req.storeType, folder, prefix.toString() + tLogOptions.toPrefix(), logId);
|
2017-05-26 04:48:44 +08:00
|
|
|
IKeyValueStore* data = openKVStore(req.storeType, filename, logId, memoryLimit);
|
2022-02-04 06:26:10 +08:00
|
|
|
const DiskQueueVersion dqv = tLogOptions.getDiskQueueVersion();
|
2019-03-16 12:01:16 +08:00
|
|
|
IDiskQueue* queue = openDiskQueue(
|
|
|
|
joinPath(folder,
|
|
|
|
fileLogQueuePrefix.toString() + tLogOptions.toPrefix() + logId.toString() + "-"),
|
|
|
|
tlogQueueExtension.toString(),
|
|
|
|
logId,
|
|
|
|
dqv);
|
2017-05-26 04:48:44 +08:00
|
|
|
filesClosed.add(data->onClosed());
|
|
|
|
filesClosed.add(queue->onClosed());
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2022-06-15 05:41:47 +08:00
|
|
|
logData.push_back(SharedLogsValue());
|
2020-02-13 07:11:38 +08:00
|
|
|
Future<Void> tLogCore = tLogFn(data,
|
|
|
|
queue,
|
|
|
|
dbInfo,
|
|
|
|
locality,
|
2022-06-15 05:41:47 +08:00
|
|
|
logData.back().requests,
|
2020-02-13 07:11:38 +08:00
|
|
|
logId,
|
|
|
|
interf.id(),
|
|
|
|
false,
|
|
|
|
Promise<Void>(),
|
|
|
|
Promise<Void>(),
|
|
|
|
folder,
|
|
|
|
degraded,
|
|
|
|
activeSharedTLog);
|
2019-07-30 14:40:28 +08:00
|
|
|
tLogCore = handleIOErrors(tLogCore, data, logId);
|
|
|
|
tLogCore = handleIOErrors(tLogCore, queue, logId);
|
|
|
|
errorForwarders.add(forwardError(errors, Role::SHARED_TRANSACTION_LOG, logId, tLogCore));
|
2022-06-15 05:41:47 +08:00
|
|
|
logData.back().actor = tLogCore;
|
|
|
|
logData.back().uid = logId;
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2022-06-15 05:41:47 +08:00
|
|
|
logData.back().requests.send(req);
|
|
|
|
activeSharedTLog->set(logData.back().uid);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
when(InitializeStorageRequest req = waitNext(interf.storage.getFuture())) {
|
2021-06-16 06:49:27 +08:00
|
|
|
// We want to prevent double recruiting on a worker unless we try to recruit something
|
|
|
|
// with a different storage engine (otherwise storage migration won't work for certain
|
|
|
|
// configuration). Additionally we also need to allow double recruitment for seed servers.
|
|
|
|
// The reason for this is that a storage will only remove itself if after it was able
|
|
|
|
// to read the system key space. But if recovery fails right after a `configure new ...`
|
|
|
|
// was run it won't be able to do so.
|
2021-06-16 06:00:43 +08:00
|
|
|
if (!storageCache.exists(req.reqId) &&
|
|
|
|
(std::all_of(runningStorages.begin(),
|
|
|
|
runningStorages.end(),
|
|
|
|
[&req](const auto& p) { return p.second != req.storeType; }) ||
|
|
|
|
req.seedTag != invalidTag)) {
|
2022-10-21 05:41:30 +08:00
|
|
|
ASSERT(req.clusterId.isValid());
|
Add fdbcli command to read/write version epoch (#6480)
* Initialize cluster version at wall-clock time
Previously, new clusters would begin at version 0. After this change,
clusters will initialize at a version matching wall-clock time. Instead
of using the Unix epoch (or Windows epoch), FDB clusters will use a new
epoch, defaulting to January 1, 2010, 01:00:00+00:00. In the future,
this base epoch will be modifiable through fdbcli, allowing
administrators to advance the cluster version.
Basing the version off of time allows different FDB clusters to share
data without running into version issues.
* Send version epoch to master
* Cleanup
* Update fdbserver/storageserver.actor.cpp
Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com>
* Jump directly to expected version if possible
* Fix initial version issue on storage servers
* Add random recovery offset to start version in simulation
* Type fixes
* Disable reference time by default
Enable on a cluster using the fdbcli command `versionepoch add 0`.
* Use correct recoveryTransactionVersion when recovering
* Allow version epoch to be adjusted forwards (to decrease the version)
* Set version epoch in simulation
* Add quiet database check to ensure small version offset
* Fix initial version issue on storage servers
* Disable reference time by default
Enable on a cluster using the fdbcli command `versionepoch add 0`.
* Add fdbcli command to read/write version epoch
* Cause recovery when version epoch is set
* Handle optional version epoch key
* Add ability to clear the version epoch
This causes version advancement to revert to the old methodology whereas
versions attempt to advance by about a million versions per second,
instead of trying to match the clock.
* Update transaction access
* Modify version epoch to use microseconds instead of seconds
* Modify fdbcli version target API
Move commands from `versionepoch` to `targetversion` top level command.
* Add fdbcli tests for
* Temporarily disable targetversion cli tests
* Fix version epoch fetch issue
* Fix Arena issue
* Reduce max version jump in simulation to 1,000,000
* Rework fdbcli API
It now requires two commands to fully switch a cluster to using the
version epoch. First, enable the version epoch with `versionepoch
enable` or `versionepoch set <versionepoch>`. At this point, versions
will be given out at a faster or slower rate in an attempt to reach the
expected version. Then, run `versionepoch commit` to perform a one time
jump to the expected version. This is essentially irreversible.
* Temporarily disable old targetversion tests
* Cleanup
* Move version epoch buggify to sequencer
This will cause some issues with the QuietDatabase check for the version
offset - namely, it won't do anything, since the version epoch is not
being written to the txnStateStore in simulation. This will get fixed in
the future.
Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com>
2022-04-09 03:33:19 +08:00
|
|
|
ASSERT(req.initialClusterVersion >= 0);
|
2021-06-14 09:23:59 +08:00
|
|
|
LocalLineage _;
|
2021-05-28 02:25:00 +08:00
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::Storage;
|
2021-05-13 02:53:20 +08:00
|
|
|
bool isTss = req.tssPairIDAndVersion.present();
|
2017-05-26 04:48:44 +08:00
|
|
|
StorageServerInterface recruited(req.interfaceId);
|
|
|
|
recruited.locality = locality;
|
2021-05-13 02:53:20 +08:00
|
|
|
recruited.tssPairID = isTss ? req.tssPairIDAndVersion.get().first : Optional<UID>();
|
2017-05-26 04:48:44 +08:00
|
|
|
recruited.initEndpoints();
|
|
|
|
|
|
|
|
std::map<std::string, std::string> details;
|
|
|
|
details["StorageEngine"] = req.storeType.toString();
|
2021-05-13 02:53:20 +08:00
|
|
|
details["IsTSS"] = std::to_string(isTss);
|
|
|
|
Role ssRole = isTss ? Role::TESTING_STORAGE_SERVER : Role::STORAGE_SERVER;
|
2021-03-06 03:28:15 +08:00
|
|
|
startRole(ssRole, recruited.id(), interf.id(), details);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
DUMPTOKEN(recruited.getValue);
|
|
|
|
DUMPTOKEN(recruited.getKey);
|
|
|
|
DUMPTOKEN(recruited.getKeyValues);
|
2022-03-18 07:45:42 +08:00
|
|
|
DUMPTOKEN(recruited.getMappedKeyValues);
|
2017-05-26 04:48:44 +08:00
|
|
|
DUMPTOKEN(recruited.getShardState);
|
|
|
|
DUMPTOKEN(recruited.waitMetrics);
|
|
|
|
DUMPTOKEN(recruited.splitMetrics);
|
2020-01-13 07:30:36 +08:00
|
|
|
DUMPTOKEN(recruited.getReadHotRanges);
|
2020-07-01 05:28:15 +08:00
|
|
|
DUMPTOKEN(recruited.getRangeSplitPoints);
|
2019-07-26 07:27:32 +08:00
|
|
|
DUMPTOKEN(recruited.getStorageMetrics);
|
2017-05-26 04:48:44 +08:00
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
DUMPTOKEN(recruited.getQueuingMetrics);
|
|
|
|
DUMPTOKEN(recruited.getKeyValueStoreType);
|
|
|
|
DUMPTOKEN(recruited.watchValue);
|
2021-06-20 00:47:13 +08:00
|
|
|
DUMPTOKEN(recruited.getKeyValuesStream);
|
2022-01-12 05:35:20 +08:00
|
|
|
DUMPTOKEN(recruited.changeFeedStream);
|
|
|
|
DUMPTOKEN(recruited.changeFeedPop);
|
|
|
|
DUMPTOKEN(recruited.changeFeedVersionUpdate);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
std::string filename =
|
2021-03-06 03:28:15 +08:00
|
|
|
filenameFromId(req.storeType,
|
|
|
|
folder,
|
2021-05-13 02:53:20 +08:00
|
|
|
isTss ? testingStoragePrefix.toString() : fileStoragePrefix.toString(),
|
2021-03-06 03:28:15 +08:00
|
|
|
recruited.id());
|
2022-09-17 06:56:05 +08:00
|
|
|
Reference<IPageEncryptionKeyProvider> encryptionKeyProvider =
|
Support Redwood encryption (#7376)
A new knob `ENABLE_STORAGE_SERVER_ENCRYPTION` is added, which despite its name, currently only Redwood supports it. The knob is mean to be only used in tests to test encryption in individual components, and otherwise enabling encryption should be done through the general `ENABLE_ENCRYPTION` knob.
Under the hood, a new `Encryption` encoding type is added to `IPager`, which use AES-256 to encrypt a page. With this encoding, `BlobCipherEncryptHeader` is inserted into page header for encryption metadata. Moreover, since we compute and store an SHA-256 auth token with the encryption header, we rely on it to checksum the data (and the encryption header), and skip the standard xxhash checksum.
`EncryptionKeyProvider` implements the `IEncryptionKeyProvider` interface to provide encryption keys, which utilizes the existing `getLatestEncryptCipherKey` and `getEncryptCipherKey` actors to fetch encryption keys from either local cache or EKP server. If multi-tenancy is used, for writing a new page, `EncryptionKeyProvider` checks if a page contain only data for a single tenant, if so, fetches tenant specific encryption key; otherwise system encryption key is used. The tenant check is done by extracting tenant id from page bound key prefixes. `EncryptionKeyProvider` also holds a reference of the `tenantPrefixIndex` map maintained by storage server, which is used to check if a tenant do exists, and getting the tenant name in order to get the encryption key.
2022-09-01 03:19:55 +08:00
|
|
|
makeReference<TenantAwareEncryptionKeyProvider>(dbInfo);
|
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-04-01 08:08:59 +08:00
|
|
|
IKeyValueStore* data = openKVStore(
|
|
|
|
req.storeType,
|
|
|
|
filename,
|
|
|
|
recruited.id(),
|
|
|
|
memoryLimit,
|
|
|
|
false,
|
|
|
|
false,
|
2022-04-07 06:09:22 +08:00
|
|
|
SERVER_KNOBS->REMOTE_KV_STORE && /* testing mixed mode in simulation if remote kvs enabled */
|
|
|
|
(g_network->isSimulated()
|
|
|
|
? (/* Disable for RocksDB */ req.storeType != KeyValueStoreType::SSD_ROCKSDB_V1 &&
|
2022-04-22 10:53:14 +08:00
|
|
|
req.storeType != KeyValueStoreType::SSD_SHARDED_ROCKSDB &&
|
2022-04-07 06:09:22 +08:00
|
|
|
deterministicRandom()->coinflip())
|
Support Redwood encryption (#7376)
A new knob `ENABLE_STORAGE_SERVER_ENCRYPTION` is added, which despite its name, currently only Redwood supports it. The knob is mean to be only used in tests to test encryption in individual components, and otherwise enabling encryption should be done through the general `ENABLE_ENCRYPTION` knob.
Under the hood, a new `Encryption` encoding type is added to `IPager`, which use AES-256 to encrypt a page. With this encoding, `BlobCipherEncryptHeader` is inserted into page header for encryption metadata. Moreover, since we compute and store an SHA-256 auth token with the encryption header, we rely on it to checksum the data (and the encryption header), and skip the standard xxhash checksum.
`EncryptionKeyProvider` implements the `IEncryptionKeyProvider` interface to provide encryption keys, which utilizes the existing `getLatestEncryptCipherKey` and `getEncryptCipherKey` actors to fetch encryption keys from either local cache or EKP server. If multi-tenancy is used, for writing a new page, `EncryptionKeyProvider` checks if a page contain only data for a single tenant, if so, fetches tenant specific encryption key; otherwise system encryption key is used. The tenant check is done by extracting tenant id from page bound key prefixes. `EncryptionKeyProvider` also holds a reference of the `tenantPrefixIndex` map maintained by storage server, which is used to check if a tenant do exists, and getting the tenant name in order to get the encryption key.
2022-09-01 03:19:55 +08:00
|
|
|
: true),
|
|
|
|
encryptionKeyProvider);
|
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-04-01 08:08:59 +08:00
|
|
|
|
2022-08-24 15:40:38 +08:00
|
|
|
Future<Void> kvClosed =
|
|
|
|
data->onClosed() ||
|
|
|
|
rebootKVSPromise2
|
|
|
|
.getFuture() /* clear the onClosed() Future in actorCollection when rebooting */;
|
2017-05-26 04:48:44 +08:00
|
|
|
filesClosed.add(kvClosed);
|
2018-05-06 09:16:28 +08:00
|
|
|
ReplyPromise<InitializeStorageReply> storageReady = req.reply;
|
2017-05-26 04:48:44 +08:00
|
|
|
storageCache.set(req.reqId, storageReady.getFuture());
|
2021-05-13 02:53:20 +08:00
|
|
|
Future<Void> s = storageServer(data,
|
|
|
|
recruited,
|
|
|
|
req.seedTag,
|
2022-10-21 05:41:30 +08:00
|
|
|
req.clusterId,
|
Add fdbcli command to read/write version epoch (#6480)
* Initialize cluster version at wall-clock time
Previously, new clusters would begin at version 0. After this change,
clusters will initialize at a version matching wall-clock time. Instead
of using the Unix epoch (or Windows epoch), FDB clusters will use a new
epoch, defaulting to January 1, 2010, 01:00:00+00:00. In the future,
this base epoch will be modifiable through fdbcli, allowing
administrators to advance the cluster version.
Basing the version off of time allows different FDB clusters to share
data without running into version issues.
* Send version epoch to master
* Cleanup
* Update fdbserver/storageserver.actor.cpp
Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com>
* Jump directly to expected version if possible
* Fix initial version issue on storage servers
* Add random recovery offset to start version in simulation
* Type fixes
* Disable reference time by default
Enable on a cluster using the fdbcli command `versionepoch add 0`.
* Use correct recoveryTransactionVersion when recovering
* Allow version epoch to be adjusted forwards (to decrease the version)
* Set version epoch in simulation
* Add quiet database check to ensure small version offset
* Fix initial version issue on storage servers
* Disable reference time by default
Enable on a cluster using the fdbcli command `versionepoch add 0`.
* Add fdbcli command to read/write version epoch
* Cause recovery when version epoch is set
* Handle optional version epoch key
* Add ability to clear the version epoch
This causes version advancement to revert to the old methodology whereas
versions attempt to advance by about a million versions per second,
instead of trying to match the clock.
* Update transaction access
* Modify version epoch to use microseconds instead of seconds
* Modify fdbcli version target API
Move commands from `versionepoch` to `targetversion` top level command.
* Add fdbcli tests for
* Temporarily disable targetversion cli tests
* Fix version epoch fetch issue
* Fix Arena issue
* Reduce max version jump in simulation to 1,000,000
* Rework fdbcli API
It now requires two commands to fully switch a cluster to using the
version epoch. First, enable the version epoch with `versionepoch
enable` or `versionepoch set <versionepoch>`. At this point, versions
will be given out at a faster or slower rate in an attempt to reach the
expected version. Then, run `versionepoch commit` to perform a one time
jump to the expected version. This is essentially irreversible.
* Temporarily disable old targetversion tests
* Cleanup
* Move version epoch buggify to sequencer
This will cause some issues with the QuietDatabase check for the version
offset - namely, it won't do anything, since the version epoch is not
being written to the txnStateStore in simulation. This will get fixed in
the future.
Co-authored-by: A.J. Beamon <aj.beamon@snowflake.com>
2022-04-09 03:33:19 +08:00
|
|
|
req.initialClusterVersion,
|
2021-05-13 02:53:20 +08:00
|
|
|
isTss ? req.tssPairIDAndVersion.get().second : 0,
|
|
|
|
storageReady,
|
|
|
|
dbInfo,
|
Support Redwood encryption (#7376)
A new knob `ENABLE_STORAGE_SERVER_ENCRYPTION` is added, which despite its name, currently only Redwood supports it. The knob is mean to be only used in tests to test encryption in individual components, and otherwise enabling encryption should be done through the general `ENABLE_ENCRYPTION` knob.
Under the hood, a new `Encryption` encoding type is added to `IPager`, which use AES-256 to encrypt a page. With this encoding, `BlobCipherEncryptHeader` is inserted into page header for encryption metadata. Moreover, since we compute and store an SHA-256 auth token with the encryption header, we rely on it to checksum the data (and the encryption header), and skip the standard xxhash checksum.
`EncryptionKeyProvider` implements the `IEncryptionKeyProvider` interface to provide encryption keys, which utilizes the existing `getLatestEncryptCipherKey` and `getEncryptCipherKey` actors to fetch encryption keys from either local cache or EKP server. If multi-tenancy is used, for writing a new page, `EncryptionKeyProvider` checks if a page contain only data for a single tenant, if so, fetches tenant specific encryption key; otherwise system encryption key is used. The tenant check is done by extracting tenant id from page bound key prefixes. `EncryptionKeyProvider` also holds a reference of the `tenantPrefixIndex` map maintained by storage server, which is used to check if a tenant do exists, and getting the tenant name in order to get the encryption key.
2022-09-01 03:19:55 +08:00
|
|
|
folder,
|
|
|
|
encryptionKeyProvider);
|
2017-05-26 04:48:44 +08:00
|
|
|
s = handleIOErrors(s, data, recruited.id(), kvClosed);
|
|
|
|
s = storageCache.removeOnReady(req.reqId, s);
|
2021-06-16 06:00:43 +08:00
|
|
|
s = storageServerRollbackRebooter(&runningStorages,
|
|
|
|
s,
|
2018-10-09 08:26:10 +08:00
|
|
|
req.storeType,
|
|
|
|
filename,
|
|
|
|
recruited.id(),
|
|
|
|
recruited.locality,
|
2021-05-13 02:53:20 +08:00
|
|
|
isTss,
|
2018-10-09 08:26:10 +08:00
|
|
|
dbInfo,
|
|
|
|
folder,
|
|
|
|
&filesClosed,
|
|
|
|
memoryLimit,
|
2022-08-24 15:40:38 +08:00
|
|
|
data,
|
|
|
|
false,
|
Support Redwood encryption (#7376)
A new knob `ENABLE_STORAGE_SERVER_ENCRYPTION` is added, which despite its name, currently only Redwood supports it. The knob is mean to be only used in tests to test encryption in individual components, and otherwise enabling encryption should be done through the general `ENABLE_ENCRYPTION` knob.
Under the hood, a new `Encryption` encoding type is added to `IPager`, which use AES-256 to encrypt a page. With this encoding, `BlobCipherEncryptHeader` is inserted into page header for encryption metadata. Moreover, since we compute and store an SHA-256 auth token with the encryption header, we rely on it to checksum the data (and the encryption header), and skip the standard xxhash checksum.
`EncryptionKeyProvider` implements the `IEncryptionKeyProvider` interface to provide encryption keys, which utilizes the existing `getLatestEncryptCipherKey` and `getEncryptCipherKey` actors to fetch encryption keys from either local cache or EKP server. If multi-tenancy is used, for writing a new page, `EncryptionKeyProvider` checks if a page contain only data for a single tenant, if so, fetches tenant specific encryption key; otherwise system encryption key is used. The tenant check is done by extracting tenant id from page bound key prefixes. `EncryptionKeyProvider` also holds a reference of the `tenantPrefixIndex` map maintained by storage server, which is used to check if a tenant do exists, and getting the tenant name in order to get the encryption key.
2022-09-01 03:19:55 +08:00
|
|
|
&rebootKVSPromise2,
|
|
|
|
encryptionKeyProvider);
|
2021-03-06 03:28:15 +08:00
|
|
|
errorForwarders.add(forwardError(errors, ssRole, recruited.id(), s));
|
2021-06-16 06:00:43 +08:00
|
|
|
} else if (storageCache.exists(req.reqId)) {
|
2017-05-26 04:48:44 +08:00
|
|
|
forwardPromise(req.reply, storageCache.get(req.reqId));
|
2021-06-16 06:00:43 +08:00
|
|
|
} else {
|
2022-01-12 05:22:00 +08:00
|
|
|
TraceEvent("AttemptedDoubleRecruitment", interf.id()).detail("ForRole", "StorageServer");
|
2021-06-16 07:37:05 +08:00
|
|
|
errorForwarders.add(map(delay(0.5), [reply = req.reply](Void) {
|
|
|
|
reply.sendError(recruitment_failed());
|
|
|
|
return Void();
|
|
|
|
}));
|
2021-06-16 06:00:43 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2021-09-21 03:42:20 +08:00
|
|
|
when(InitializeBlobWorkerRequest req = waitNext(interf.blobWorker.getFuture())) {
|
2021-12-04 02:29:22 +08:00
|
|
|
if (!blobWorkerCache.exists(req.reqId)) {
|
|
|
|
BlobWorkerInterface recruited(locality, req.interfaceId);
|
|
|
|
recruited.initEndpoints();
|
|
|
|
startRole(Role::BLOB_WORKER, recruited.id(), interf.id());
|
2021-09-21 03:42:20 +08:00
|
|
|
|
2022-03-09 23:49:49 +08:00
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
DUMPTOKEN(recruited.blobGranuleFileRequest);
|
|
|
|
DUMPTOKEN(recruited.assignBlobRangeRequest);
|
|
|
|
DUMPTOKEN(recruited.revokeBlobRangeRequest);
|
|
|
|
DUMPTOKEN(recruited.granuleAssignmentsRequest);
|
|
|
|
DUMPTOKEN(recruited.granuleStatusStreamRequest);
|
|
|
|
DUMPTOKEN(recruited.haltBlobWorker);
|
2022-08-13 04:15:56 +08:00
|
|
|
DUMPTOKEN(recruited.minBlobVersionRequest);
|
2021-09-21 03:42:20 +08:00
|
|
|
|
2021-12-04 02:29:22 +08:00
|
|
|
ReplyPromise<InitializeBlobWorkerReply> blobWorkerReady = req.reply;
|
|
|
|
Future<Void> bw = blobWorker(recruited, blobWorkerReady, dbInfo);
|
|
|
|
errorForwarders.add(forwardError(errors, Role::BLOB_WORKER, recruited.id(), bw));
|
2022-03-09 23:49:49 +08:00
|
|
|
|
2021-12-04 02:29:22 +08:00
|
|
|
} else {
|
|
|
|
forwardPromise(req.reply, blobWorkerCache.get(req.reqId));
|
|
|
|
}
|
2021-09-21 03:42:20 +08:00
|
|
|
}
|
2020-09-11 08:44:15 +08:00
|
|
|
when(InitializeCommitProxyRequest req = waitNext(interf.commitProxy.getFuture())) {
|
2021-06-14 09:23:59 +08:00
|
|
|
LocalLineage _;
|
2021-05-28 02:25:00 +08:00
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::CommitProxy;
|
2020-09-11 08:44:15 +08:00
|
|
|
CommitProxyInterface recruited;
|
2020-05-02 07:41:20 +08:00
|
|
|
recruited.processId = locality.processId();
|
2019-03-20 04:37:50 +08:00
|
|
|
recruited.provisional = false;
|
2017-05-26 04:48:44 +08:00
|
|
|
recruited.initEndpoints();
|
|
|
|
|
|
|
|
std::map<std::string, std::string> details;
|
|
|
|
details["ForMaster"] = req.master.id().shortString();
|
2020-09-11 08:44:15 +08:00
|
|
|
startRole(Role::COMMIT_PROXY, recruited.id(), interf.id(), details);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
DUMPTOKEN(recruited.commit);
|
|
|
|
DUMPTOKEN(recruited.getConsistentReadVersion);
|
|
|
|
DUMPTOKEN(recruited.getKeyServersLocations);
|
|
|
|
DUMPTOKEN(recruited.getStorageServerRejoinInfo);
|
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
DUMPTOKEN(recruited.txnState);
|
|
|
|
|
2020-09-11 08:44:15 +08:00
|
|
|
// printf("Recruited as commitProxyServer\n");
|
|
|
|
errorForwarders.add(zombie(recruited,
|
|
|
|
forwardError(errors,
|
|
|
|
Role::COMMIT_PROXY,
|
|
|
|
recruited.id(),
|
|
|
|
commitProxyServer(recruited, req, dbInfo, whitelistBinPaths))));
|
2017-05-26 04:48:44 +08:00
|
|
|
req.reply.send(recruited);
|
|
|
|
}
|
2020-07-15 15:37:41 +08:00
|
|
|
when(InitializeGrvProxyRequest req = waitNext(interf.grvProxy.getFuture())) {
|
2021-06-14 09:23:59 +08:00
|
|
|
LocalLineage _;
|
2021-05-28 02:25:00 +08:00
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::GrvProxy;
|
2020-07-15 15:37:41 +08:00
|
|
|
GrvProxyInterface recruited;
|
|
|
|
recruited.processId = locality.processId();
|
|
|
|
recruited.provisional = false;
|
|
|
|
recruited.initEndpoints();
|
|
|
|
|
|
|
|
std::map<std::string, std::string> details;
|
|
|
|
details["ForMaster"] = req.master.id().shortString();
|
|
|
|
startRole(Role::GRV_PROXY, recruited.id(), interf.id(), details);
|
|
|
|
|
|
|
|
DUMPTOKEN(recruited.getConsistentReadVersion);
|
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
DUMPTOKEN(recruited.getHealthMetrics);
|
|
|
|
|
|
|
|
// printf("Recruited as grvProxyServer\n");
|
|
|
|
errorForwarders.add(zombie(
|
2017-06-30 06:50:19 +08:00
|
|
|
recruited,
|
2020-07-15 15:37:41 +08:00
|
|
|
forwardError(errors, Role::GRV_PROXY, recruited.id(), grvProxyServer(recruited, req, dbInfo))));
|
|
|
|
req.reply.send(recruited);
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
when(InitializeResolverRequest req = waitNext(interf.resolver.getFuture())) {
|
2021-06-14 09:23:59 +08:00
|
|
|
LocalLineage _;
|
2021-05-28 02:25:00 +08:00
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::Resolver;
|
2017-05-26 04:48:44 +08:00
|
|
|
ResolverInterface recruited;
|
|
|
|
recruited.locality = locality;
|
|
|
|
recruited.initEndpoints();
|
|
|
|
|
|
|
|
std::map<std::string, std::string> details;
|
2018-09-06 06:06:14 +08:00
|
|
|
startRole(Role::RESOLVER, recruited.id(), interf.id(), details);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
DUMPTOKEN(recruited.resolve);
|
|
|
|
DUMPTOKEN(recruited.metrics);
|
|
|
|
DUMPTOKEN(recruited.split);
|
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
|
2018-09-06 06:06:14 +08:00
|
|
|
errorForwarders.add(zombie(
|
|
|
|
recruited, forwardError(errors, Role::RESOLVER, recruited.id(), resolver(recruited, req, dbInfo))));
|
2017-05-26 04:48:44 +08:00
|
|
|
req.reply.send(recruited);
|
|
|
|
}
|
2017-06-30 06:50:19 +08:00
|
|
|
when(InitializeLogRouterRequest req = waitNext(interf.logRouter.getFuture())) {
|
2021-06-14 09:23:59 +08:00
|
|
|
LocalLineage _;
|
2021-05-28 02:25:00 +08:00
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::LogRouter;
|
2018-03-11 01:52:09 +08:00
|
|
|
TLogInterface recruited(locality);
|
2017-06-30 06:50:19 +08:00
|
|
|
recruited.initEndpoints();
|
|
|
|
|
|
|
|
std::map<std::string, std::string> details;
|
2018-09-06 06:06:14 +08:00
|
|
|
startRole(Role::LOG_ROUTER, recruited.id(), interf.id(), details);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-06-30 06:50:19 +08:00
|
|
|
DUMPTOKEN(recruited.peekMessages);
|
2021-07-01 12:32:30 +08:00
|
|
|
DUMPTOKEN(recruited.peekStreamMessages);
|
2017-06-30 06:50:19 +08:00
|
|
|
DUMPTOKEN(recruited.popMessages);
|
|
|
|
DUMPTOKEN(recruited.commit);
|
|
|
|
DUMPTOKEN(recruited.lock);
|
|
|
|
DUMPTOKEN(recruited.getQueuingMetrics);
|
|
|
|
DUMPTOKEN(recruited.confirmRunning);
|
2020-08-28 08:17:27 +08:00
|
|
|
DUMPTOKEN(recruited.waitFailure);
|
|
|
|
DUMPTOKEN(recruited.recoveryFinished);
|
|
|
|
DUMPTOKEN(recruited.disablePopRequest);
|
|
|
|
DUMPTOKEN(recruited.enablePopRequest);
|
|
|
|
DUMPTOKEN(recruited.snapRequest);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-01-08 11:53:09 +08:00
|
|
|
errorForwarders.add(
|
|
|
|
zombie(recruited,
|
|
|
|
forwardError(errors, Role::LOG_ROUTER, recruited.id(), logRouter(recruited, req, dbInfo))));
|
2017-06-30 06:50:19 +08:00
|
|
|
req.reply.send(recruited);
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
when(CoordinationPingMessage m = waitNext(interf.coordinationPing.getFuture())) {
|
|
|
|
TraceEvent("CoordinationPing", interf.id())
|
|
|
|
.detail("CCID", m.clusterControllerId)
|
|
|
|
.detail("TimeStep", m.timeStep);
|
|
|
|
}
|
|
|
|
when(SetMetricsLogRateRequest req = waitNext(interf.setMetricsRate.getFuture())) {
|
|
|
|
TraceEvent("LoggingRateChange", interf.id())
|
|
|
|
.detail("OldDelay", loggingDelay)
|
|
|
|
.detail("NewLogPS", req.metricsLogsPerSecond);
|
|
|
|
if (req.metricsLogsPerSecond != 0) {
|
|
|
|
loggingDelay = 1.0 / req.metricsLogsPerSecond;
|
|
|
|
loggingTrigger = Void();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
when(EventLogRequest req = waitNext(interf.eventLogRequest.getFuture())) {
|
2018-05-03 01:44:38 +08:00
|
|
|
TraceEventFields e;
|
2017-05-26 04:48:44 +08:00
|
|
|
if (req.getLastError)
|
2018-05-03 01:44:38 +08:00
|
|
|
e = latestEventCache.getLatestError();
|
2017-05-26 04:48:44 +08:00
|
|
|
else
|
2018-05-03 01:44:38 +08:00
|
|
|
e = latestEventCache.get(req.eventName.toString());
|
2017-05-26 04:48:44 +08:00
|
|
|
req.reply.send(e);
|
|
|
|
}
|
|
|
|
when(TraceBatchDumpRequest req = waitNext(interf.traceBatchDumpRequest.getFuture())) {
|
|
|
|
g_traceBatch.dump();
|
|
|
|
req.reply.send(Void());
|
|
|
|
}
|
|
|
|
when(DiskStoreRequest req = waitNext(interf.diskStoreRequest.getFuture())) {
|
|
|
|
Standalone<VectorRef<UID>> ids;
|
|
|
|
for (DiskStore d : getDiskStores(folder)) {
|
|
|
|
bool included = true;
|
|
|
|
if (!req.includePartialStores) {
|
|
|
|
if (d.storeType == KeyValueStoreType::SSD_BTREE_V1) {
|
|
|
|
included = fileExists(d.filename + ".fdb-wal");
|
|
|
|
} else if (d.storeType == KeyValueStoreType::SSD_BTREE_V2) {
|
|
|
|
included = fileExists(d.filename + ".sqlite-wal");
|
2017-09-22 14:51:55 +08:00
|
|
|
} else if (d.storeType == KeyValueStoreType::SSD_REDWOOD_V1) {
|
|
|
|
included = fileExists(d.filename + "0.pagerlog") && fileExists(d.filename + "1.pagerlog");
|
2020-06-16 00:45:36 +08:00
|
|
|
} else if (d.storeType == KeyValueStoreType::SSD_ROCKSDB_V1) {
|
|
|
|
included = fileExists(joinPath(d.filename, "CURRENT")) &&
|
|
|
|
fileExists(joinPath(d.filename, "IDENTITY"));
|
2022-04-22 10:53:14 +08:00
|
|
|
} else if (d.storeType == KeyValueStoreType::SSD_SHARDED_ROCKSDB) {
|
|
|
|
included = fileExists(joinPath(d.filename, "CURRENT")) &&
|
|
|
|
fileExists(joinPath(d.filename, "IDENTITY"));
|
2019-01-10 10:03:54 +08:00
|
|
|
} else if (d.storeType == KeyValueStoreType::MEMORY) {
|
2017-05-26 04:48:44 +08:00
|
|
|
included = fileExists(d.filename + "1.fdq");
|
2019-01-10 10:03:54 +08:00
|
|
|
} else {
|
|
|
|
ASSERT(d.storeType == KeyValueStoreType::MEMORY_RADIXTREE);
|
|
|
|
included = fileExists(d.filename + "1.fdr");
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
if (d.storedComponent == DiskStore::COMPONENT::TLogData && included) {
|
2019-02-20 14:02:07 +08:00
|
|
|
included = false;
|
2019-02-23 04:15:23 +08:00
|
|
|
// The previous code assumed that d.filename is a filename. But that is not true.
|
|
|
|
// d.filename is a path. Removing a prefix and adding a new one just makes a broken
|
|
|
|
// directory name. So fileExists would always return false.
|
|
|
|
// Weirdly, this doesn't break anything, as tested by taking a clean check of FDB,
|
|
|
|
// setting included to false always, and then running correctness. So I'm just
|
|
|
|
// improving the situation by actually marking it as broken.
|
|
|
|
// FIXME: this whole thing
|
|
|
|
/*
|
|
|
|
std::string logDataBasename;
|
|
|
|
StringRef filename = d.filename;
|
|
|
|
if (filename.startsWith(fileLogDataPrefix)) {
|
|
|
|
logDataBasename = fileLogQueuePrefix.toString() +
|
|
|
|
d.filename.substr(fileLogDataPrefix.size()); } else { StringRef optionsString =
|
|
|
|
filename.removePrefix(fileVersionedLogDataPrefix).eat("-"); logDataBasename =
|
|
|
|
fileLogQueuePrefix.toString() + optionsString.toString() + "-";
|
|
|
|
}
|
2019-02-20 14:02:07 +08:00
|
|
|
TraceEvent("DiskStoreRequest").detail("FilenameBasename", logDataBasename);
|
|
|
|
if (fileExists(logDataBasename + "0.fdq") && fileExists(logDataBasename + "1.fdq")) {
|
|
|
|
included = true;
|
|
|
|
}
|
2019-02-23 04:15:23 +08:00
|
|
|
*/
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
if (included) {
|
|
|
|
ids.push_back(ids.arena(), d.storeID);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
req.reply.send(ids);
|
|
|
|
}
|
2018-08-11 04:57:10 +08:00
|
|
|
when(wait(loggingTrigger)) {
|
2017-05-26 04:48:44 +08:00
|
|
|
systemMonitor();
|
2019-06-25 17:47:35 +08:00
|
|
|
loggingTrigger = delay(loggingDelay, TaskPriority::FlushTrace);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2019-06-20 02:20:44 +08:00
|
|
|
when(state WorkerSnapRequest snapReq = waitNext(interf.workerSnapReq.getFuture())) {
|
2022-08-13 15:10:20 +08:00
|
|
|
std::string snapReqKey = snapReq.snapUID.toString() + snapReq.role.toString();
|
|
|
|
if (snapReqResultMap.count(snapReqKey)) {
|
|
|
|
CODE_PROBE(true, "Worker received a duplicate finished snapshot request");
|
|
|
|
auto result = snapReqResultMap[snapReqKey];
|
2022-06-30 02:23:07 +08:00
|
|
|
result.isError() ? snapReq.reply.sendError(result.getError()) : snapReq.reply.send(result.get());
|
|
|
|
TraceEvent("RetryFinishedWorkerSnapRequest")
|
2022-08-13 15:10:20 +08:00
|
|
|
.detail("SnapUID", snapReq.snapUID.toString())
|
2022-06-30 02:23:07 +08:00
|
|
|
.detail("Role", snapReq.role)
|
2022-08-13 15:10:20 +08:00
|
|
|
.detail("Result", result.isError() ? result.getError().code() : success().code());
|
|
|
|
} else if (snapReqMap.count(snapReqKey)) {
|
|
|
|
CODE_PROBE(true, "Worker received a duplicate ongoing snapshot request");
|
|
|
|
TraceEvent("RetryOngoingWorkerSnapRequest")
|
|
|
|
.detail("SnapUID", snapReq.snapUID.toString())
|
|
|
|
.detail("Role", snapReq.role);
|
|
|
|
ASSERT(snapReq.role == snapReqMap[snapReqKey].role);
|
|
|
|
ASSERT(snapReq.snapPayload == snapReqMap[snapReqKey].snapPayload);
|
|
|
|
snapReqMap[snapReqKey] = snapReq;
|
2022-06-30 02:23:07 +08:00
|
|
|
} else {
|
2022-08-13 15:10:20 +08:00
|
|
|
snapReqMap[snapReqKey] = snapReq; // set map point to the request
|
2022-06-30 02:23:07 +08:00
|
|
|
if (g_network->isSimulated() && (now() - lastSnapTime) < SERVER_KNOBS->SNAP_MINIMUM_TIME_GAP) {
|
|
|
|
// only allow duplicate snapshots on same process in a short time for different roles
|
|
|
|
auto okay = (lastSnapReq.snapUID == snapReq.snapUID) && lastSnapReq.role != snapReq.role;
|
|
|
|
TraceEvent(okay ? SevInfo : SevError, "RapidSnapRequestsOnSameProcess")
|
2022-08-13 15:10:20 +08:00
|
|
|
.detail("CurrSnapUID", snapReqKey)
|
2022-06-30 02:23:07 +08:00
|
|
|
.detail("PrevSnapUID", lastSnapReq.snapUID)
|
|
|
|
.detail("CurrRole", snapReq.role)
|
|
|
|
.detail("PrevRole", lastSnapReq.role)
|
|
|
|
.detail("GapTime", now() - lastSnapTime);
|
|
|
|
}
|
|
|
|
errorForwarders.add(workerSnapCreate(snapReq,
|
|
|
|
snapReq.role.toString() == "coord" ? coordFolder : folder,
|
|
|
|
&snapReqMap,
|
|
|
|
&snapReqResultMap));
|
|
|
|
auto* snapReqResultMapPtr = &snapReqResultMap;
|
|
|
|
errorForwarders.add(fmap(
|
2022-08-13 15:10:20 +08:00
|
|
|
[snapReqResultMapPtr, snapReqKey](Void _) {
|
|
|
|
snapReqResultMapPtr->erase(snapReqKey);
|
2022-06-30 02:23:07 +08:00
|
|
|
return Void();
|
|
|
|
},
|
|
|
|
delay(SERVER_KNOBS->SNAP_MINIMUM_TIME_GAP)));
|
|
|
|
if (g_network->isSimulated()) {
|
|
|
|
lastSnapReq = snapReq;
|
|
|
|
lastSnapTime = now();
|
|
|
|
}
|
2019-06-20 02:20:44 +08:00
|
|
|
}
|
|
|
|
}
|
2018-08-11 04:57:10 +08:00
|
|
|
when(wait(errorForwarders.getResult())) {}
|
|
|
|
when(wait(handleErrors)) {}
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
} catch (Error& err) {
|
2019-08-28 01:45:09 +08:00
|
|
|
// Make sure actors are cancelled before "recovery" promises are destructed.
|
2019-08-23 02:02:14 +08:00
|
|
|
for (auto f : recoveries)
|
|
|
|
f.cancel();
|
2017-05-26 04:48:44 +08:00
|
|
|
state Error e = err;
|
|
|
|
bool ok = e.code() == error_code_please_reboot || e.code() == error_code_actor_cancelled ||
|
2022-10-21 05:41:30 +08:00
|
|
|
e.code() == error_code_please_reboot_delete || e.code() == error_code_local_config_changed;
|
2018-09-06 06:06:14 +08:00
|
|
|
endRole(Role::WORKER, interf.id(), "WorkerError", ok, e);
|
2017-05-26 04:48:44 +08:00
|
|
|
errorForwarders.clear(false);
|
2018-11-08 13:09:51 +08:00
|
|
|
sharedLogs.clear();
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2022-08-24 15:40:38 +08:00
|
|
|
if (e.code() != error_code_actor_cancelled) {
|
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-04-01 08:08:59 +08:00
|
|
|
// 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
|
2017-05-26 04:48:44 +08:00
|
|
|
stopping.send(Void());
|
2018-08-11 04:57:10 +08:00
|
|
|
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
|
2017-05-26 04:48:44 +08:00
|
|
|
TraceEvent(SevInfo, "WorkerShutdownComplete", interf.id());
|
|
|
|
}
|
|
|
|
|
|
|
|
throw e;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-08-15 08:41:57 +08:00
|
|
|
ACTOR Future<Void> extractClusterInterface(Reference<AsyncVar<Optional<ClusterControllerFullInterface>> const> in,
|
|
|
|
Reference<AsyncVar<Optional<ClusterInterface>>> out) {
|
2017-05-26 04:48:44 +08:00
|
|
|
loop {
|
2022-01-21 16:42:03 +08:00
|
|
|
if (in->get().present()) {
|
2021-08-15 08:41:57 +08:00
|
|
|
out->set(in->get().get().clientInterface);
|
2022-01-21 16:42:03 +08:00
|
|
|
} else {
|
2021-08-15 08:41:57 +08:00
|
|
|
out->set(Optional<ClusterInterface>());
|
2022-01-21 16:42:03 +08:00
|
|
|
}
|
2021-08-15 08:41:57 +08:00
|
|
|
wait(in->onChange());
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
static std::set<int> const& normalWorkerErrors() {
|
|
|
|
static std::set<int> s;
|
|
|
|
if (s.empty()) {
|
|
|
|
s.insert(error_code_please_reboot);
|
|
|
|
s.insert(error_code_please_reboot_delete);
|
2022-10-19 04:46:42 +08:00
|
|
|
s.insert(error_code_local_config_changed);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
2020-02-19 08:41:19 +08:00
|
|
|
ACTOR Future<Void> fileNotFoundToNever(Future<Void> f) {
|
2017-05-26 04:48:44 +08:00
|
|
|
try {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(f);
|
2017-05-26 04:48:44 +08:00
|
|
|
return Void();
|
2020-02-04 03:11:31 +08:00
|
|
|
} catch (Error& e) {
|
|
|
|
if (e.code() == error_code_file_not_found) {
|
2020-02-19 08:41:19 +08:00
|
|
|
TraceEvent(SevWarn, "ClusterCoordinatorFailed").error(e);
|
2017-05-26 04:48:44 +08:00
|
|
|
return Never();
|
|
|
|
}
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
ACTOR Future<Void> printTimeout() {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(delay(5));
|
2017-05-26 04:48:44 +08:00
|
|
|
if (!g_network->isSimulated()) {
|
|
|
|
fprintf(stderr, "Warning: FDBD has not joined the cluster after 5 seconds.\n");
|
|
|
|
fprintf(stderr, " Check configuration and availability using the 'status' command with the fdbcli\n");
|
|
|
|
}
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-08-15 08:41:57 +08:00
|
|
|
ACTOR Future<Void> printOnFirstConnected(Reference<AsyncVar<Optional<ClusterInterface>> const> ci) {
|
2017-05-26 04:48:44 +08:00
|
|
|
state Future<Void> timeoutFuture = printTimeout();
|
|
|
|
loop {
|
|
|
|
choose {
|
2018-08-11 04:57:10 +08:00
|
|
|
when(wait(ci->get().present() ? IFailureMonitor::failureMonitor().onStateEqual(
|
|
|
|
ci->get().get().openDatabase.getEndpoint(), FailureStatus(false))
|
|
|
|
: Never())) {
|
2017-05-26 04:48:44 +08:00
|
|
|
printf("FDBD joined cluster.\n");
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("FDBDConnected").log();
|
2017-05-26 04:48:44 +08:00
|
|
|
return Void();
|
|
|
|
}
|
2018-08-11 04:57:10 +08:00
|
|
|
when(wait(ci->onChange())) {}
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-06-09 02:51:51 +08:00
|
|
|
ClusterControllerPriorityInfo getCCPriorityInfo(std::string filePath, ProcessClass processClass) {
|
2018-06-08 04:07:19 +08:00
|
|
|
if (!fileExists(filePath))
|
|
|
|
return ClusterControllerPriorityInfo(ProcessClass(processClass.classType(), ProcessClass::CommandLineSource)
|
|
|
|
.machineClassFitness(ProcessClass::ClusterController),
|
|
|
|
false,
|
|
|
|
ClusterControllerPriorityInfo::FitnessUnknown);
|
|
|
|
std::string contents(readFileBytes(filePath, 1000));
|
|
|
|
BinaryReader br(StringRef(contents), IncludeVersion());
|
|
|
|
ClusterControllerPriorityInfo priorityInfo(
|
|
|
|
ProcessClass::UnsetFit, false, ClusterControllerPriorityInfo::FitnessUnknown);
|
|
|
|
br >> priorityInfo;
|
2018-06-09 05:03:36 +08:00
|
|
|
if (!br.empty()) {
|
|
|
|
if (g_network->isSimulated()) {
|
2018-06-09 07:09:59 +08:00
|
|
|
ASSERT(false);
|
2018-06-09 05:03:36 +08:00
|
|
|
} else {
|
|
|
|
TraceEvent(SevWarnAlways, "FitnessFileCorrupted").detail("filePath", filePath);
|
|
|
|
return ClusterControllerPriorityInfo(ProcessClass(processClass.classType(), ProcessClass::CommandLineSource)
|
|
|
|
.machineClassFitness(ProcessClass::ClusterController),
|
|
|
|
false,
|
|
|
|
ClusterControllerPriorityInfo::FitnessUnknown);
|
|
|
|
}
|
|
|
|
}
|
2018-06-08 04:07:19 +08:00
|
|
|
return priorityInfo;
|
|
|
|
}
|
|
|
|
|
2018-06-09 02:51:51 +08:00
|
|
|
ACTOR Future<Void> monitorAndWriteCCPriorityInfo(std::string filePath,
|
|
|
|
Reference<AsyncVar<ClusterControllerPriorityInfo>> asyncPriorityInfo) {
|
2018-06-08 04:07:19 +08:00
|
|
|
loop {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(asyncPriorityInfo->onChange());
|
2020-05-23 08:14:21 +08:00
|
|
|
std::string contents(BinaryWriter::toValue(asyncPriorityInfo->get(),
|
|
|
|
IncludeVersion(ProtocolVersion::withClusterControllerPriorityInfo()))
|
|
|
|
.toString());
|
2018-06-08 04:07:19 +08:00
|
|
|
atomicReplace(filePath, contents, false);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-03-29 07:49:17 +08:00
|
|
|
static const std::string versionFileName = "sw-version";
|
|
|
|
|
2022-04-05 09:42:52 +08:00
|
|
|
ACTOR Future<SWVersion> testSoftwareVersionCompatibility(std::string folder, ProtocolVersion currentVersion) {
|
2022-03-29 07:49:17 +08:00
|
|
|
try {
|
|
|
|
state std::string versionFilePath = joinPath(folder, versionFileName);
|
|
|
|
state ErrorOr<Reference<IAsyncFile>> versionFile = wait(
|
|
|
|
errorOr(IAsyncFileSystem::filesystem(g_network)->open(versionFilePath, IAsyncFile::OPEN_READONLY, 0600)));
|
|
|
|
|
|
|
|
if (versionFile.isError()) {
|
|
|
|
if (versionFile.getError().code() == error_code_file_not_found && !fileExists(versionFilePath)) {
|
|
|
|
// If a version file does not exist, we assume this is either a fresh
|
|
|
|
// installation or an upgrade from a version that does not support version files.
|
|
|
|
// Either way, we can safely continue running this version of software.
|
|
|
|
TraceEvent(SevInfo, "NoPreviousSWVersion").log();
|
2022-04-02 06:45:24 +08:00
|
|
|
return SWVersion();
|
2022-03-29 07:49:17 +08:00
|
|
|
} else {
|
|
|
|
// Dangerous to continue if we cannot do a software compatibility test
|
|
|
|
throw versionFile.getError();
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
// Test whether the most newest software version that has been run on this cluster is
|
|
|
|
// compatible with the current software version
|
2022-04-07 12:06:01 +08:00
|
|
|
state int64_t filesize = wait(versionFile.get()->size());
|
2022-04-05 09:42:52 +08:00
|
|
|
state Standalone<StringRef> buf = makeString(filesize);
|
2022-04-07 12:06:01 +08:00
|
|
|
int readLen = wait(versionFile.get()->read(mutateString(buf), filesize, 0));
|
2022-04-09 06:03:39 +08:00
|
|
|
if (filesize == 0 || readLen != filesize) {
|
|
|
|
throw file_corrupt();
|
|
|
|
}
|
2022-03-29 07:49:17 +08:00
|
|
|
|
|
|
|
try {
|
2022-04-05 09:42:52 +08:00
|
|
|
SWVersion swversion = ObjectReader::fromStringRef<SWVersion>(buf, IncludeVersion());
|
|
|
|
ProtocolVersion lowestCompatibleVersion(swversion.lowestCompatibleProtocolVersion());
|
|
|
|
if (currentVersion >= lowestCompatibleVersion) {
|
2022-04-02 06:45:24 +08:00
|
|
|
return swversion;
|
2022-03-29 07:49:17 +08:00
|
|
|
} else {
|
2022-04-05 10:31:09 +08:00
|
|
|
throw incompatible_software_version();
|
2022-03-29 07:49:17 +08:00
|
|
|
}
|
|
|
|
} catch (Error& e) {
|
|
|
|
throw e;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
} catch (Error& e) {
|
|
|
|
if (e.code() == error_code_actor_cancelled) {
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
// TODO(bvr): Inject faults
|
2022-04-07 12:06:01 +08:00
|
|
|
TraceEvent(SevWarnAlways, "OpenReadSWVersionFileError").error(e);
|
2022-03-29 07:49:17 +08:00
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-04-08 05:46:27 +08:00
|
|
|
ACTOR Future<Void> updateNewestSoftwareVersion(std::string folder,
|
|
|
|
ProtocolVersion currentVersion,
|
|
|
|
ProtocolVersion latestVersion,
|
|
|
|
ProtocolVersion minCompatibleVersion) {
|
2022-04-02 06:45:24 +08:00
|
|
|
|
|
|
|
ASSERT(currentVersion >= minCompatibleVersion);
|
|
|
|
|
2022-03-29 07:49:17 +08:00
|
|
|
try {
|
|
|
|
state std::string versionFilePath = joinPath(folder, versionFileName);
|
2022-04-07 12:06:01 +08:00
|
|
|
ErrorOr<Reference<IAsyncFile>> versionFile = wait(
|
|
|
|
errorOr(IAsyncFileSystem::filesystem(g_network)->open(versionFilePath, IAsyncFile::OPEN_READONLY, 0600)));
|
2022-03-29 07:49:17 +08:00
|
|
|
|
2022-04-07 12:06:01 +08:00
|
|
|
if (versionFile.isError() &&
|
|
|
|
(versionFile.getError().code() != error_code_file_not_found || fileExists(versionFilePath))) {
|
|
|
|
throw versionFile.getError();
|
2022-03-29 07:49:17 +08:00
|
|
|
}
|
|
|
|
|
2022-04-07 12:06:01 +08:00
|
|
|
state Reference<IAsyncFile> newVersionFile = wait(IAsyncFileSystem::filesystem()->open(
|
|
|
|
versionFilePath,
|
|
|
|
IAsyncFile::OPEN_ATOMIC_WRITE_AND_CREATE | IAsyncFile::OPEN_CREATE | IAsyncFile::OPEN_READWRITE,
|
|
|
|
0600));
|
|
|
|
|
2022-04-05 09:42:52 +08:00
|
|
|
SWVersion swVersion(latestVersion, currentVersion, minCompatibleVersion);
|
2022-06-08 05:48:01 +08:00
|
|
|
Value s = swVersionValue(swVersion);
|
|
|
|
ErrorOr<Void> e = wait(holdWhile(s, errorOr(newVersionFile->write(s.begin(), s.size(), 0))));
|
2022-04-07 12:06:01 +08:00
|
|
|
if (e.isError()) {
|
|
|
|
throw e.getError();
|
|
|
|
}
|
|
|
|
wait(newVersionFile->sync());
|
2022-03-29 07:49:17 +08:00
|
|
|
} catch (Error& e) {
|
|
|
|
if (e.code() == error_code_actor_cancelled) {
|
|
|
|
throw;
|
|
|
|
}
|
2022-04-07 12:06:01 +08:00
|
|
|
TraceEvent(SevWarnAlways, "OpenWriteSWVersionFileError").error(e);
|
2022-03-29 07:49:17 +08:00
|
|
|
throw;
|
|
|
|
}
|
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2022-04-08 05:46:27 +08:00
|
|
|
ACTOR Future<Void> testAndUpdateSoftwareVersionCompatibility(std::string dataFolder, UID processIDUid) {
|
2022-08-08 23:29:49 +08:00
|
|
|
ErrorOr<SWVersion> swVersion =
|
|
|
|
wait(errorOr(testSoftwareVersionCompatibility(dataFolder, currentProtocolVersion())));
|
2022-04-08 05:46:27 +08:00
|
|
|
if (swVersion.isError()) {
|
|
|
|
TraceEvent(SevWarnAlways, "SWVersionCompatibilityCheckError", processIDUid).error(swVersion.getError());
|
|
|
|
throw swVersion.getError();
|
|
|
|
}
|
|
|
|
|
|
|
|
TraceEvent(SevInfo, "SWVersionCompatible", processIDUid).detail("SWVersion", swVersion.get());
|
|
|
|
|
|
|
|
if (!swVersion.get().isValid() ||
|
2022-08-08 23:29:49 +08:00
|
|
|
currentProtocolVersion() > ProtocolVersion(swVersion.get().newestProtocolVersion())) {
|
2022-04-08 07:26:05 +08:00
|
|
|
ErrorOr<Void> updatedSWVersion = wait(errorOr(updateNewestSoftwareVersion(
|
2022-08-08 23:29:49 +08:00
|
|
|
dataFolder, currentProtocolVersion(), currentProtocolVersion(), minCompatibleProtocolVersion)));
|
2022-04-08 07:26:05 +08:00
|
|
|
if (updatedSWVersion.isError()) {
|
|
|
|
throw updatedSWVersion.getError();
|
2022-04-08 05:46:27 +08:00
|
|
|
}
|
2022-08-08 23:29:49 +08:00
|
|
|
} else if (currentProtocolVersion() < ProtocolVersion(swVersion.get().newestProtocolVersion())) {
|
2022-04-08 05:46:27 +08:00
|
|
|
ErrorOr<Void> updatedSWVersion = wait(
|
|
|
|
errorOr(updateNewestSoftwareVersion(dataFolder,
|
2022-08-08 23:29:49 +08:00
|
|
|
currentProtocolVersion(),
|
2022-04-08 07:26:05 +08:00
|
|
|
ProtocolVersion(swVersion.get().newestProtocolVersion()),
|
2022-04-08 05:46:27 +08:00
|
|
|
ProtocolVersion(swVersion.get().lowestCompatibleProtocolVersion()))));
|
|
|
|
if (updatedSWVersion.isError()) {
|
2022-04-08 07:26:05 +08:00
|
|
|
throw updatedSWVersion.getError();
|
2022-04-08 05:46:27 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
ErrorOr<SWVersion> newSWVersion =
|
2022-08-08 23:29:49 +08:00
|
|
|
wait(errorOr(testSoftwareVersionCompatibility(dataFolder, currentProtocolVersion())));
|
2022-04-08 05:46:27 +08:00
|
|
|
if (newSWVersion.isError()) {
|
|
|
|
TraceEvent(SevWarnAlways, "SWVersionCompatibilityCheckError", processIDUid).error(newSWVersion.getError());
|
|
|
|
throw newSWVersion.getError();
|
|
|
|
}
|
|
|
|
|
|
|
|
TraceEvent(SevInfo, "VerifiedNewSoftwareVersion", processIDUid).detail("SWVersion", newSWVersion.get());
|
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2022-04-02 06:45:24 +08:00
|
|
|
static const std::string swversionTestDirName = "sw-version-test";
|
|
|
|
|
|
|
|
TEST_CASE("/fdbserver/worker/swversion/noversionhistory") {
|
|
|
|
if (!platform::createDirectory("sw-version-test")) {
|
2022-04-09 06:03:39 +08:00
|
|
|
TraceEvent(SevWarnAlways, "FailedToCreateDirectory").detail("Directory", "sw-version-test");
|
2022-04-02 06:45:24 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2022-04-05 09:42:52 +08:00
|
|
|
ErrorOr<SWVersion> swversion = wait(errorOr(
|
|
|
|
testSoftwareVersionCompatibility(swversionTestDirName, ProtocolVersion::withStorageInterfaceReadiness())));
|
|
|
|
|
|
|
|
if (!swversion.isError()) {
|
|
|
|
ASSERT(!swversion.get().isValid());
|
|
|
|
}
|
|
|
|
|
2022-04-09 06:03:39 +08:00
|
|
|
wait(IAsyncFileSystem::filesystem()->deleteFile(joinPath(swversionTestDirName, versionFileName), true));
|
2022-04-05 09:42:52 +08:00
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
TEST_CASE("/fdbserver/worker/swversion/writeVerifyVersion") {
|
|
|
|
if (!platform::createDirectory("sw-version-test")) {
|
2022-04-09 06:03:39 +08:00
|
|
|
TraceEvent(SevWarnAlways, "FailedToCreateDirectory").detail("Directory", "sw-version-test");
|
2022-04-05 09:42:52 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2022-04-08 05:46:27 +08:00
|
|
|
ErrorOr<Void> f = wait(errorOr(updateNewestSoftwareVersion(swversionTestDirName,
|
|
|
|
ProtocolVersion::withStorageInterfaceReadiness(),
|
|
|
|
ProtocolVersion::withStorageInterfaceReadiness(),
|
|
|
|
ProtocolVersion::withTSS())));
|
2022-04-05 09:42:52 +08:00
|
|
|
|
2022-04-05 10:31:09 +08:00
|
|
|
ErrorOr<SWVersion> swversion = wait(errorOr(
|
|
|
|
testSoftwareVersionCompatibility(swversionTestDirName, ProtocolVersion::withStorageInterfaceReadiness())));
|
|
|
|
|
|
|
|
if (!swversion.isError()) {
|
2022-04-08 07:26:05 +08:00
|
|
|
ASSERT(swversion.get().newestProtocolVersion() == ProtocolVersion::withStorageInterfaceReadiness().version());
|
|
|
|
ASSERT(swversion.get().lastRunProtocolVersion() == ProtocolVersion::withStorageInterfaceReadiness().version());
|
2022-04-05 10:31:09 +08:00
|
|
|
ASSERT(swversion.get().lowestCompatibleProtocolVersion() == ProtocolVersion::withTSS().version());
|
|
|
|
}
|
|
|
|
|
2022-04-09 06:03:39 +08:00
|
|
|
wait(IAsyncFileSystem::filesystem()->deleteFile(joinPath(swversionTestDirName, versionFileName), true));
|
2022-04-05 10:31:09 +08:00
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
TEST_CASE("/fdbserver/worker/swversion/runCompatibleOlder") {
|
|
|
|
if (!platform::createDirectory("sw-version-test")) {
|
2022-04-09 06:03:39 +08:00
|
|
|
TraceEvent(SevWarnAlways, "FailedToCreateDirectory").detail("Directory", "sw-version-test");
|
2022-04-05 10:31:09 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2022-10-05 07:01:02 +08:00
|
|
|
{
|
|
|
|
ErrorOr<Void> f = wait(errorOr(updateNewestSoftwareVersion(swversionTestDirName,
|
|
|
|
ProtocolVersion::withStorageInterfaceReadiness(),
|
|
|
|
ProtocolVersion::withStorageInterfaceReadiness(),
|
|
|
|
ProtocolVersion::withTSS())));
|
|
|
|
}
|
2022-04-05 10:31:09 +08:00
|
|
|
|
2022-10-05 07:01:02 +08:00
|
|
|
{
|
|
|
|
ErrorOr<SWVersion> swversion = wait(errorOr(
|
|
|
|
testSoftwareVersionCompatibility(swversionTestDirName, ProtocolVersion::withStorageInterfaceReadiness())));
|
2022-04-05 10:31:09 +08:00
|
|
|
|
2022-10-05 07:01:02 +08:00
|
|
|
if (!swversion.isError()) {
|
|
|
|
ASSERT(swversion.get().newestProtocolVersion() ==
|
|
|
|
ProtocolVersion::withStorageInterfaceReadiness().version());
|
|
|
|
ASSERT(swversion.get().lastRunProtocolVersion() ==
|
|
|
|
ProtocolVersion::withStorageInterfaceReadiness().version());
|
|
|
|
ASSERT(swversion.get().lowestCompatibleProtocolVersion() == ProtocolVersion::withTSS().version());
|
2022-04-07 12:06:01 +08:00
|
|
|
|
2022-10-05 07:01:02 +08:00
|
|
|
TraceEvent(SevInfo, "UT/swversion/runCompatibleOlder").detail("SWVersion", swversion.get());
|
|
|
|
}
|
2022-04-05 10:31:09 +08:00
|
|
|
}
|
|
|
|
|
2022-10-05 07:01:02 +08:00
|
|
|
{
|
|
|
|
ErrorOr<Void> f = wait(errorOr(updateNewestSoftwareVersion(swversionTestDirName,
|
|
|
|
ProtocolVersion::withTSS(),
|
|
|
|
ProtocolVersion::withStorageInterfaceReadiness(),
|
|
|
|
ProtocolVersion::withTSS())));
|
|
|
|
}
|
2022-04-05 10:31:09 +08:00
|
|
|
|
2022-10-05 07:01:02 +08:00
|
|
|
{
|
|
|
|
ErrorOr<SWVersion> swversion = wait(errorOr(
|
|
|
|
testSoftwareVersionCompatibility(swversionTestDirName, ProtocolVersion::withStorageInterfaceReadiness())));
|
|
|
|
|
|
|
|
if (!swversion.isError()) {
|
|
|
|
ASSERT(swversion.get().newestProtocolVersion() ==
|
|
|
|
ProtocolVersion::withStorageInterfaceReadiness().version());
|
|
|
|
ASSERT(swversion.get().lastRunProtocolVersion() == ProtocolVersion::withTSS().version());
|
|
|
|
ASSERT(swversion.get().lowestCompatibleProtocolVersion() == ProtocolVersion::withTSS().version());
|
|
|
|
}
|
2022-04-05 10:31:09 +08:00
|
|
|
}
|
|
|
|
|
2022-04-09 06:03:39 +08:00
|
|
|
wait(IAsyncFileSystem::filesystem()->deleteFile(joinPath(swversionTestDirName, versionFileName), true));
|
2022-04-05 10:31:09 +08:00
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
TEST_CASE("/fdbserver/worker/swversion/runIncompatibleOlder") {
|
|
|
|
if (!platform::createDirectory("sw-version-test")) {
|
2022-04-09 06:03:39 +08:00
|
|
|
TraceEvent(SevWarnAlways, "FailedToCreateDirectory").detail("Directory", "sw-version-test");
|
2022-04-05 10:31:09 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2022-10-05 07:01:02 +08:00
|
|
|
{
|
|
|
|
ErrorOr<Void> f = wait(errorOr(updateNewestSoftwareVersion(swversionTestDirName,
|
|
|
|
ProtocolVersion::withStorageInterfaceReadiness(),
|
|
|
|
ProtocolVersion::withStorageInterfaceReadiness(),
|
|
|
|
ProtocolVersion::withTSS())));
|
|
|
|
}
|
2022-04-05 10:31:09 +08:00
|
|
|
|
2022-10-05 07:01:02 +08:00
|
|
|
{
|
|
|
|
ErrorOr<SWVersion> swversion = wait(errorOr(
|
|
|
|
testSoftwareVersionCompatibility(swversionTestDirName, ProtocolVersion::withStorageInterfaceReadiness())));
|
|
|
|
|
|
|
|
if (!swversion.isError()) {
|
|
|
|
ASSERT(swversion.get().newestProtocolVersion() ==
|
|
|
|
ProtocolVersion::withStorageInterfaceReadiness().version());
|
|
|
|
ASSERT(swversion.get().lastRunProtocolVersion() ==
|
|
|
|
ProtocolVersion::withStorageInterfaceReadiness().version());
|
|
|
|
ASSERT(swversion.get().lowestCompatibleProtocolVersion() == ProtocolVersion::withTSS().version());
|
|
|
|
}
|
2022-04-05 10:31:09 +08:00
|
|
|
}
|
|
|
|
|
2022-10-05 07:01:02 +08:00
|
|
|
{
|
|
|
|
ErrorOr<SWVersion> swversion =
|
|
|
|
wait(errorOr(testSoftwareVersionCompatibility(swversionTestDirName, ProtocolVersion::withCacheRole())));
|
2022-04-05 10:31:09 +08:00
|
|
|
|
2022-10-05 07:01:02 +08:00
|
|
|
ASSERT(swversion.isError() && swversion.getError().code() == error_code_incompatible_software_version);
|
|
|
|
}
|
2022-04-05 10:31:09 +08:00
|
|
|
|
2022-04-09 06:03:39 +08:00
|
|
|
wait(IAsyncFileSystem::filesystem()->deleteFile(joinPath(swversionTestDirName, versionFileName), true));
|
2022-04-05 10:31:09 +08:00
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
TEST_CASE("/fdbserver/worker/swversion/runNewer") {
|
|
|
|
if (!platform::createDirectory("sw-version-test")) {
|
2022-04-09 06:03:39 +08:00
|
|
|
TraceEvent(SevWarnAlways, "FailedToCreateDirectory").detail("Directory", "sw-version-test");
|
2022-04-05 10:31:09 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2022-10-05 07:01:02 +08:00
|
|
|
{
|
|
|
|
ErrorOr<Void> f = wait(errorOr(updateNewestSoftwareVersion(swversionTestDirName,
|
|
|
|
ProtocolVersion::withTSS(),
|
|
|
|
ProtocolVersion::withTSS(),
|
|
|
|
ProtocolVersion::withCacheRole())));
|
|
|
|
}
|
2022-04-05 10:31:09 +08:00
|
|
|
|
2022-10-05 07:01:02 +08:00
|
|
|
{
|
|
|
|
ErrorOr<SWVersion> swversion = wait(errorOr(
|
|
|
|
testSoftwareVersionCompatibility(swversionTestDirName, ProtocolVersion::withStorageInterfaceReadiness())));
|
2022-04-05 10:31:09 +08:00
|
|
|
|
2022-10-05 07:01:02 +08:00
|
|
|
if (!swversion.isError()) {
|
|
|
|
ASSERT(swversion.get().newestProtocolVersion() == ProtocolVersion::withTSS().version());
|
|
|
|
ASSERT(swversion.get().lastRunProtocolVersion() == ProtocolVersion::withTSS().version());
|
|
|
|
ASSERT(swversion.get().lowestCompatibleProtocolVersion() == ProtocolVersion::withCacheRole().version());
|
|
|
|
}
|
2022-04-05 10:31:09 +08:00
|
|
|
}
|
|
|
|
|
2022-10-05 07:01:02 +08:00
|
|
|
{
|
|
|
|
ErrorOr<Void> f = wait(errorOr(updateNewestSoftwareVersion(swversionTestDirName,
|
|
|
|
ProtocolVersion::withStorageInterfaceReadiness(),
|
|
|
|
ProtocolVersion::withStorageInterfaceReadiness(),
|
|
|
|
ProtocolVersion::withTSS())));
|
|
|
|
}
|
2022-04-05 09:42:52 +08:00
|
|
|
|
2022-10-05 07:01:02 +08:00
|
|
|
{
|
|
|
|
ErrorOr<SWVersion> swversion = wait(errorOr(
|
|
|
|
testSoftwareVersionCompatibility(swversionTestDirName, ProtocolVersion::withStorageInterfaceReadiness())));
|
|
|
|
|
|
|
|
if (!swversion.isError()) {
|
|
|
|
ASSERT(swversion.get().newestProtocolVersion() ==
|
|
|
|
ProtocolVersion::withStorageInterfaceReadiness().version());
|
|
|
|
ASSERT(swversion.get().lastRunProtocolVersion() ==
|
|
|
|
ProtocolVersion::withStorageInterfaceReadiness().version());
|
|
|
|
ASSERT(swversion.get().lowestCompatibleProtocolVersion() == ProtocolVersion::withTSS().version());
|
|
|
|
}
|
2022-04-05 09:42:52 +08:00
|
|
|
}
|
|
|
|
|
2022-04-09 06:03:39 +08:00
|
|
|
wait(IAsyncFileSystem::filesystem()->deleteFile(joinPath(swversionTestDirName, versionFileName), true));
|
2022-04-02 06:45:24 +08:00
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2018-06-08 04:07:19 +08:00
|
|
|
ACTOR Future<UID> createAndLockProcessIdFile(std::string folder) {
|
|
|
|
state UID processIDUid;
|
|
|
|
platform::createDirectory(folder);
|
|
|
|
|
2020-02-20 07:21:42 +08:00
|
|
|
loop {
|
|
|
|
try {
|
|
|
|
state std::string lockFilePath = joinPath(folder, "processId");
|
|
|
|
state ErrorOr<Reference<IAsyncFile>> lockFile = wait(errorOr(IAsyncFileSystem::filesystem(g_network)->open(
|
|
|
|
lockFilePath, IAsyncFile::OPEN_READWRITE | IAsyncFile::OPEN_LOCK, 0600)));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-02-20 07:21:42 +08:00
|
|
|
if (lockFile.isError() && lockFile.getError().code() == error_code_file_not_found &&
|
|
|
|
!fileExists(lockFilePath)) {
|
|
|
|
Reference<IAsyncFile> _lockFile = wait(IAsyncFileSystem::filesystem()->open(
|
|
|
|
lockFilePath,
|
|
|
|
IAsyncFile::OPEN_ATOMIC_WRITE_AND_CREATE | IAsyncFile::OPEN_CREATE | IAsyncFile::OPEN_LOCK |
|
|
|
|
IAsyncFile::OPEN_READWRITE,
|
|
|
|
0600));
|
|
|
|
lockFile = _lockFile;
|
|
|
|
processIDUid = deterministicRandom()->randomUniqueID();
|
2020-05-23 08:16:59 +08:00
|
|
|
BinaryWriter wr(IncludeVersion(ProtocolVersion::withProcessIDFile()));
|
2020-02-20 07:21:42 +08:00
|
|
|
wr << processIDUid;
|
|
|
|
wait(lockFile.get()->write(wr.getData(), wr.getLength(), 0));
|
|
|
|
wait(lockFile.get()->sync());
|
|
|
|
} else {
|
|
|
|
if (lockFile.isError())
|
|
|
|
throw lockFile.getError(); // If we've failed to open the file, throw an exception
|
2018-06-08 04:07:19 +08:00
|
|
|
|
2020-02-20 07:21:42 +08:00
|
|
|
int64_t fileSize = wait(lockFile.get()->size());
|
|
|
|
state Key fileData = makeString(fileSize);
|
|
|
|
wait(success(lockFile.get()->read(mutateString(fileData), fileSize, 0)));
|
2020-02-21 15:04:39 +08:00
|
|
|
try {
|
|
|
|
processIDUid = BinaryReader::fromStringRef<UID>(fileData, IncludeVersion());
|
|
|
|
return processIDUid;
|
|
|
|
} catch (Error& e) {
|
|
|
|
if (!g_network->isSimulated()) {
|
|
|
|
throw;
|
|
|
|
}
|
2020-05-06 06:59:02 +08:00
|
|
|
lockFile = ErrorOr<Reference<IAsyncFile>>();
|
|
|
|
wait(IAsyncFileSystem::filesystem()->deleteFile(lockFilePath, true));
|
2020-02-21 15:04:39 +08:00
|
|
|
}
|
2020-02-20 07:21:42 +08:00
|
|
|
}
|
|
|
|
} catch (Error& e) {
|
|
|
|
if (e.code() == error_code_actor_cancelled) {
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
if (!e.isInjectedFault()) {
|
2018-06-08 04:07:19 +08:00
|
|
|
fprintf(stderr,
|
|
|
|
"ERROR: error creating or opening process id file `%s'.\n",
|
|
|
|
joinPath(folder, "processId").c_str());
|
2020-02-20 07:21:42 +08:00
|
|
|
}
|
2018-06-08 04:07:19 +08:00
|
|
|
TraceEvent(SevError, "OpenProcessIdError").error(e);
|
2020-02-21 15:04:39 +08:00
|
|
|
throw;
|
2018-06-08 04:07:19 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-09-17 05:22:44 +08:00
|
|
|
ACTOR Future<MonitorLeaderInfo> monitorLeaderWithDelayedCandidacyImplOneGeneration(
|
2021-10-11 11:44:56 +08:00
|
|
|
Reference<IClusterConnectionRecord> connRecord,
|
2021-09-17 05:22:44 +08:00
|
|
|
Reference<AsyncVar<Value>> result,
|
|
|
|
MonitorLeaderInfo info) {
|
2022-04-28 12:54:13 +08:00
|
|
|
ClusterConnectionString cs = info.intermediateConnRecord->getConnectionString();
|
2022-05-24 02:42:56 +08:00
|
|
|
state int coordinatorsSize = cs.hostnames.size() + cs.coords.size();
|
2020-05-02 05:40:21 +08:00
|
|
|
state ElectionResultRequest request;
|
|
|
|
state int index = 0;
|
2020-05-11 05:20:50 +08:00
|
|
|
state int successIndex = 0;
|
2022-04-28 12:54:13 +08:00
|
|
|
state std::vector<LeaderElectionRegInterface> leaderElectionServers;
|
2020-06-15 13:26:06 +08:00
|
|
|
|
2022-04-28 12:54:13 +08:00
|
|
|
leaderElectionServers.reserve(coordinatorsSize);
|
|
|
|
for (const auto& h : cs.hostnames) {
|
|
|
|
leaderElectionServers.push_back(LeaderElectionRegInterface(h));
|
|
|
|
}
|
2022-05-24 02:42:56 +08:00
|
|
|
for (const auto& c : cs.coords) {
|
2022-04-28 12:54:13 +08:00
|
|
|
leaderElectionServers.push_back(LeaderElectionRegInterface(c));
|
|
|
|
}
|
|
|
|
deterministicRandom()->randomShuffle(leaderElectionServers);
|
|
|
|
|
|
|
|
request.key = cs.clusterKey();
|
2022-04-29 09:32:55 +08:00
|
|
|
request.hostnames = cs.hostnames;
|
2022-05-24 02:42:56 +08:00
|
|
|
request.coordinators = cs.coords;
|
2020-05-02 05:40:21 +08:00
|
|
|
|
|
|
|
loop {
|
2022-04-28 12:54:13 +08:00
|
|
|
LeaderElectionRegInterface interf = leaderElectionServers[index];
|
|
|
|
bool usingHostname = interf.hostname.present();
|
2020-05-08 16:00:18 +08:00
|
|
|
request.reply = ReplyPromise<Optional<LeaderInfo>>();
|
2020-05-02 05:40:21 +08:00
|
|
|
|
2022-04-28 12:54:13 +08:00
|
|
|
state ErrorOr<Optional<LeaderInfo>> leader;
|
|
|
|
if (usingHostname) {
|
|
|
|
wait(store(
|
|
|
|
leader,
|
|
|
|
tryGetReplyFromHostname(request, interf.hostname.get(), WLTOKEN_LEADERELECTIONREG_ELECTIONRESULT)));
|
|
|
|
} else {
|
|
|
|
wait(store(leader, interf.electionResult.tryGetReply(request)));
|
|
|
|
}
|
|
|
|
|
2020-05-11 05:20:50 +08:00
|
|
|
if (leader.present()) {
|
|
|
|
if (leader.get().present()) {
|
|
|
|
if (leader.get().get().forward) {
|
2021-10-11 11:44:56 +08:00
|
|
|
info.intermediateConnRecord = connRecord->makeIntermediateRecord(
|
|
|
|
ClusterConnectionString(leader.get().get().serializedInfo.toString()));
|
2020-05-11 05:20:50 +08:00
|
|
|
return info;
|
|
|
|
}
|
2021-10-11 11:44:56 +08:00
|
|
|
if (connRecord != info.intermediateConnRecord) {
|
2020-05-11 05:20:50 +08:00
|
|
|
if (!info.hasConnected) {
|
|
|
|
TraceEvent(SevWarnAlways, "IncorrectClusterFileContentsAtConnection")
|
2021-10-11 11:44:56 +08:00
|
|
|
.detail("ClusterFile", connRecord->toString())
|
|
|
|
.detail("StoredConnectionString", connRecord->getConnectionString().toString())
|
2020-05-11 05:20:50 +08:00
|
|
|
.detail("CurrentConnectionString",
|
2021-10-11 11:44:56 +08:00
|
|
|
info.intermediateConnRecord->getConnectionString().toString());
|
2020-05-11 05:20:50 +08:00
|
|
|
}
|
2022-06-23 15:22:09 +08:00
|
|
|
wait(connRecord->setAndPersistConnectionString(info.intermediateConnRecord->getConnectionString()));
|
2021-10-11 11:44:56 +08:00
|
|
|
info.intermediateConnRecord = connRecord;
|
2020-05-11 05:20:50 +08:00
|
|
|
}
|
2020-05-02 05:40:21 +08:00
|
|
|
|
2020-05-11 05:20:50 +08:00
|
|
|
info.hasConnected = true;
|
2021-10-11 11:44:56 +08:00
|
|
|
connRecord->notifyConnected();
|
2020-05-11 05:20:50 +08:00
|
|
|
request.knownLeader = leader.get().get().changeID;
|
2020-05-02 05:40:21 +08:00
|
|
|
|
2020-05-11 05:20:50 +08:00
|
|
|
ClusterControllerPriorityInfo info = leader.get().get().getPriorityInfo();
|
|
|
|
if (leader.get().get().serializedInfo.size() && !info.isExcluded &&
|
|
|
|
(info.dcFitness == ClusterControllerPriorityInfo::FitnessPrimary ||
|
|
|
|
info.dcFitness == ClusterControllerPriorityInfo::FitnessPreferred ||
|
|
|
|
info.dcFitness == ClusterControllerPriorityInfo::FitnessUnknown)) {
|
|
|
|
result->set(leader.get().get().serializedInfo);
|
|
|
|
} else {
|
|
|
|
result->set(Value());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
successIndex = index;
|
|
|
|
} else {
|
2022-04-28 12:54:13 +08:00
|
|
|
index = (index + 1) % coordinatorsSize;
|
2020-05-11 05:20:50 +08:00
|
|
|
if (index == successIndex) {
|
|
|
|
wait(delay(CLIENT_KNOBS->COORDINATOR_RECONNECTION_DELAY));
|
|
|
|
}
|
2020-05-02 05:40:21 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-10-11 11:44:56 +08:00
|
|
|
ACTOR Future<Void> monitorLeaderWithDelayedCandidacyImplInternal(Reference<IClusterConnectionRecord> connRecord,
|
2021-09-17 05:22:44 +08:00
|
|
|
Reference<AsyncVar<Value>> outSerializedLeaderInfo) {
|
2021-10-11 11:44:56 +08:00
|
|
|
state MonitorLeaderInfo info(connRecord);
|
2020-05-02 05:40:21 +08:00
|
|
|
loop {
|
2022-04-28 12:54:13 +08:00
|
|
|
MonitorLeaderInfo _info =
|
|
|
|
wait(monitorLeaderWithDelayedCandidacyImplOneGeneration(connRecord, outSerializedLeaderInfo, info));
|
|
|
|
info = _info;
|
2020-05-02 05:40:21 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
template <class LeaderInterface>
|
2021-09-17 05:22:44 +08:00
|
|
|
Future<Void> monitorLeaderWithDelayedCandidacyImpl(
|
2021-10-11 11:44:56 +08:00
|
|
|
Reference<IClusterConnectionRecord> const& connRecord,
|
2021-09-17 05:22:44 +08:00
|
|
|
Reference<AsyncVar<Optional<LeaderInterface>>> const& outKnownLeader) {
|
2020-05-02 05:40:21 +08:00
|
|
|
LeaderDeserializer<LeaderInterface> deserializer;
|
2020-11-07 15:50:55 +08:00
|
|
|
auto serializedInfo = makeReference<AsyncVar<Value>>();
|
2021-10-11 11:44:56 +08:00
|
|
|
Future<Void> m = monitorLeaderWithDelayedCandidacyImplInternal(connRecord, serializedInfo);
|
2020-05-02 05:40:21 +08:00
|
|
|
return m || deserializer(serializedInfo, outKnownLeader);
|
|
|
|
}
|
|
|
|
|
2021-09-17 05:22:44 +08:00
|
|
|
ACTOR Future<Void> monitorLeaderWithDelayedCandidacy(
|
2021-10-11 11:44:56 +08:00
|
|
|
Reference<IClusterConnectionRecord> connRecord,
|
2020-05-11 05:20:50 +08:00
|
|
|
Reference<AsyncVar<Optional<ClusterControllerFullInterface>>> currentCC,
|
|
|
|
Reference<AsyncVar<ClusterControllerPriorityInfo>> asyncPriorityInfo,
|
|
|
|
Future<Void> recoveredDiskFiles,
|
|
|
|
LocalityData locality,
|
2021-05-19 01:47:16 +08:00
|
|
|
Reference<AsyncVar<ServerDBInfo>> dbInfo,
|
2022-10-14 08:30:40 +08:00
|
|
|
ConfigDBType configDBType,
|
|
|
|
Reference<AsyncVar<Optional<UID>>> clusterId) {
|
2021-10-11 11:44:56 +08:00
|
|
|
state Future<Void> monitor = monitorLeaderWithDelayedCandidacyImpl(connRecord, currentCC);
|
2020-05-11 05:20:50 +08:00
|
|
|
state Future<Void> timeout;
|
2020-05-05 16:00:17 +08:00
|
|
|
|
2020-05-11 05:20:50 +08:00
|
|
|
wait(recoveredDiskFiles);
|
|
|
|
|
|
|
|
loop {
|
|
|
|
if (currentCC->get().present() && dbInfo->get().clusterInterface == currentCC->get().get() &&
|
|
|
|
IFailureMonitor::failureMonitor()
|
|
|
|
.getState(currentCC->get().get().registerWorker.getEndpoint())
|
|
|
|
.isAvailable()) {
|
|
|
|
timeout = Future<Void>();
|
|
|
|
} else if (!timeout.isValid()) {
|
2020-06-11 00:59:56 +08:00
|
|
|
timeout =
|
|
|
|
delay(SERVER_KNOBS->MIN_DELAY_CC_WORST_FIT_CANDIDACY_SECONDS +
|
|
|
|
(deterministicRandom()->random01() * (SERVER_KNOBS->MAX_DELAY_CC_WORST_FIT_CANDIDACY_SECONDS -
|
|
|
|
SERVER_KNOBS->MIN_DELAY_CC_WORST_FIT_CANDIDACY_SECONDS)));
|
2020-05-05 16:00:17 +08:00
|
|
|
}
|
2020-05-11 05:20:50 +08:00
|
|
|
choose {
|
|
|
|
when(wait(currentCC->onChange())) {}
|
|
|
|
when(wait(dbInfo->onChange())) {}
|
|
|
|
when(wait(currentCC->get().present() ? IFailureMonitor::failureMonitor().onStateChanged(
|
|
|
|
currentCC->get().get().registerWorker.getEndpoint())
|
|
|
|
: Never())) {}
|
|
|
|
when(wait(timeout.isValid() ? timeout : Never())) {
|
|
|
|
monitor.cancel();
|
2021-05-19 01:47:16 +08:00
|
|
|
wait(clusterController(
|
2022-10-14 08:30:40 +08:00
|
|
|
connRecord, currentCC, asyncPriorityInfo, recoveredDiskFiles, locality, configDBType, clusterId));
|
2020-05-11 05:20:50 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
}
|
2020-05-05 16:00:17 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-06-04 06:10:04 +08:00
|
|
|
extern void setupStackSignal();
|
|
|
|
|
2020-09-19 09:32:32 +08:00
|
|
|
ACTOR Future<Void> serveProtocolInfo() {
|
|
|
|
state RequestStream<ProtocolInfoRequest> protocolInfo(
|
|
|
|
PeerCompatibilityPolicy{ RequirePeer::AtLeast, ProtocolVersion::withStableInterfaces() });
|
|
|
|
protocolInfo.makeWellKnownEndpoint(WLTOKEN_PROTOCOL_INFO, TaskPriority::DefaultEndpoint);
|
|
|
|
loop {
|
2020-10-21 10:30:16 +08:00
|
|
|
state ProtocolInfoRequest req = waitNext(protocolInfo.getFuture());
|
2020-09-29 01:58:49 +08:00
|
|
|
req.reply.send(ProtocolInfoReply{ g_network->protocolVersion() });
|
2020-09-19 09:32:32 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-06-04 06:10:04 +08:00
|
|
|
// Handles requests from ProcessInterface, an interface meant for direct
|
|
|
|
// communication between the client and FDB processes.
|
|
|
|
ACTOR Future<Void> serveProcess() {
|
|
|
|
state ProcessInterface process;
|
|
|
|
process.getInterface.makeWellKnownEndpoint(WLTOKEN_PROCESS, TaskPriority::DefaultEndpoint);
|
|
|
|
loop {
|
|
|
|
choose {
|
|
|
|
when(GetProcessInterfaceRequest req = waitNext(process.getInterface.getFuture())) {
|
|
|
|
req.reply.send(process);
|
|
|
|
}
|
|
|
|
when(ActorLineageRequest req = waitNext(process.actorLineage.getFuture())) {
|
|
|
|
state SampleCollection sampleCollector;
|
|
|
|
auto samples = sampleCollector->get(req.timeStart, req.timeEnd);
|
|
|
|
|
|
|
|
std::vector<SerializedSample> serializedSamples;
|
|
|
|
for (const auto& samplePtr : samples) {
|
|
|
|
auto serialized = SerializedSample{ .time = samplePtr->time };
|
|
|
|
for (const auto& [waitState, pair] : samplePtr->data) {
|
|
|
|
if (waitState >= req.waitStateStart && waitState <= req.waitStateEnd) {
|
|
|
|
serialized.data[waitState] = std::string(pair.first, pair.second);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
serializedSamples.push_back(std::move(serialized));
|
|
|
|
}
|
|
|
|
ActorLineageReply reply{ serializedSamples };
|
|
|
|
req.reply.send(reply);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-10-14 08:30:40 +08:00
|
|
|
Optional<UID> readClusterId(std::string filePath) {
|
|
|
|
if (!fileExists(filePath)) {
|
|
|
|
return Optional<UID>();
|
|
|
|
}
|
|
|
|
std::string contents(readFileBytes(filePath, 10000));
|
|
|
|
BinaryReader br(StringRef(contents), IncludeVersion());
|
|
|
|
UID clusterId;
|
|
|
|
br >> clusterId;
|
|
|
|
return clusterId;
|
|
|
|
}
|
|
|
|
|
2021-10-11 11:44:56 +08:00
|
|
|
ACTOR Future<Void> fdbd(Reference<IClusterConnectionRecord> connRecord,
|
2017-05-26 04:48:44 +08:00
|
|
|
LocalityData localities,
|
|
|
|
ProcessClass processClass,
|
|
|
|
std::string dataFolder,
|
|
|
|
std::string coordFolder,
|
|
|
|
int64_t memoryLimit,
|
|
|
|
std::string metricsConnFile,
|
2019-04-06 07:06:30 +08:00
|
|
|
std::string metricsPrefix,
|
2019-04-03 20:27:11 +08:00
|
|
|
int64_t memoryProfileThreshold,
|
2021-05-12 09:23:33 +08:00
|
|
|
std::string whitelistBinPaths,
|
2021-05-18 10:14:32 +08:00
|
|
|
std::string configPath,
|
|
|
|
std::map<std::string, std::string> manualKnobOverrides,
|
2021-08-07 14:18:10 +08:00
|
|
|
ConfigDBType configDBType) {
|
2021-09-17 08:42:34 +08:00
|
|
|
state std::vector<Future<Void>> actors;
|
2019-08-23 05:21:13 +08:00
|
|
|
state Promise<Void> recoveredDiskFiles;
|
2022-02-02 14:27:12 +08:00
|
|
|
state Reference<ConfigNode> configNode;
|
2022-07-30 08:28:34 +08:00
|
|
|
state Reference<LocalConfiguration> localConfig;
|
|
|
|
if (configDBType != ConfigDBType::DISABLED) {
|
|
|
|
localConfig = makeReference<LocalConfiguration>(
|
2022-09-13 13:31:13 +08:00
|
|
|
dataFolder, configPath, manualKnobOverrides, IsTest(g_network->isSimulated()));
|
2022-07-30 08:28:34 +08:00
|
|
|
}
|
2021-07-28 01:07:18 +08:00
|
|
|
// setupStackSignal();
|
2021-05-28 02:25:00 +08:00
|
|
|
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::Worker;
|
2021-05-19 01:47:16 +08:00
|
|
|
|
2022-02-02 14:27:12 +08:00
|
|
|
if (configDBType != ConfigDBType::DISABLED) {
|
|
|
|
configNode = makeReference<ConfigNode>(dataFolder);
|
|
|
|
}
|
|
|
|
|
2020-09-19 09:32:32 +08:00
|
|
|
actors.push_back(serveProtocolInfo());
|
2021-06-04 06:10:04 +08:00
|
|
|
actors.push_back(serveProcess());
|
2020-09-19 09:32:32 +08:00
|
|
|
|
2019-08-23 02:02:14 +08:00
|
|
|
try {
|
2022-07-28 05:02:01 +08:00
|
|
|
ServerCoordinators coordinators(connRecord, configDBType);
|
2019-04-03 20:27:11 +08:00
|
|
|
if (g_network->isSimulated()) {
|
2019-04-21 03:58:24 +08:00
|
|
|
whitelistBinPaths = ",, random_path, /bin/snap_create.sh,,";
|
2019-04-03 20:27:11 +08:00
|
|
|
}
|
2019-04-13 04:23:02 +08:00
|
|
|
TraceEvent("StartingFDBD")
|
|
|
|
.detail("ZoneID", localities.zoneId())
|
|
|
|
.detail("MachineId", localities.machineId())
|
|
|
|
.detail("DiskPath", dataFolder)
|
|
|
|
.detail("CoordPath", coordFolder)
|
2022-07-30 08:28:34 +08:00
|
|
|
.detail("WhiteListBinPath", whitelistBinPaths)
|
|
|
|
.detail("ConfigDBType", configDBType);
|
2017-05-27 08:43:28 +08:00
|
|
|
|
2022-02-02 14:27:12 +08:00
|
|
|
state ConfigBroadcastInterface configBroadcastInterface;
|
2017-05-27 08:43:28 +08:00
|
|
|
// SOMEDAY: start the services on the machine in a staggered fashion in simulation?
|
2020-02-19 08:41:19 +08:00
|
|
|
// Endpoints should be registered first before any process trying to connect to it.
|
|
|
|
// So coordinationServer actor should be the first one executed before any other.
|
2020-02-19 08:41:59 +08:00
|
|
|
if (coordFolder.size()) {
|
|
|
|
// SOMEDAY: remove the fileNotFound wrapper and make DiskQueue construction safe from errors setting up
|
|
|
|
// their files
|
2022-03-25 10:20:42 +08:00
|
|
|
actors.push_back(
|
|
|
|
fileNotFoundToNever(coordinationServer(coordFolder, connRecord, configNode, configBroadcastInterface)));
|
2020-02-19 08:41:19 +08:00
|
|
|
}
|
2020-02-01 12:23:35 +08:00
|
|
|
|
2018-06-08 04:07:19 +08:00
|
|
|
state UID processIDUid = wait(createAndLockProcessIdFile(dataFolder));
|
2018-09-29 03:12:06 +08:00
|
|
|
localities.set(LocalityData::keyProcessId, processIDUid.toString());
|
2018-06-08 04:07:19 +08:00
|
|
|
// Only one process can execute on a dataFolder from this point onwards
|
|
|
|
|
2022-04-13 01:22:35 +08:00
|
|
|
wait(testAndUpdateSoftwareVersionCompatibility(dataFolder, processIDUid));
|
2022-03-29 07:49:17 +08:00
|
|
|
|
2022-07-28 05:02:01 +08:00
|
|
|
if (configDBType != ConfigDBType::DISABLED) {
|
|
|
|
wait(localConfig->initialize());
|
|
|
|
}
|
|
|
|
|
2018-06-09 02:51:51 +08:00
|
|
|
std::string fitnessFilePath = joinPath(dataFolder, "fitness");
|
2020-11-07 15:50:55 +08:00
|
|
|
auto cc = makeReference<AsyncVar<Optional<ClusterControllerFullInterface>>>();
|
|
|
|
auto ci = makeReference<AsyncVar<Optional<ClusterInterface>>>();
|
|
|
|
auto asyncPriorityInfo =
|
|
|
|
makeReference<AsyncVar<ClusterControllerPriorityInfo>>(getCCPriorityInfo(fitnessFilePath, processClass));
|
2022-08-02 06:19:21 +08:00
|
|
|
auto serverDBInfo = ServerDBInfo();
|
|
|
|
serverDBInfo.client.isEncryptionEnabled = SERVER_KNOBS->ENABLE_ENCRYPTION;
|
2022-08-26 04:47:51 +08:00
|
|
|
serverDBInfo.myLocality = localities;
|
2022-08-02 06:19:21 +08:00
|
|
|
auto dbInfo = makeReference<AsyncVar<ServerDBInfo>>(serverDBInfo);
|
2022-10-14 08:30:40 +08:00
|
|
|
Reference<AsyncVar<Optional<UID>>> clusterId(
|
|
|
|
new AsyncVar<Optional<UID>>(readClusterId(joinPath(dataFolder, clusterIdFilename))));
|
2022-08-26 05:11:37 +08:00
|
|
|
TraceEvent("MyLocality").detail("Locality", dbInfo->get().myLocality.toString());
|
2018-06-08 04:07:19 +08:00
|
|
|
|
2019-08-28 01:45:09 +08:00
|
|
|
actors.push_back(reportErrors(monitorAndWriteCCPriorityInfo(fitnessFilePath, asyncPriorityInfo),
|
|
|
|
"MonitorAndWriteCCPriorityInfo"));
|
2020-06-06 07:27:04 +08:00
|
|
|
if (processClass.machineClassFitness(ProcessClass::ClusterController) == ProcessClass::NeverAssign) {
|
2021-10-11 11:44:56 +08:00
|
|
|
actors.push_back(reportErrors(monitorLeader(connRecord, cc), "ClusterController"));
|
2020-06-11 00:59:56 +08:00
|
|
|
} else if (processClass.machineClassFitness(ProcessClass::ClusterController) == ProcessClass::WorstFit &&
|
|
|
|
SERVER_KNOBS->MAX_DELAY_CC_WORST_FIT_CANDIDACY_SECONDS > 0) {
|
2021-10-11 11:44:56 +08:00
|
|
|
actors.push_back(reportErrors(monitorLeaderWithDelayedCandidacy(connRecord,
|
|
|
|
cc,
|
|
|
|
asyncPriorityInfo,
|
|
|
|
recoveredDiskFiles.getFuture(),
|
|
|
|
localities,
|
|
|
|
dbInfo,
|
2022-10-14 08:30:40 +08:00
|
|
|
configDBType,
|
|
|
|
clusterId),
|
2021-10-11 11:44:56 +08:00
|
|
|
"ClusterController"));
|
2020-05-02 05:40:21 +08:00
|
|
|
} else {
|
2022-10-14 08:30:40 +08:00
|
|
|
actors.push_back(reportErrors(clusterController(connRecord,
|
|
|
|
cc,
|
|
|
|
asyncPriorityInfo,
|
|
|
|
recoveredDiskFiles.getFuture(),
|
|
|
|
localities,
|
|
|
|
configDBType,
|
|
|
|
clusterId),
|
|
|
|
"ClusterController"));
|
2020-05-02 05:40:21 +08:00
|
|
|
}
|
2019-08-28 01:45:09 +08:00
|
|
|
actors.push_back(reportErrors(extractClusterInterface(cc, ci), "ExtractClusterInterface"));
|
2021-10-11 11:44:56 +08:00
|
|
|
actors.push_back(reportErrorsExcept(workerServer(connRecord,
|
2020-05-11 05:20:50 +08:00
|
|
|
cc,
|
|
|
|
localities,
|
|
|
|
asyncPriorityInfo,
|
|
|
|
processClass,
|
|
|
|
dataFolder,
|
|
|
|
memoryLimit,
|
|
|
|
metricsConnFile,
|
|
|
|
metricsPrefix,
|
|
|
|
recoveredDiskFiles,
|
|
|
|
memoryProfileThreshold,
|
|
|
|
coordFolder,
|
|
|
|
whitelistBinPaths,
|
2021-08-06 09:50:11 +08:00
|
|
|
dbInfo,
|
2022-02-02 14:27:12 +08:00
|
|
|
configBroadcastInterface,
|
|
|
|
configNode,
|
2022-10-14 08:30:40 +08:00
|
|
|
localConfig,
|
|
|
|
clusterId),
|
2020-05-11 05:20:50 +08:00
|
|
|
"WorkerServer",
|
|
|
|
UID(),
|
|
|
|
&normalWorkerErrors()));
|
2017-05-27 08:43:28 +08:00
|
|
|
state Future<Void> firstConnect = reportErrors(printOnFirstConnected(ci), "ClusterFirstConnectedError");
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2019-08-28 01:45:09 +08:00
|
|
|
wait(quorum(actors, 1));
|
2017-05-27 08:43:28 +08:00
|
|
|
ASSERT(false); // None of these actors should terminate normally
|
|
|
|
throw internal_error();
|
2019-08-23 02:02:14 +08:00
|
|
|
} catch (Error& e) {
|
2019-08-28 01:45:09 +08:00
|
|
|
// Make sure actors are cancelled before recoveredDiskFiles is destructed.
|
|
|
|
// Otherwise, these actors may get a broken promise error.
|
|
|
|
for (auto f : actors)
|
|
|
|
f.cancel();
|
2022-10-21 05:41:30 +08:00
|
|
|
Error err = checkIOTimeout(e);
|
2017-05-27 08:43:28 +08:00
|
|
|
throw err;
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2018-09-06 06:06:14 +08:00
|
|
|
|
2018-09-06 06:53:12 +08:00
|
|
|
const Role Role::WORKER("Worker", "WK", false);
|
2018-09-06 06:06:14 +08:00
|
|
|
const Role Role::STORAGE_SERVER("StorageServer", "SS");
|
2021-05-26 04:06:32 +08:00
|
|
|
const Role Role::TESTING_STORAGE_SERVER("TestingStorageServer", "ST");
|
2018-09-06 06:06:14 +08:00
|
|
|
const Role Role::TRANSACTION_LOG("TLog", "TL");
|
2018-09-06 06:53:12 +08:00
|
|
|
const Role Role::SHARED_TRANSACTION_LOG("SharedTLog", "SL", false);
|
2020-09-11 08:44:15 +08:00
|
|
|
const Role Role::COMMIT_PROXY("CommitProxyServer", "CP");
|
2020-07-15 15:37:41 +08:00
|
|
|
const Role Role::GRV_PROXY("GrvProxyServer", "GP");
|
2018-09-06 06:06:14 +08:00
|
|
|
const Role Role::MASTER("MasterServer", "MS");
|
|
|
|
const Role Role::RESOLVER("Resolver", "RV");
|
|
|
|
const Role Role::CLUSTER_CONTROLLER("ClusterController", "CC");
|
|
|
|
const Role Role::TESTER("Tester", "TS");
|
|
|
|
const Role Role::LOG_ROUTER("LogRouter", "LR");
|
2018-12-14 05:31:37 +08:00
|
|
|
const Role Role::DATA_DISTRIBUTOR("DataDistributor", "DD");
|
2019-03-27 23:15:19 +08:00
|
|
|
const Role Role::RATEKEEPER("Ratekeeper", "RK");
|
2021-09-15 23:35:58 +08:00
|
|
|
const Role Role::BLOB_MANAGER("BlobManager", "BM");
|
2021-09-19 05:35:56 +08:00
|
|
|
const Role Role::BLOB_WORKER("BlobWorker", "BW");
|
2022-09-03 10:21:52 +08:00
|
|
|
const Role Role::BLOB_MIGRATOR("BlobMigrator", "MG");
|
2019-11-13 05:01:29 +08:00
|
|
|
const Role Role::STORAGE_CACHE("StorageCache", "SC");
|
2019-07-04 02:09:36 +08:00
|
|
|
const Role Role::COORDINATOR("Coordinator", "CD");
|
2019-04-25 06:12:37 +08:00
|
|
|
const Role Role::BACKUP("Backup", "BK");
|
2022-01-14 03:11:01 +08:00
|
|
|
const Role Role::ENCRYPT_KEY_PROXY("EncryptKeyProxy", "EP");
|
2022-09-17 00:03:06 +08:00
|
|
|
const Role Role::CONSISTENCYSCAN("ConsistencyScan", "CS");
|