Merge branch 'main' into expand-optional-erroror-map

This commit is contained in:
A.J. Beamon 2023-01-04 15:46:23 -08:00
commit 62604bcc28
103 changed files with 4127 additions and 1749 deletions

View File

@ -95,6 +95,8 @@ endif()
# The tests don't build on windows
if(NOT WIN32)
set(MAKO_SRCS
test/mako/admin_server.hpp
test/mako/admin_server.cpp
test/mako/async.hpp
test/mako/async.cpp
test/mako/blob_granules.hpp
@ -109,6 +111,8 @@ if(NOT WIN32)
test/mako/process.hpp
test/mako/shm.hpp
test/mako/stats.hpp
test/mako/tenant.cpp
test/mako/tenant.hpp
test/mako/time.hpp
test/mako/utils.cpp
test/mako/utils.hpp)
@ -218,7 +222,7 @@ if(NOT WIN32)
# do not set RPATH for mako
set_property(TARGET mako PROPERTY SKIP_BUILD_RPATH TRUE)
target_link_libraries(mako PRIVATE fdb_c fdbclient fmt::fmt Threads::Threads fdb_cpp boost_target rapidjson)
target_link_libraries(mako PRIVATE fdb_c tokensign fdbclient fmt::fmt Threads::Threads fdb_cpp boost_target rapidjson)
if(NOT OPEN_FOR_IDE)
# Make sure that fdb_c.h is compatible with c90
@ -287,27 +291,27 @@ if(NOT WIN32)
foreach(test_file ${API_TEST_FILES})
get_filename_component(file_name "${test_file}" NAME_WE)
set(test_name "fdb_c_api_test_${file_name}")
add_scripted_fdb_test(NAME "${test_name}"
TIMEOUT 300
COMMAND ${CMAKE_SOURCE_DIR}/bindings/c/test/apitester/run_c_api_tests.py
add_python_venv_test(NAME "${test_name}"
COMMAND python ${CMAKE_SOURCE_DIR}/bindings/c/test/apitester/run_c_api_tests.py
--build-dir ${CMAKE_BINARY_DIR}
--api-tester-bin $<TARGET_FILE:fdb_c_api_tester>
--external-client-library ${CMAKE_CURRENT_BINARY_DIR}/libfdb_c_external.so
--test-file ${test_file}
--retain-client-lib-copies
TEST_TIMEOUT 300
)
endforeach()
add_scripted_fdb_test(NAME fdb_c_upgrade_to_future_version
COMMAND ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
add_python_venv_test(NAME fdb_c_upgrade_to_future_version
COMMAND python ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
--build-dir ${CMAKE_BINARY_DIR}
--test-file ${CMAKE_SOURCE_DIR}/bindings/c/test/apitester/tests/upgrade/MixedApiWorkloadMultiThr.toml
--upgrade-path "${FDB_CURRENT_VERSION}" "${FDB_FUTURE_VERSION}" "${FDB_CURRENT_VERSION}"
--process-number 3
)
add_scripted_fdb_test(NAME fdb_c_upgrade_to_future_version_blob_granules
COMMAND ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
add_python_venv_test(NAME fdb_c_upgrade_to_future_version_blob_granules
COMMAND python ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
--build-dir ${CMAKE_BINARY_DIR}
--test-file ${CMAKE_SOURCE_DIR}/bindings/c/test/apitester/tests/upgrade/ApiBlobGranulesCorrectness.toml
--upgrade-path "${FDB_CURRENT_VERSION}" "${FDB_FUTURE_VERSION}" "${FDB_CURRENT_VERSION}"
@ -316,54 +320,54 @@ if(NOT WIN32)
)
if(CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" AND NOT USE_SANITIZER)
add_scripted_fdb_test(NAME fdb_c_client_config_tests
COMMAND $<TARGET_FILE:Python3::Interpreter> ${CMAKE_CURRENT_SOURCE_DIR}/test/fdb_c_client_config_tests.py
add_python_venv_test(NAME fdb_c_client_config_tests
COMMAND python ${CMAKE_CURRENT_SOURCE_DIR}/test/fdb_c_client_config_tests.py
--build-dir ${CMAKE_BINARY_DIR}
--client-config-tester-bin $<TARGET_FILE:fdb_c_client_config_tester>
)
add_scripted_fdb_test(NAME fdb_c_upgrade_from_prev3_gradual
COMMAND ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
add_python_venv_test(NAME fdb_c_upgrade_from_prev3_gradual
COMMAND python ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
--build-dir ${CMAKE_BINARY_DIR}
--test-file ${CMAKE_SOURCE_DIR}/bindings/c/test/apitester/tests/upgrade/MixedApiWorkloadMultiThr.toml
--upgrade-path "${FDB_PREV3_RELEASE_VERSION}" "${FDB_PREV2_RELEASE_VERSION}" "${FDB_PREV_RELEASE_VERSION}" "${FDB_CURRENT_VERSION}" "${FDB_PREV_RELEASE_VERSION}"
--process-number 3
)
add_scripted_fdb_test(NAME fdb_c_upgrade_from_prev3_direct
COMMAND ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
add_python_venv_test(NAME fdb_c_upgrade_from_prev3_direct
COMMAND python ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
--build-dir ${CMAKE_BINARY_DIR}
--test-file ${CMAKE_SOURCE_DIR}/bindings/c/test/apitester/tests/upgrade/MixedApiWorkloadMultiThr.toml
--upgrade-path "${FDB_PREV3_RELEASE_VERSION}" "${FDB_CURRENT_VERSION}" "${FDB_PREV_RELEASE_VERSION}"
--process-number 3
)
add_scripted_fdb_test(NAME fdb_c_upgrade_from_prev2_gradual
COMMAND ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
add_python_venv_test(NAME fdb_c_upgrade_from_prev2_gradual
COMMAND python ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
--build-dir ${CMAKE_BINARY_DIR}
--test-file ${CMAKE_SOURCE_DIR}/bindings/c/test/apitester/tests/upgrade/MixedApiWorkloadMultiThr.toml
--upgrade-path "${FDB_PREV2_RELEASE_VERSION}" "${FDB_PREV_RELEASE_VERSION}" "${FDB_CURRENT_VERSION}" "${FDB_PREV_RELEASE_VERSION}"
--process-number 3
)
add_scripted_fdb_test(NAME fdb_c_upgrade_from_prev2_direct
COMMAND ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
add_python_venv_test(NAME fdb_c_upgrade_from_prev2_direct
COMMAND python ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
--build-dir ${CMAKE_BINARY_DIR}
--test-file ${CMAKE_SOURCE_DIR}/bindings/c/test/apitester/tests/upgrade/MixedApiWorkloadMultiThr.toml
--upgrade-path "${FDB_PREV2_RELEASE_VERSION}" "${FDB_CURRENT_VERSION}" "${FDB_PREV_RELEASE_VERSION}"
--process-number 3
)
add_scripted_fdb_test(NAME fdb_c_upgrade_from_prev
COMMAND ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
add_python_venv_test(NAME fdb_c_upgrade_from_prev
COMMAND python ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
--build-dir ${CMAKE_BINARY_DIR}
--test-file ${CMAKE_SOURCE_DIR}/bindings/c/test/apitester/tests/upgrade/MixedApiWorkloadMultiThr.toml
--upgrade-path "${FDB_PREV_RELEASE_VERSION}" "${FDB_CURRENT_VERSION}" "${FDB_PREV_RELEASE_VERSION}"
--process-number 3
)
add_scripted_fdb_test(NAME fdb_c_wiggle_only
COMMAND ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
add_python_venv_test(NAME fdb_c_wiggle_only
COMMAND python ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
--build-dir ${CMAKE_BINARY_DIR}
--test-file ${CMAKE_SOURCE_DIR}/bindings/c/test/apitester/tests/upgrade/MixedApiWorkloadMultiThr.toml
--upgrade-path "${FDB_CURRENT_VERSION}" "wiggle"
@ -372,8 +376,8 @@ if(NOT WIN32)
--redundancy double
)
add_scripted_fdb_test(NAME fdb_c_wiggle_and_upgrade
COMMAND ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
add_python_venv_test(NAME fdb_c_wiggle_and_upgrade
COMMAND python ${CMAKE_SOURCE_DIR}/tests/TestRunner/upgrade_test.py
--build-dir ${CMAKE_BINARY_DIR}
--test-file ${CMAKE_SOURCE_DIR}/bindings/c/test/apitester/tests/upgrade/MixedApiWorkloadMultiThr.toml
--upgrade-path "${FDB_PREV_RELEASE_VERSION}" "wiggle" "${FDB_CURRENT_VERSION}"
@ -463,8 +467,8 @@ elseif(NOT WIN32 AND NOT APPLE AND NOT USE_SANITIZER) # Linux Only, non-santizer
target_link_libraries(fdb_c_shim_lib_tester PRIVATE fdb_c_shim SimpleOpt fdb_cpp Threads::Threads)
target_include_directories(fdb_c_shim_lib_tester PUBLIC ${CMAKE_CURRENT_SOURCE_DIR} ${CMAKE_CURRENT_BINARY_DIR}/foundationdb/ ${CMAKE_SOURCE_DIR}/flow/include)
add_scripted_fdb_test(NAME fdb_c_shim_library_tests
COMMAND $<TARGET_FILE:Python3::Interpreter> ${CMAKE_CURRENT_SOURCE_DIR}/test/fdb_c_shim_tests.py
add_python_venv_test(NAME fdb_c_shim_library_tests
COMMAND python ${CMAKE_CURRENT_SOURCE_DIR}/test/fdb_c_shim_tests.py
--build-dir ${CMAKE_BINARY_DIR}
--unit-tests-bin $<TARGET_FILE:fdb_c_shim_unit_tests>
--api-tester-bin $<TARGET_FILE:fdb_c_shim_api_tester>

View File

@ -110,18 +110,18 @@ inline int intSize(BytesRef b) {
}
template <template <class...> class StringLike, class Char>
ByteString strinc(const StringLike<Char>& s) {
int index;
StringLike<Char> strinc(const StringLike<Char>& s) {
int index = -1;
for (index = s.size() - 1; index >= 0; index--)
if (s[index] != 255)
if (static_cast<uint8_t>(s[index]) != 255)
break;
// Must not be called with a string that consists only of zero or more '\xff' bytes.
// Must not be called with a string that is empty or only consists of '\xff' bytes.
assert(index >= 0);
ByteString byteResult(s.substr(0, index + 1));
byteResult[byteResult.size() - 1]++;
return byteResult;
auto ret = s.substr(0, index + 1);
ret.back()++;
return ret;
}
class Error {

View File

@ -7,11 +7,13 @@ import sys
import os
import glob
import unittest
from threading import Thread
import time
from fdb_version import CURRENT_VERSION, PREV_RELEASE_VERSION, PREV2_RELEASE_VERSION
from binary_download import FdbBinaryDownloader
from local_cluster import LocalCluster, PortProvider, random_secret_string
from local_cluster import LocalCluster, PortProvider
from test_util import random_alphanum_string
args = None
downloader = None
@ -38,7 +40,7 @@ class TestCluster(LocalCluster):
self.build_dir = Path(args.build_dir).resolve()
assert self.build_dir.exists(), "{} does not exist".format(args.build_dir)
assert self.build_dir.is_dir(), "{} is not a directory".format(args.build_dir)
self.tmp_dir = self.build_dir.joinpath("tmp", random_secret_string(16))
self.tmp_dir = self.build_dir.joinpath("tmp", random_alphanum_string(16))
print("Creating temp dir {}".format(self.tmp_dir), file=sys.stderr)
self.tmp_dir.mkdir(parents=True)
self.version = version
@ -77,7 +79,7 @@ class ClientConfigTest:
self.cluster = tc.cluster
self.external_lib_dir = None
self.external_lib_path = None
self.test_dir = self.cluster.tmp_dir.joinpath(random_secret_string(16))
self.test_dir = self.cluster.tmp_dir.joinpath(random_alphanum_string(16))
self.test_dir.mkdir(parents=True)
self.log_dir = self.test_dir.joinpath("log")
self.log_dir.mkdir(parents=True)
@ -110,13 +112,13 @@ class ClientConfigTest:
self.tc.assertTrue(self.external_lib_path.exists(), "{} does not exist".format(self.external_lib_path))
def create_cluster_file_with_wrong_port(self):
self.test_cluster_file = self.test_dir.joinpath("{}.cluster".format(random_secret_string(16)))
self.test_cluster_file = self.test_dir.joinpath("{}.cluster".format(random_alphanum_string(16)))
port = self.cluster.port_provider.get_free_port()
with open(self.test_cluster_file, "w") as file:
file.write("abcde:fghijk@127.0.0.1:{}".format(port))
def create_invalid_cluster_file(self):
self.test_cluster_file = self.test_dir.joinpath("{}.cluster".format(random_secret_string(16)))
self.test_cluster_file = self.test_dir.joinpath("{}.cluster".format(random_alphanum_string(16)))
port = self.cluster.port_provider.get_free_port()
with open(self.test_cluster_file, "w") as file:
file.write("abcde:fghijk@")

View File

@ -6,8 +6,10 @@ import shutil
import subprocess
import sys
import os
from binary_download import FdbBinaryDownloader
from local_cluster import LocalCluster, random_secret_string
from local_cluster import LocalCluster
from test_util import random_alphanum_string
from fdb_version import CURRENT_VERSION, PREV_RELEASE_VERSION
TESTER_STATS_INTERVAL_SEC = 5
@ -40,7 +42,7 @@ class TestEnv(LocalCluster):
self.build_dir = Path(build_dir).resolve()
assert self.build_dir.exists(), "{} does not exist".format(build_dir)
assert self.build_dir.is_dir(), "{} is not a directory".format(build_dir)
self.tmp_dir = self.build_dir.joinpath("tmp", random_secret_string(16))
self.tmp_dir = self.build_dir.joinpath("tmp", random_alphanum_string(16))
self.tmp_dir.mkdir(parents=True)
self.downloader = downloader
self.version = version

View File

@ -0,0 +1,319 @@
/*
* admin_server.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "admin_server.hpp"
#include "future.hpp"
#include "logger.hpp"
#include "tenant.hpp"
#include "utils.hpp"
#include <map>
#include <cerrno>
#include <cstring> // strerror
#include <optional>
#include <sstream>
#include <stdexcept>
#include <thread>
#include <unistd.h>
#include <sys/wait.h>
#include <boost/archive/binary_iarchive.hpp>
#include <boost/archive/binary_oarchive.hpp>
#include <boost/serialization/optional.hpp>
#include <boost/serialization/string.hpp>
#include <boost/serialization/variant.hpp>
#include "rapidjson/document.h"
extern thread_local mako::Logger logr;
using oarchive = boost::archive::binary_oarchive;
using iarchive = boost::archive::binary_iarchive;
namespace {
template <class T>
void sendObject(boost::process::pstream& pipe, T obj) {
oarchive oa(pipe);
oa << obj;
}
template <class T>
T receiveObject(boost::process::pstream& pipe) {
iarchive ia(pipe);
T obj;
ia >> obj;
return obj;
}
fdb::Database getOrCreateDatabase(std::map<std::string, fdb::Database>& db_map, const std::string& cluster_file) {
auto iter = db_map.find(cluster_file);
if (iter == db_map.end()) {
auto [inserted_iter, _] = db_map.insert({ cluster_file, fdb::Database(cluster_file) });
return inserted_iter->second;
} else {
return iter->second;
}
}
} // anonymous namespace
namespace mako::ipc {
void AdminServer::start() {
assert(server_pid == -1);
assert(logr.isFor(ProcKind::MAIN));
logr.info("forking admin server process");
auto pid = fork();
if (pid == 0) {
// Update thread-local logger to modify log line header
logr = Logger(AdminProcess{}, args.verbose);
} else if (pid > 0) {
server_pid = pid;
return;
} else {
logr.error("Failed to fork admin server process: {}", std::strerror(errno));
throw std::runtime_error("fork error");
}
assert(logr.isFor(ProcKind::ADMIN));
logr.info("starting admin server process");
boost::optional<std::string> setup_error;
if (args.setGlobalOptions() < 0) {
setup_error = std::string("global option setup failed");
} else if (auto err = fdb::network::setupNothrow()) {
setup_error = std::string(err.what());
}
std::map<std::string, fdb::Database> databases;
std::optional<std::thread> network_thread;
if (!setup_error) {
network_thread.emplace([parent_logr = logr]() {
logr = parent_logr;
logr.debug("network thread started");
if (auto err = fdb::network::run()) {
logr.error("fdb::network::run(): {}", err.what());
}
});
}
auto network_thread_guard = ExitGuard([&network_thread]() {
if (network_thread.has_value()) {
logr.debug("fdb::network::stop()");
if (auto err = fdb::network::stop()) {
logr.error("network::stop(): {}", err.what());
}
logr.debug("waiting for network thread to join");
network_thread.value().join();
}
});
while (true) {
try {
auto req = receiveObject<Request>(pipe_to_server);
if (setup_error) {
sendObject(pipe_to_client, Response{ setup_error });
} else if (boost::get<PingRequest>(&req)) {
sendObject(pipe_to_client, Response{});
} else if (boost::get<StopRequest>(&req)) {
logr.info("server was requested to stop");
sendObject(pipe_to_client, Response{});
return;
} else if (auto p = boost::get<BatchCreateTenantRequest>(&req)) {
auto err_msg = createTenant(getOrCreateDatabase(databases, p->cluster_file), p->id_begin, p->id_end);
sendObject(pipe_to_client, Response{ std::move(err_msg) });
} else if (auto p = boost::get<BatchDeleteTenantRequest>(&req)) {
auto err_msg = deleteTenant(getOrCreateDatabase(databases, p->cluster_file), p->id_begin, p->id_end);
sendObject(pipe_to_client, Response{ std::move(err_msg) });
} else {
sendObject(pipe_to_client, Response{ std::string("unknown request type") });
}
} catch (const std::exception& e) {
logr.error("fatal exception: {}", e.what());
return;
}
}
}
boost::optional<std::string> AdminServer::createTenant(fdb::Database db, int id_begin, int id_end) {
try {
auto tx = db.createTransaction();
while (true) {
for (auto id = id_begin; id < id_end; id++) {
auto tenant_name = getTenantNameByIndex(id);
fdb::Tenant::createTenant(tx, fdb::toBytesRef(tenant_name));
}
auto f = tx.commit();
const auto rc = waitAndHandleError(tx, f);
if (rc == FutureRC::OK) {
tx.reset();
break;
} else if (rc == FutureRC::RETRY) {
continue;
} else {
return fmt::format("create_tenant [{}:{}) failed with '{}'", id_begin, id_end, f.error().what());
}
}
// tenants created
// blobbify tenants
std::vector<fdb::TypedFuture<fdb::future_var::Bool>> blobbify_results(id_end - id_begin);
for (auto id = id_begin; id < id_end; id++) {
auto tenant = db.openTenant(fdb::toBytesRef(getTenantNameByIndex(id)));
std::string range_end = "\xff";
blobbify_results[id - id_begin] = tenant.blobbifyRange(fdb::BytesRef(), fdb::toBytesRef(range_end));
}
for (auto id = id_begin; id < id_end; id++) {
while (true) {
auto blobbify_future = blobbify_results[id - id_begin];
const auto rc = waitAndHandleError(tx, blobbify_future);
if (rc == FutureRC::OK) {
if (!blobbify_future.get()) {
return fmt::format("failed to blobbify tenant {}", id);
}
break;
} else if (rc == FutureRC::RETRY) {
continue;
} else {
return fmt::format("critical error encountered while blobbifying tenant {}: {}",
id,
blobbify_future.error().what());
}
}
}
return {};
} catch (const std::exception& e) {
return std::string(e.what());
}
}
boost::optional<std::string> AdminServer::getTenantPrefixes(fdb::Transaction tx,
int id_begin,
int id_end,
std::vector<fdb::ByteString>& out_prefixes) {
const auto num_tenants = id_end - id_begin;
auto tenant_futures = std::vector<fdb::TypedFuture<fdb::future_var::ValueRef>>(num_tenants);
while (true) {
for (auto id = id_begin; id < id_end; id++) {
tenant_futures[id - id_begin] = fdb::Tenant::getTenant(tx, fdb::toBytesRef(getTenantNameByIndex(id)));
}
out_prefixes.clear();
out_prefixes.reserve(num_tenants);
bool success = true;
for (auto id = id_begin; id < id_end; id++) {
auto f = tenant_futures[id - id_begin];
const auto rc = waitAndHandleError(tx, f);
if (rc == FutureRC::OK) {
if (f.get().has_value()) {
auto val = std::string(fdb::toCharsRef(f.get().value()));
auto doc = rapidjson::Document();
doc.Parse(val.c_str());
if (!doc.HasParseError()) {
// rapidjson does not decode the prefix as the same byte string that
// was passed as input. This is because we use a non-standard encoding.
// The encoding will likely change in the future.
// For a workaround, we take the id and compute the prefix on our own
auto tenant_prefix = fdb::ByteString(8, '\0');
computeTenantPrefix(tenant_prefix, doc["id"].GetUint64());
out_prefixes.push_back(tenant_prefix);
}
}
} else if (rc == FutureRC::RETRY) {
success = false;
break;
} else {
return fmt::format("unretryable error while getting metadata for tenant {}: {}", id, f.error().what());
}
}
if (success)
return {};
}
}
boost::optional<std::string> AdminServer::deleteTenant(fdb::Database db, int id_begin, int id_end) {
try {
while (true) {
std::vector<fdb::ByteString> prefixes;
if (auto error = getTenantPrefixes(db.createTransaction(), id_begin, id_end, prefixes)) {
return error;
}
auto tx = db.createTransaction();
tx.setOption(FDBTransactionOption::FDB_TR_OPTION_LOCK_AWARE, fdb::BytesRef());
tx.setOption(FDBTransactionOption::FDB_TR_OPTION_RAW_ACCESS, fdb::BytesRef());
for (const auto& tenant_prefix : prefixes) {
tx.clearRange(tenant_prefix, fdb::strinc(tenant_prefix));
}
auto commit_future = tx.commit();
auto rc = waitAndHandleError(tx, commit_future);
if (rc == FutureRC::OK) {
tx.reset();
// continue on this iteration
} else if (rc == FutureRC::RETRY) {
continue;
} else {
return fmt::format("unretryable error while clearing key ranges for tenant [{}:{}): {}",
id_begin,
id_end,
commit_future.error().what());
}
// tenant keyspaces have been cleared. now delete tenants
for (int id = id_begin; id < id_end; id++) {
fdb::Tenant::deleteTenant(tx, fdb::toBytesRef(getTenantNameByIndex(id)));
}
commit_future = tx.commit();
rc = waitAndHandleError(tx, commit_future);
if (rc == FutureRC::OK) {
return {};
} else if (rc == FutureRC::ABORT) {
return fmt::format("unretryable error while committing delete-tenant for tenant id range [{}:{}): {}",
id_begin,
id_end,
commit_future.error().what());
} else {
// try again
}
}
} catch (const std::exception& e) {
return std::string(e.what());
}
}
Response AdminServer::request(Request req) {
// should always be invoked from client side (currently just the main process)
assert(server_pid > 0);
assert(logr.isFor(ProcKind::MAIN));
sendObject(pipe_to_server, std::move(req));
return receiveObject<Response>(pipe_to_client);
}
AdminServer::~AdminServer() {
if (server_pid > 0) {
// may only be called from main process
assert(logr.isFor(ProcKind::MAIN));
auto res = send(ipc::StopRequest{});
if (res.error_message) {
logr.error("failed to stop admin server: {}", *res.error_message);
}
logr.info("waiting for admin server to terminate");
int status = 0;
::waitpid(server_pid, &status, 0);
if (status) {
logr.warn("admin server existed with error code {}", status);
}
}
}
} // namespace mako::ipc

View File

@ -0,0 +1,120 @@
/*
* admin_server.hpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include <string>
#include <utility>
#include <boost/process/pipe.hpp>
#include <boost/optional.hpp>
#include <boost/variant.hpp>
#include <unistd.h>
#include "fdb_api.hpp"
#include "logger.hpp"
#include "mako.hpp"
extern thread_local mako::Logger logr;
// IPC mechanism for executing administrative tasks (e.g. create/delete tenant) in support of benchmark.
// This is necessary because currently TLS configuration is a process-global setting.
// Therefore, order to benchmark for authorization
namespace mako::ipc {
struct Response {
boost::optional<std::string> error_message;
template <class Ar>
void serialize(Ar& ar, unsigned int) {
ar& error_message;
}
};
struct BatchCreateTenantRequest {
std::string cluster_file;
int id_begin = 0;
int id_end = 0;
template <class Ar>
void serialize(Ar& ar, unsigned int) {
ar& cluster_file;
ar& id_begin;
ar& id_end;
}
};
struct BatchDeleteTenantRequest {
std::string cluster_file;
int id_begin = 0;
int id_end = 0;
template <class Ar>
void serialize(Ar& ar, unsigned int) {
ar& id_begin;
ar& id_end;
}
};
struct PingRequest {
template <class Ar>
void serialize(Ar&, unsigned int) {}
};
struct StopRequest {
template <class Ar>
void serialize(Ar&, unsigned int) {}
};
using Request = boost::variant<PingRequest, StopRequest, BatchCreateTenantRequest, BatchDeleteTenantRequest>;
class AdminServer {
const Arguments& args;
pid_t server_pid;
boost::process::pstream pipe_to_server;
boost::process::pstream pipe_to_client;
void start();
void configure();
Response request(Request req);
boost::optional<std::string> getTenantPrefixes(fdb::Transaction tx,
int id_begin,
int id_end,
std::vector<fdb::ByteString>& out_prefixes);
boost::optional<std::string> createTenant(fdb::Database db, int id_begin, int id_end);
boost::optional<std::string> deleteTenant(fdb::Database db, int id_begin, int id_end);
public:
AdminServer(const Arguments& args)
: args(args), server_pid(-1), pipe_to_server(boost::process::pipe()), pipe_to_client(boost::process::pipe()) {
start();
}
~AdminServer();
// forks a server subprocess internally
bool isClient() const noexcept { return server_pid > 0; }
template <class T>
Response send(T req) {
return request(Request(std::forward<T>(req)));
}
AdminServer(const AdminServer&) = delete;
AdminServer(AdminServer&&) = delete;
AdminServer& operator=(const AdminServer&) = delete;
AdminServer& operator=(AdminServer&&) = delete;
};
} // namespace mako::ipc

View File

@ -24,6 +24,7 @@
#include <fdb_api.hpp>
#include <cassert>
#include <string_view>
#include <type_traits>
#include "logger.hpp"
#include "macro.hpp"
@ -33,29 +34,48 @@ namespace mako {
enum class FutureRC { OK, RETRY, ABORT };
template <class FutureType>
force_inline bool waitFuture(FutureType& f, std::string_view step) {
struct LogContext {
static constexpr const bool do_log = true;
LogContext(std::string_view step) noexcept : step(step), transaction_timeout_expected(false) {}
LogContext(std::string_view step, bool transaction_timeout_expected) noexcept
: step(step), transaction_timeout_expected(transaction_timeout_expected) {}
std::string_view step;
bool transaction_timeout_expected;
};
struct NoLog {
static constexpr const bool do_log = false;
};
template <class FutureType, class LogInfo>
force_inline bool waitFuture(FutureType& f, LogInfo log_info) {
assert(f);
auto err = f.blockUntilReady();
if (err) {
assert(!err.retryable());
logr.error("'{}' found at blockUntilReady during step '{}'", err.what(), step);
if constexpr (LogInfo::do_log) {
logr.error("'{}' found at blockUntilReady during step '{}'", err.what(), log_info.step);
}
return false;
} else {
return true;
}
}
template <class FutureType>
force_inline FutureRC
handleForOnError(fdb::Transaction& tx, FutureType& f, std::string_view step, bool timeouts_expected = false) {
namespace detail {
template <class FutureType, class LogInfo>
force_inline FutureRC handleForOnError(fdb::Transaction& tx, FutureType& f, LogInfo log_info) {
if (auto err = f.error()) {
assert(!(err.retryable()));
logr.printWithLogLevel(err.is(1031 /*timeout*/) && timeouts_expected ? VERBOSE_WARN : VERBOSE_NONE,
"ERROR",
"Unretryable error '{}' found at on_error(), step: {}",
err.what(),
step);
if constexpr (LogInfo::do_log) {
logr.printWithLogLevel(err.is(1031 /*timeout*/) && log_info.transaction_timeout_expected ? VERBOSE_WARN
: VERBOSE_NONE,
"ERROR",
"Unretryable error '{}' found at on_error(), step: {}",
err.what(),
log_info.step);
}
tx.reset();
return FutureRC::ABORT;
} else {
@ -63,39 +83,85 @@ handleForOnError(fdb::Transaction& tx, FutureType& f, std::string_view step, boo
}
}
} // namespace detail
template <class FutureType>
force_inline FutureRC
waitAndHandleForOnError(fdb::Transaction& tx, FutureType& f, std::string_view step, bool timeouts_expected = false) {
handleForOnError(fdb::Transaction& tx, FutureType& f, std::string_view step, bool transaction_timeout_expected) {
return detail::handleForOnError(tx, f, LogContext(step, transaction_timeout_expected));
}
template <class FutureType>
force_inline FutureRC handleForOnError(fdb::Transaction& tx, FutureType& f, std::string_view step) {
return detail::handleForOnError(tx, f, LogContext(step));
}
template <class FutureType>
force_inline FutureRC handleForOnError(fdb::Transaction& tx, FutureType& f) {
return detail::handleForOnError(tx, f, NoLog{});
}
namespace detail {
template <class FutureType, class LogInfo>
force_inline FutureRC waitAndHandleForOnError(fdb::Transaction& tx, FutureType& f, LogInfo log_info) {
assert(f);
if (!waitFuture(f, step)) {
if (!waitFuture(f, log_info)) {
return FutureRC::ABORT;
}
return handleForOnError(tx, f, step, timeouts_expected);
return detail::handleForOnError(tx, f, log_info);
}
// wait on any non-immediate tx-related step to complete. Follow up with on_error().
template <class FutureType>
force_inline FutureRC
waitAndHandleError(fdb::Transaction& tx, FutureType& f, std::string_view step, bool timeouts_expected = false) {
template <class FutureType, class LogInfo>
force_inline FutureRC waitAndHandleError(fdb::Transaction& tx, FutureType& f, LogInfo log_info) {
assert(f);
if (!waitFuture(f, step)) {
if (!waitFuture(f, log_info)) {
return FutureRC::ABORT;
}
auto err = f.error();
if (!err) {
return FutureRC::OK;
}
logr.printWithLogLevel(((err.is(1031 /*timeout*/) && timeouts_expected) || err.retryable()) ? VERBOSE_WARN
: VERBOSE_NONE,
"ERROR",
"step {} returned '{}'",
step,
err.what());
if constexpr (LogInfo::do_log) {
logr.printWithLogLevel(((err.is(1031 /*timeout*/) && log_info.transaction_timeout_expected) || err.retryable())
? VERBOSE_WARN
: VERBOSE_NONE,
"ERROR",
"step {} returned '{}'",
log_info.step,
err.what());
}
// implicit backoff
auto follow_up = tx.onError(err);
return waitAndHandleForOnError(tx, follow_up, step, timeouts_expected);
return waitAndHandleForOnError(tx, follow_up, log_info);
}
} // namespace detail
template <class FutureType>
force_inline FutureRC
waitAndHandleForOnError(fdb::Transaction& tx, FutureType& f, std::string_view step, bool transaction_timeout_expected) {
return detail::waitAndHandleForOnError(tx, f, LogContext(step, transaction_timeout_expected));
}
template <class FutureType>
force_inline FutureRC
waitAndHandleError(fdb::Transaction& tx, FutureType& f, std::string_view step, bool transaction_timeout_expected) {
return detail::waitAndHandleError(tx, f, LogContext(step, transaction_timeout_expected));
}
template <class FutureType>
force_inline FutureRC waitAndHandleError(fdb::Transaction& tx, FutureType& f, std::string_view step) {
return detail::waitAndHandleError(tx, f, LogContext(step));
}
template <class FutureType>
force_inline FutureRC waitAndHandleError(fdb::Transaction& tx, FutureType& f) {
return detail::waitAndHandleError(tx, f, NoLog{});
}
} // namespace mako
#endif /*MAKO_FUTURE_HPP*/

View File

@ -40,6 +40,7 @@ using ProcKindConstant = std::integral_constant<ProcKind, P>;
using MainProcess = ProcKindConstant<ProcKind::MAIN>;
using StatsProcess = ProcKindConstant<ProcKind::STATS>;
using WorkerProcess = ProcKindConstant<ProcKind::WORKER>;
using AdminProcess = ProcKindConstant<ProcKind::ADMIN>;
class Logger {
ProcKind proc;
@ -49,9 +50,11 @@ class Logger {
void putHeader(fmt::memory_buffer& buf, std::string_view category) {
if (proc == ProcKind::MAIN) {
fmt::format_to(std::back_inserter(buf), "[MAIN] {}: ", category);
fmt::format_to(std::back_inserter(buf), "[ MAIN] {}: ", category);
} else if (proc == ProcKind::STATS) {
fmt::format_to(std::back_inserter(buf), "[STATS] {}: ", category);
} else if (proc == ProcKind::ADMIN) {
fmt::format_to(std::back_inserter(buf), "[ADMIN] {}: ", category);
} else {
if (thread_id == -1) {
fmt::format_to(std::back_inserter(buf), "[WORKER{:3d}] {}: ", process_id + 1, category);
@ -67,6 +70,8 @@ public:
Logger(StatsProcess, int verbosity) noexcept : proc(StatsProcess::value), verbosity(verbosity) {}
Logger(AdminProcess, int verbosity) noexcept : proc(AdminProcess::value), verbosity(verbosity) {}
Logger(WorkerProcess, int verbosity, int process_id, int thread_id = -1) noexcept
: proc(WorkerProcess::value), verbosity(verbosity), process_id(process_id), thread_id(thread_id) {}
@ -113,6 +118,13 @@ public:
void debug(const fmt::format_string<Args...>& fmt_str, Args&&... args) {
printWithLogLevel(VERBOSE_DEBUG, "DEBUG", fmt_str, std::forward<Args>(args)...);
}
template <typename... Args>
void imm(Args&&... args) {
printWithLogLevel(VERBOSE_NONE, "IMMEDIATE", std::forward<Args>(args)...);
}
bool isFor(ProcKind procKind) const noexcept { return proc == procKind; }
};
} // namespace mako

View File

@ -29,6 +29,7 @@
#include <fstream>
#include <map>
#include <new>
#include <optional>
#if defined(__linux__)
#include <pthread.h>
#endif
@ -52,6 +53,7 @@
#include <unordered_map>
#include "fdbclient/zipf.h"
#include "admin_server.hpp"
#include "async.hpp"
#include "future.hpp"
#include "logger.hpp"
@ -61,6 +63,7 @@
#include "utils.hpp"
#include "shm.hpp"
#include "stats.hpp"
#include "tenant.hpp"
#include "time.hpp"
#include "rapidjson/document.h"
#include "rapidjson/error/en.h"
@ -94,49 +97,50 @@ Transaction createNewTransaction(Database db, Arguments const& args, int id = -1
// Create Tenant Transaction
int tenant_id = (id == -1) ? urand(0, args.active_tenants - 1) : id;
Transaction tr;
std::string tenantStr;
std::string tenant_name;
// If provided tenants array, use it
if (tenants) {
tr = tenants[tenant_id].createTransaction();
} else {
tenantStr = "tenant" + std::to_string(tenant_id);
BytesRef tenant_name = toBytesRef(tenantStr);
Tenant t = db.openTenant(tenant_name);
tenant_name = getTenantNameByIndex(tenant_id);
Tenant t = db.openTenant(toBytesRef(tenant_name));
tr = t.createTransaction();
}
if (!args.authorization_tokens.empty()) {
if (args.enable_token_based_authorization) {
assert(!args.authorization_tokens.empty());
// lookup token based on tenant name and, if found, set authz token to transaction
if (tenantStr.empty())
tenantStr = "tenant" + std::to_string(tenant_id);
auto tokenMapItr = args.authorization_tokens.find(tenantStr);
if (tokenMapItr != args.authorization_tokens.end()) {
tr.setOption(FDB_TR_OPTION_AUTHORIZATION_TOKEN, tokenMapItr->second);
if (tenant_name.empty())
tenant_name = getTenantNameByIndex(tenant_id);
auto token_map_iter = args.authorization_tokens.find(tenant_name);
if (token_map_iter != args.authorization_tokens.end()) {
tr.setOption(FDB_TR_OPTION_AUTHORIZATION_TOKEN, token_map_iter->second);
} else {
logr.warn("Authorization token map is not empty, but could not find token for tenant '{}'", tenantStr);
logr.error("could not find token for tenant '{}'", tenant_name);
exit(1);
}
}
return tr;
}
uint64_t byteswapHelper(uint64_t input) {
uint64_t output = 0;
for (int i = 0; i < 8; ++i) {
output <<= 8;
output += input & 0xFF;
input >>= 8;
int cleanupTenants(ipc::AdminServer& server, Arguments const& args, int db_id) {
for (auto tenant_id = 0; tenant_id < args.total_tenants;) {
const auto tenant_id_end = std::min(args.tenant_batch_size, args.total_tenants - tenant_id);
auto res = server.send(ipc::BatchDeleteTenantRequest{ args.cluster_files[db_id], tenant_id, tenant_id_end });
if (res.error_message) {
logr.error("{}", *res.error_message);
return -1;
} else {
logr.debug("deleted tenant [{}:{})", tenant_id, tenant_id_end);
tenant_id = tenant_id_end;
}
}
return output;
return 0;
}
void computeTenantPrefix(ByteString& s, uint64_t id) {
uint64_t swapped = byteswapHelper(id);
BytesRef temp = reinterpret_cast<const uint8_t*>(&swapped);
memcpy(&s[0], temp.data(), 8);
}
/* cleanup database */
int cleanup(Database db, Arguments const& args) {
/* cleanup database (no tenant-awareness) */
int cleanupNormalKeyspace(Database db, Arguments const& args) {
assert(args.total_tenants == 0 && args.active_tenants == 0);
const auto prefix_len = args.prefixpadding ? args.key_length - args.row_digits : intSize(KEY_PREFIX);
auto genprefix = [&args](ByteString& s) {
const auto padding_len = args.key_length - intSize(KEY_PREFIX) - args.row_digits;
@ -156,103 +160,17 @@ int cleanup(Database db, Arguments const& args) {
auto watch = Stopwatch(StartAtCtor{});
Transaction tx = db.createTransaction();
if (args.total_tenants == 0) {
while (true) {
tx.clearRange(beginstr, endstr);
auto future_commit = tx.commit();
const auto rc = waitAndHandleError(tx, future_commit, "COMMIT_CLEANUP");
if (rc == FutureRC::OK) {
break;
} else if (rc == FutureRC::RETRY) {
// tx already reset
continue;
} else {
return -1;
}
}
} else {
int batch_size = args.tenant_batch_size;
int batches = (args.total_tenants + batch_size - 1) / batch_size;
// First loop to clear all tenant key ranges
for (int batch = 0; batch < batches; ++batch) {
fdb::TypedFuture<fdb::future_var::ValueRef> tenantResults[batch_size];
// Issue all tenant reads first
Transaction getTx = db.createTransaction();
for (int i = batch * batch_size; i < args.total_tenants && i < (batch + 1) * batch_size; ++i) {
std::string tenant_name = "tenant" + std::to_string(i);
tenantResults[i - (batch * batch_size)] = Tenant::getTenant(getTx, toBytesRef(tenant_name));
}
tx.setOption(FDBTransactionOption::FDB_TR_OPTION_LOCK_AWARE, BytesRef());
tx.setOption(FDBTransactionOption::FDB_TR_OPTION_RAW_ACCESS, BytesRef());
for (int i = batch * batch_size; i < args.total_tenants && i < (batch + 1) * batch_size; ++i) {
std::string tenant_name = "tenant" + std::to_string(i);
while (true) {
const auto rc = waitAndHandleError(getTx, tenantResults[i - (batch * batch_size)], "GET_TENANT");
if (rc == FutureRC::OK) {
// Read the tenant metadata for the prefix and issue a range clear
if (tenantResults[i - (batch * batch_size)].get().has_value()) {
ByteString val(tenantResults[i - (batch * batch_size)].get().value());
rapidjson::Document doc;
const char* metadata = reinterpret_cast<const char*>(val.c_str());
doc.Parse(metadata);
if (!doc.HasParseError()) {
// rapidjson does not decode the prefix as the same byte string that
// was passed as input. This is because we use a non-standard encoding.
// The encoding will likely change in the future.
// For a workaround, we take the id and compute the prefix on our own
rapidjson::Value& docVal = doc["id"];
uint64_t id = docVal.GetUint64();
ByteString tenantPrefix(8, '\0');
computeTenantPrefix(tenantPrefix, id);
ByteString tenantPrefixEnd = strinc(tenantPrefix);
tx.clearRange(toBytesRef(tenantPrefix), toBytesRef(tenantPrefixEnd));
}
}
break;
} else if (rc == FutureRC::RETRY) {
continue;
} else {
// Abort
return -1;
}
}
}
auto future_commit = tx.commit();
const auto rc = waitAndHandleError(tx, future_commit, "TENANT_COMMIT_CLEANUP");
if (rc == FutureRC::OK) {
// Keep going with reset transaction if commit was successful
tx.reset();
} else if (rc == FutureRC::RETRY) {
// We want to retry this batch, so decrement the number
// and go back through the loop to get the same value
// Transaction is already reset
--batch;
} else {
// Abort
return -1;
}
}
// Second loop to delete the tenants
tx.reset();
for (int batch = 0; batch < batches; ++batch) {
for (int i = batch * batch_size; i < args.total_tenants && i < (batch + 1) * batch_size; ++i) {
std::string tenant_name = "tenant" + std::to_string(i);
Tenant::deleteTenant(tx, toBytesRef(tenant_name));
}
auto future_commit = tx.commit();
const auto rc = waitAndHandleError(tx, future_commit, "DELETE_TENANT");
if (rc == FutureRC::OK) {
// Keep going with reset transaction if commit was successful
tx.reset();
} else if (rc == FutureRC::RETRY) {
// We want to retry this batch, so decrement the number
// and go back through the loop to get the same value
// Transaction is already reset
--batch;
} else {
// Abort
return -1;
}
while (true) {
tx.clearRange(beginstr, endstr);
auto future_commit = tx.commit();
const auto rc = waitAndHandleError(tx, future_commit, "COMMIT_CLEANUP");
if (rc == FutureRC::OK) {
break;
} else if (rc == FutureRC::RETRY) {
// tx already reset
continue;
} else {
return -1;
}
}
@ -261,12 +179,10 @@ int cleanup(Database db, Arguments const& args) {
}
/* populate database */
int populate(Database db,
Arguments const& args,
int worker_id,
int thread_id,
int thread_tps,
ThreadStatistics& stats) {
int populate(Database db, const ThreadArgs& thread_args, int thread_tps, ThreadStatistics& stats) {
Arguments const& args = *thread_args.args;
const auto worker_id = thread_args.worker_id;
const auto thread_id = thread_args.thread_id;
auto xacts = 0;
auto keystr = ByteString{};
auto valstr = ByteString{};
@ -279,96 +195,10 @@ int populate(Database db,
auto watch_tx = Stopwatch(watch_total.getStart());
auto watch_trace = Stopwatch(watch_total.getStart());
if (args.total_tenants > 0) {
Transaction systemTx = db.createTransaction();
// Have one thread create all the tenants, then let the rest help with data population
if (worker_id == 0 && thread_id == 0) {
int batch_size = args.tenant_batch_size;
int batches = (args.total_tenants + batch_size - 1) / batch_size;
for (int batch = 0; batch < batches; ++batch) {
while (1) {
for (int i = batch * batch_size; i < args.total_tenants && i < (batch + 1) * batch_size; ++i) {
std::string tenant_str = "tenant" + std::to_string(i);
Tenant::createTenant(systemTx, toBytesRef(tenant_str));
}
auto future_commit = systemTx.commit();
const auto rc = waitAndHandleError(systemTx, future_commit, "CREATE_TENANT");
if (rc == FutureRC::OK) {
// Keep going with reset transaction if commit was successful
systemTx.reset();
break;
} else if (rc == FutureRC::RETRY) {
// We want to retry this batch. Transaction is already reset
} else {
// Abort
return -1;
}
}
Tenant tenants[batch_size];
fdb::TypedFuture<fdb::future_var::Bool> blobbifyResults[batch_size];
// blobbify tenant ranges explicitly
// FIXME: skip if database not configured for blob granules?
for (int i = batch * batch_size; i < args.total_tenants && i < (batch + 1) * batch_size; ++i) {
std::string tenant_str = "tenant" + std::to_string(i);
BytesRef tenant_name = toBytesRef(tenant_str);
tenants[i] = db.openTenant(tenant_name);
std::string rangeEnd = "\xff";
blobbifyResults[i - (batch * batch_size)] =
tenants[i].blobbifyRange(BytesRef(), toBytesRef(rangeEnd));
}
for (int i = batch * batch_size; i < args.total_tenants && i < (batch + 1) * batch_size; ++i) {
while (true) {
// not technically an operation that's part of systemTx, but it works
const auto rc =
waitAndHandleError(systemTx, blobbifyResults[i - (batch * batch_size)], "BLOBBIFY_TENANT");
if (rc == FutureRC::OK) {
if (!blobbifyResults[i - (batch * batch_size)].get()) {
fmt::print("Blobbifying tenant {0} failed!\n", i);
return -1;
}
break;
} else if (rc == FutureRC::RETRY) {
continue;
} else {
// Abort
return -1;
}
}
}
systemTx.reset();
}
} else {
std::string last_tenant_name = "tenant" + std::to_string(args.total_tenants - 1);
while (true) {
auto result = Tenant::getTenant(systemTx, toBytesRef(last_tenant_name));
const auto rc = waitAndHandleError(systemTx, result, "GET_TENANT");
if (rc == FutureRC::OK) {
// If we get valid tenant metadata, the main thread has finished
if (result.get().has_value()) {
break;
}
systemTx.reset();
} else if (rc == FutureRC::RETRY) {
continue;
} else {
// Abort
return -1;
}
usleep(1000);
}
}
}
// mimic typical tenant usage: keep tenants in memory
// and create transactions as needed
// tenants are assumed to have been generated by populateTenants() at main process, pre-fork
Tenant tenants[args.active_tenants];
for (int i = 0; i < args.active_tenants; ++i) {
std::string tenantStr = "tenant" + std::to_string(i);
BytesRef tenant_name = toBytesRef(tenantStr);
tenants[i] = db.openTenant(tenant_name);
tenants[i] = db.openTenant(toBytesRef(getTenantNameByIndex(i)));
}
int populate_iters = args.active_tenants > 0 ? args.active_tenants : 1;
// Each tenant should have the same range populated
@ -617,9 +447,7 @@ int runWorkload(Database db,
// and create transactions as needed
Tenant tenants[args.active_tenants];
for (int i = 0; i < args.active_tenants; ++i) {
std::string tenantStr = "tenant" + std::to_string(i);
BytesRef tenant_name = toBytesRef(tenantStr);
tenants[i] = db.openTenant(tenant_name);
tenants[i] = db.openTenant(toBytesRef(getTenantNameByIndex(i)));
}
/* main transaction loop */
@ -808,7 +636,7 @@ void runAsyncWorkload(Arguments const& args,
}
/* mako worker thread */
void workerThread(ThreadArgs& thread_args) {
void workerThread(const ThreadArgs& thread_args) {
const auto& args = *thread_args.args;
const auto parent_id = thread_args.parent_id;
const auto worker_id = thread_args.worker_id;
@ -846,12 +674,12 @@ void workerThread(ThreadArgs& thread_args) {
}
if (args.mode == MODE_CLEAN) {
auto rc = cleanup(database, args);
auto rc = cleanupNormalKeyspace(database, args);
if (rc < 0) {
logr.error("cleanup failed");
}
} else if (args.mode == MODE_BUILD) {
auto rc = populate(database, args, worker_id, thread_id, thread_tps, stats);
auto rc = populate(database, thread_args, thread_tps, stats);
if (rc < 0) {
logr.error("populate failed");
}
@ -874,111 +702,8 @@ int workerProcessMain(Arguments const& args, int worker_id, shared_memory::Acces
auto err = Error{};
/* Everything starts from here */
selectApiVersion(args.api_version);
/* enable distributed tracing */
switch (args.distributed_tracer_client) {
case DistributedTracerClient::NETWORK_LOSSY:
err = network::setOptionNothrow(FDB_NET_OPTION_DISTRIBUTED_CLIENT_TRACER, BytesRef(toBytePtr("network_lossy")));
break;
case DistributedTracerClient::LOG_FILE:
err = network::setOptionNothrow(FDB_NET_OPTION_DISTRIBUTED_CLIENT_TRACER, BytesRef(toBytePtr("log_file")));
break;
}
if (err) {
logr.error("network::setOption(FDB_NET_OPTION_DISTRIBUTED_CLIENT_TRACER): {}", err.what());
}
if (args.tls_certificate_file.has_value()) {
network::setOption(FDB_NET_OPTION_TLS_CERT_PATH, args.tls_certificate_file.value());
}
if (args.tls_key_file.has_value()) {
network::setOption(FDB_NET_OPTION_TLS_KEY_PATH, args.tls_key_file.value());
}
if (args.tls_ca_file.has_value()) {
network::setOption(FDB_NET_OPTION_TLS_CA_PATH, args.tls_ca_file.value());
}
/* enable flatbuffers if specified */
if (args.flatbuffers) {
#ifdef FDB_NET_OPTION_USE_FLATBUFFERS
logr.debug("Using flatbuffers");
err = network::setOptionNothrow(FDB_NET_OPTION_USE_FLATBUFFERS,
BytesRef(&args.flatbuffers, sizeof(args.flatbuffers)));
if (err) {
logr.error("network::setOption(USE_FLATBUFFERS): {}", err.what());
}
#else
logr.info("flatbuffers is not supported in FDB API version {}", FDB_API_VERSION);
#endif
}
/* Set client logr group */
if (args.log_group[0] != '\0') {
err = network::setOptionNothrow(FDB_NET_OPTION_TRACE_LOG_GROUP, BytesRef(toBytePtr(args.log_group)));
if (err) {
logr.error("network::setOption(FDB_NET_OPTION_TRACE_LOG_GROUP): {}", err.what());
}
}
/* enable tracing if specified */
if (args.trace) {
logr.debug("Enable Tracing in {} ({})",
(args.traceformat == 0) ? "XML" : "JSON",
(args.tracepath[0] == '\0') ? "current directory" : args.tracepath);
err = network::setOptionNothrow(FDB_NET_OPTION_TRACE_ENABLE, BytesRef(toBytePtr(args.tracepath)));
if (err) {
logr.error("network::setOption(TRACE_ENABLE): {}", err.what());
}
if (args.traceformat == 1) {
err = network::setOptionNothrow(FDB_NET_OPTION_TRACE_FORMAT, BytesRef(toBytePtr("json")));
if (err) {
logr.error("network::setOption(FDB_NET_OPTION_TRACE_FORMAT): {}", err.what());
}
}
}
/* enable knobs if specified */
if (args.knobs[0] != '\0') {
auto knobs = std::string_view(args.knobs);
const auto delim = std::string_view(", ");
while (true) {
knobs.remove_prefix(std::min(knobs.find_first_not_of(delim), knobs.size()));
auto knob = knobs.substr(0, knobs.find_first_of(delim));
if (knob.empty())
break;
logr.debug("Setting client knob: {}", knob);
err = network::setOptionNothrow(FDB_NET_OPTION_KNOB, toBytesRef(knob));
if (err) {
logr.error("network::setOption({}): {}", knob, err.what());
}
knobs.remove_prefix(knob.size());
}
}
if (args.client_threads_per_version > 0) {
err = network::setOptionNothrow(FDB_NET_OPTION_CLIENT_THREADS_PER_VERSION, args.client_threads_per_version);
if (err) {
logr.error("network::setOption (FDB_NET_OPTION_CLIENT_THREADS_PER_VERSION) ({}): {}",
args.client_threads_per_version,
err.what());
// let's exit here since we do not want to confuse users
// that mako is running with multi-threaded client enabled
return -1;
}
}
if (args.disable_client_bypass) {
err = network::setOptionNothrow(FDB_NET_OPTION_DISABLE_CLIENT_BYPASS);
if (err) {
logr.error("network::setOption (FDB_NET_OPTION_DISABLE_CLIENT_BYPASS): {}",
args.disable_client_bypass,
err.what());
return -1;
}
if (args.setGlobalOptions() < 0) {
return -1;
}
/* Network thread must be setup before doing anything */
@ -998,12 +723,26 @@ int workerProcessMain(Arguments const& args, int worker_id, shared_memory::Acces
pthread_setname_np(network_thread.native_handle(), "mako_network");
#endif
// prevent any exception from unwinding stack without joining the network thread
auto network_thread_guard = ExitGuard([&network_thread]() {
/* stop the network thread */
logr.debug("network::stop()");
auto err = network::stop();
if (err) {
logr.error("network::stop(): {}", err.what());
}
/* wait for the network thread to join */
logr.debug("waiting for network thread to join");
network_thread.join();
});
/*** let's party! ***/
auto databases = std::vector<fdb::Database>(args.num_databases);
/* set up database for worker threads */
for (auto i = 0; i < args.num_databases; i++) {
size_t cluster_index = args.num_fdb_clusters <= 1 ? 0 : i % args.num_fdb_clusters;
int cluster_index = i % args.num_fdb_clusters;
databases[i] = Database(args.cluster_files[cluster_index]);
logr.debug("creating database at cluster {}", args.cluster_files[cluster_index]);
if (args.disable_ryw) {
@ -1067,18 +806,6 @@ int workerProcessMain(Arguments const& args, int worker_id, shared_memory::Acces
thread.join();
shm.header().stopcount.fetch_add(args.num_threads);
}
/* stop the network thread */
logr.debug("network::stop()");
err = network::stop();
if (err) {
logr.error("network::stop(): {}", err.what());
}
/* wait for the network thread to join */
logr.debug("waiting for network thread to join");
network_thread.join();
return 0;
}
@ -1120,7 +847,9 @@ Arguments::Arguments() {
streaming_mode = FDB_STREAMING_MODE_WANT_ALL;
txntrace = 0;
txntagging = 0;
memset(cluster_files, 0, sizeof(cluster_files));
memset(txntagging_prefix, 0, TAGPREFIXLENGTH_MAX);
enable_token_based_authorization = false;
for (auto i = 0; i < MAX_OP; i++) {
txnspec.ops[i][OP_COUNT] = 0;
}
@ -1137,6 +866,116 @@ Arguments::Arguments() {
num_report_files = 0;
}
int Arguments::setGlobalOptions() const {
selectApiVersion(api_version);
auto err = Error{};
/* enable distributed tracing */
switch (distributed_tracer_client) {
case DistributedTracerClient::NETWORK_LOSSY:
err = network::setOptionNothrow(FDB_NET_OPTION_DISTRIBUTED_CLIENT_TRACER, BytesRef(toBytePtr("network_lossy")));
break;
case DistributedTracerClient::LOG_FILE:
err = network::setOptionNothrow(FDB_NET_OPTION_DISTRIBUTED_CLIENT_TRACER, BytesRef(toBytePtr("log_file")));
break;
}
if (err) {
logr.error("network::setOption(FDB_NET_OPTION_DISTRIBUTED_CLIENT_TRACER): {}", err.what());
}
if (tls_certificate_file.has_value() && (logr.isFor(ProcKind::ADMIN) || !isAuthorizationEnabled())) {
logr.debug("TLS certificate file: {}", tls_certificate_file.value());
network::setOption(FDB_NET_OPTION_TLS_CERT_PATH, tls_certificate_file.value());
}
if (tls_key_file.has_value() && (logr.isFor(ProcKind::ADMIN) || !isAuthorizationEnabled())) {
logr.debug("TLS key file: {}", tls_key_file.value());
network::setOption(FDB_NET_OPTION_TLS_KEY_PATH, tls_key_file.value());
}
if (tls_ca_file.has_value()) {
logr.debug("TLS CA file: {}", tls_ca_file.value());
network::setOption(FDB_NET_OPTION_TLS_CA_PATH, tls_ca_file.value());
}
/* enable flatbuffers if specified */
if (flatbuffers) {
#ifdef FDB_NET_OPTION_USE_FLATBUFFERS
logr.debug("Using flatbuffers");
err = network::setOptionNothrow(FDB_NET_OPTION_USE_FLATBUFFERS, BytesRef(&flatbuffers, sizeof(flatbuffers)));
if (err) {
logr.error("network::setOption(USE_FLATBUFFERS): {}", err.what());
}
#else
logr.info("flatbuffers is not supported in FDB API version {}", FDB_API_VERSION);
#endif
}
/* Set client logr group */
if (log_group[0] != '\0') {
err = network::setOptionNothrow(FDB_NET_OPTION_TRACE_LOG_GROUP, BytesRef(toBytePtr(log_group)));
if (err) {
logr.error("network::setOption(FDB_NET_OPTION_TRACE_LOG_GROUP): {}", err.what());
}
}
/* enable tracing if specified */
if (trace) {
logr.debug("Enable Tracing in {} ({})",
(traceformat == 0) ? "XML" : "JSON",
(tracepath[0] == '\0') ? "current directory" : tracepath);
err = network::setOptionNothrow(FDB_NET_OPTION_TRACE_ENABLE, BytesRef(toBytePtr(tracepath)));
if (err) {
logr.error("network::setOption(TRACE_ENABLE): {}", err.what());
}
if (traceformat == 1) {
err = network::setOptionNothrow(FDB_NET_OPTION_TRACE_FORMAT, BytesRef(toBytePtr("json")));
if (err) {
logr.error("network::setOption(FDB_NET_OPTION_TRACE_FORMAT): {}", err.what());
}
}
}
/* enable knobs if specified */
if (knobs[0] != '\0') {
auto k = std::string_view(knobs);
const auto delim = std::string_view(", ");
while (true) {
k.remove_prefix(std::min(k.find_first_not_of(delim), k.size()));
auto knob = k.substr(0, k.find_first_of(delim));
if (knob.empty())
break;
logr.debug("Setting client knob: {}", knob);
err = network::setOptionNothrow(FDB_NET_OPTION_KNOB, toBytesRef(knob));
if (err) {
logr.error("network::setOption({}): {}", knob, err.what());
}
k.remove_prefix(knob.size());
}
}
if (client_threads_per_version > 0) {
err = network::setOptionNothrow(FDB_NET_OPTION_CLIENT_THREADS_PER_VERSION, client_threads_per_version);
if (err) {
logr.error("network::setOption (FDB_NET_OPTION_CLIENT_THREADS_PER_VERSION) ({}): {}",
client_threads_per_version,
err.what());
// let's exit here since we do not want to confuse users
// that mako is running with multi-threaded client enabled
return -1;
}
}
if (disable_client_bypass) {
err = network::setOptionNothrow(FDB_NET_OPTION_DISABLE_CLIENT_BYPASS);
if (err) {
logr.error(
"network::setOption (FDB_NET_OPTION_DISABLE_CLIENT_BYPASS): {}", disable_client_bypass, err.what());
return -1;
}
}
return 0;
}
bool Arguments::isAnyTimeoutEnabled() const {
return (transaction_timeout_tx > 0 || transaction_timeout_db > 0);
}
@ -1327,6 +1166,18 @@ void usage() {
"Write the serialized DDSketch data to file at PATH. Can be used in either run or build mode.");
printf(
"%-24s %s\n", " --distributed_tracer_client=CLIENT", "Specify client (disabled, network_lossy, log_file)");
printf("%-24s %s\n", " --tls_key_file=PATH", "Location of TLS key file");
printf("%-24s %s\n", " --tls_ca_file=PATH", "Location of TLS CA file");
printf("%-24s %s\n", " --tls_certificate_file=PATH", "Location of TLS certificate file");
printf("%-24s %s\n",
" --enable_token_based_authorization",
"Make worker thread connect to server as untrusted clients to access tenant data");
printf("%-24s %s\n",
" --authorization_keypair_id",
"ID of the public key in the public key set which will verify the authorization tokens generated by Mako");
printf("%-24s %s\n",
" --authorization_private_key_pem",
"PEM-encoded private key with which Mako will sign generated tokens");
printf("%-24s %s\n",
" --transaction_timeout_db=DURATION",
"Duration in milliseconds after which a transaction times out in run mode. Set as database option.");
@ -1344,6 +1195,7 @@ int parseArguments(int argc, char* argv[], Arguments& args) {
const char* short_options = "a:c:d:p:t:r:s:i:x:v:m:hz";
static struct option long_options[] = {
/* name, has_arg, flag, val */
/* options requiring an argument */
{ "api_version", required_argument, NULL, 'a' },
{ "cluster", required_argument, NULL, 'c' },
{ "num_databases", required_argument, NULL, 'd' },
@ -1374,29 +1226,32 @@ int parseArguments(int argc, char* argv[], Arguments& args) {
{ "trace_format", required_argument, NULL, ARG_TRACEFORMAT },
{ "streaming", required_argument, NULL, ARG_STREAMING_MODE },
{ "txntrace", required_argument, NULL, ARG_TXNTRACE },
/* no args */
{ "txntagging", required_argument, NULL, ARG_TXNTAGGING },
{ "txntagging_prefix", required_argument, NULL, ARG_TXNTAGGINGPREFIX },
{ "client_threads_per_version", required_argument, NULL, ARG_CLIENT_THREADS_PER_VERSION },
{ "bg_file_path", required_argument, NULL, ARG_BG_FILE_PATH },
{ "distributed_tracer_client", required_argument, NULL, ARG_DISTRIBUTED_TRACER_CLIENT },
{ "tls_certificate_file", required_argument, NULL, ARG_TLS_CERTIFICATE_FILE },
{ "tls_key_file", required_argument, NULL, ARG_TLS_KEY_FILE },
{ "tls_ca_file", required_argument, NULL, ARG_TLS_CA_FILE },
{ "authorization_keypair_id", required_argument, NULL, ARG_AUTHORIZATION_KEYPAIR_ID },
{ "authorization_private_key_pem_file", required_argument, NULL, ARG_AUTHORIZATION_PRIVATE_KEY_PEM_FILE },
{ "transaction_timeout_tx", required_argument, NULL, ARG_TRANSACTION_TIMEOUT_TX },
{ "transaction_timeout_db", required_argument, NULL, ARG_TRANSACTION_TIMEOUT_DB },
/* options which may or may not have an argument */
{ "json_report", optional_argument, NULL, ARG_JSON_REPORT },
{ "stats_export_path", optional_argument, NULL, ARG_EXPORT_PATH },
/* options without an argument */
{ "help", no_argument, NULL, 'h' },
{ "zipf", no_argument, NULL, 'z' },
{ "commitget", no_argument, NULL, ARG_COMMITGET },
{ "flatbuffers", no_argument, NULL, ARG_FLATBUFFERS },
{ "prefix_padding", no_argument, NULL, ARG_PREFIXPADDING },
{ "trace", no_argument, NULL, ARG_TRACE },
{ "txntagging", required_argument, NULL, ARG_TXNTAGGING },
{ "txntagging_prefix", required_argument, NULL, ARG_TXNTAGGINGPREFIX },
{ "version", no_argument, NULL, ARG_VERSION },
{ "client_threads_per_version", required_argument, NULL, ARG_CLIENT_THREADS_PER_VERSION },
{ "disable_client_bypass", no_argument, NULL, ARG_DISABLE_CLIENT_BYPASS },
{ "disable_ryw", no_argument, NULL, ARG_DISABLE_RYW },
{ "json_report", optional_argument, NULL, ARG_JSON_REPORT },
{ "bg_file_path", required_argument, NULL, ARG_BG_FILE_PATH },
{ "stats_export_path", optional_argument, NULL, ARG_EXPORT_PATH },
{ "distributed_tracer_client", required_argument, NULL, ARG_DISTRIBUTED_TRACER_CLIENT },
{ "tls_certificate_file", required_argument, NULL, ARG_TLS_CERTIFICATE_FILE },
{ "tls_key_file", required_argument, NULL, ARG_TLS_KEY_FILE },
{ "tls_ca_file", required_argument, NULL, ARG_TLS_CA_FILE },
{ "authorization_token_file", required_argument, NULL, ARG_AUTHORIZATION_TOKEN_FILE },
{ "transaction_timeout_tx", required_argument, NULL, ARG_TRANSACTION_TIMEOUT_TX },
{ "transaction_timeout_db", required_argument, NULL, ARG_TRANSACTION_TIMEOUT_DB },
{ "enable_token_based_authorization", no_argument, NULL, ARG_ENABLE_TOKEN_BASED_AUTHORIZATION },
{ NULL, 0, NULL, 0 }
};
@ -1622,6 +1477,7 @@ int parseArguments(int argc, char* argv[], Arguments& args) {
case ARG_BG_FILE_PATH:
args.bg_materialize_files = true;
strncpy(args.bg_file_path, optarg, std::min(sizeof(args.bg_file_path), strlen(optarg) + 1));
break;
case ARG_EXPORT_PATH:
SET_OPT_ARG_IF_PRESENT();
if (!optarg) {
@ -1651,35 +1507,15 @@ int parseArguments(int argc, char* argv[], Arguments& args) {
case ARG_TLS_CA_FILE:
args.tls_ca_file = std::string(optarg);
break;
case ARG_AUTHORIZATION_TOKEN_FILE: {
std::string tokenFilename(optarg);
std::ifstream ifs(tokenFilename);
case ARG_AUTHORIZATION_KEYPAIR_ID:
args.keypair_id = optarg;
break;
case ARG_AUTHORIZATION_PRIVATE_KEY_PEM_FILE: {
std::string pem_filename(optarg);
std::ifstream ifs(pem_filename);
std::ostringstream oss;
oss << ifs.rdbuf();
rapidjson::Document d;
d.Parse(oss.str().c_str());
if (d.HasParseError()) {
logr.error("Failed to parse authorization token JSON file '{}': {} at offset {}",
tokenFilename,
GetParseError_En(d.GetParseError()),
d.GetErrorOffset());
return -1;
} else if (!d.IsObject()) {
logr.error("Authorization token JSON file '{}' must contain a JSON object", tokenFilename);
return -1;
}
for (auto itr = d.MemberBegin(); itr != d.MemberEnd(); ++itr) {
if (!itr->value.IsString()) {
logr.error("Token '{}' is not a string", itr->name.GetString());
return -1;
}
args.authorization_tokens.insert_or_assign(
std::string(itr->name.GetString(), itr->name.GetStringLength()),
std::string(itr->value.GetString(), itr->value.GetStringLength()));
}
logr.info("Added {} tenant authorization tokens to map from file '{}'",
args.authorization_tokens.size(),
tokenFilename);
args.private_key_pem = oss.str();
} break;
case ARG_TRANSACTION_TIMEOUT_TX:
args.transaction_timeout_tx = atoi(optarg);
@ -1687,6 +1523,9 @@ int parseArguments(int argc, char* argv[], Arguments& args) {
case ARG_TRANSACTION_TIMEOUT_DB:
args.transaction_timeout_db = atoi(optarg);
break;
case ARG_ENABLE_TOKEN_BASED_AUTHORIZATION:
args.enable_token_based_authorization = true;
break;
}
}
@ -1815,12 +1654,42 @@ int Arguments::validate() {
return -1;
}
if (!authorization_tokens.empty() && !tls_ca_file.has_value()) {
logr.warn("Authorization tokens are being used without explicit TLS CA file configured");
if (enable_token_based_authorization) {
if (active_tenants <= 0 || total_tenants <= 0) {
logr.error("--enable_token_based_authorization must be used with at least one tenant");
return -1;
}
if (!private_key_pem.has_value() || !keypair_id.has_value()) {
logr.error("--enable_token_based_authorization must be used with --authorization_keypair_id and "
"--authorization_private_key_pem_file");
return -1;
}
if (!tls_key_file.has_value() || !tls_certificate_file.has_value() || !tls_ca_file.has_value()) {
logr.error(
"token-based authorization is enabled without explicit TLS parameter(s) (certificate, key, CA).");
return -1;
}
}
return 0;
}
bool Arguments::isAuthorizationEnabled() const noexcept {
return mode != MODE_CLEAN && enable_token_based_authorization && active_tenants > 0 && tls_ca_file.has_value() &&
private_key_pem.has_value();
}
void Arguments::generateAuthorizationTokens() {
assert(active_tenants > 0);
assert(keypair_id.has_value());
assert(private_key_pem.has_value());
authorization_tokens.clear();
logr.info("generating authorization tokens to be used by worker threads");
auto stopwatch = Stopwatch(StartAtCtor{});
authorization_tokens = generateAuthorizationTokenMap(active_tenants, keypair_id.value(), private_key_pem.value());
assert(authorization_tokens.size() == active_tenants);
logr.info("generated {} tokens in {:6.3f} seconds", active_tenants, toDoubleSeconds(stopwatch.stop().diff()));
}
void printStats(Arguments const& args, ThreadStatistics const* stats, double const duration_sec, FILE* fp) {
static ThreadStatistics prev;
@ -2489,6 +2358,27 @@ ThreadStatistics mergeSketchReport(Arguments& args) {
return stats;
}
int populateTenants(ipc::AdminServer& admin, const Arguments& args) {
const auto num_dbs = std::min(args.num_fdb_clusters, args.num_databases);
logr.info("populating {} tenants for {} database(s)", args.total_tenants, num_dbs);
auto stopwatch = Stopwatch(StartAtCtor{});
for (auto i = 0; i < num_dbs; i++) {
for (auto tenant_id = 0; tenant_id < args.total_tenants;) {
const auto tenant_id_end = tenant_id + std::min(args.tenant_batch_size, args.total_tenants - tenant_id);
auto res = admin.send(ipc::BatchCreateTenantRequest{ args.cluster_files[i], tenant_id, tenant_id_end });
if (res.error_message) {
logr.error("cluster {}: {}", i + 1, *res.error_message);
return -1;
} else {
logr.debug("created tenants [{}:{}) for cluster {}", tenant_id, tenant_id_end, i + 1);
tenant_id = tenant_id_end;
}
}
}
logr.info("populated tenants in {:6.3f} seconds", toDoubleSeconds(stopwatch.stop().diff()));
return 0;
}
int main(int argc, char* argv[]) {
setlinebuf(stdout);
@ -2514,11 +2404,23 @@ int main(int argc, char* argv[]) {
args.seconds = 30; // default value accodring to documentation
}
// if no cluster file is passed, fall back to default parameters
// (envvar, 'fdb.cluster' or platform-dependent path)
if (args.num_fdb_clusters == 0) {
args.num_fdb_clusters = 1;
}
rc = args.validate();
if (rc < 0)
return -1;
logr.setVerbosity(args.verbose);
if (args.isAuthorizationEnabled()) {
args.generateAuthorizationTokens();
}
if (args.mode == MODE_CLEAN) {
/* cleanup will be done from a single thread */
args.num_processes = 1;
@ -2537,6 +2439,46 @@ int main(int argc, char* argv[]) {
return 0;
}
if (args.total_tenants > 0 && (args.mode == MODE_BUILD || args.mode == MODE_CLEAN)) {
// below construction fork()s internally
auto server = ipc::AdminServer(args);
if (!server.isClient()) {
// admin server has finished running. exit immediately
return 0;
} else {
auto res = server.send(ipc::PingRequest{});
if (res.error_message) {
logr.error("admin server setup failed: {}", *res.error_message);
return -1;
} else {
logr.info("admin server ready");
}
}
// Use admin server to request tenant creation or deletion.
// This is necessary when tenant authorization is enabled,
// in which case the worker threads connect to database as untrusted clients,
// as which they wouldn't be allowed to create/delete tenants on their own.
// Although it is possible to allow worker threads to create/delete
// tenants in a authorization-disabled mode, use the admin server anyway for simplicity.
if (args.mode == MODE_CLEAN) {
// short-circuit tenant cleanup
const auto num_dbs = std::min(args.num_fdb_clusters, args.num_databases);
for (auto db_id = 0; db_id < num_dbs; db_id++) {
if (cleanupTenants(server, args, db_id) < 0) {
return -1;
}
}
return 0;
} else if (args.mode == MODE_BUILD) {
// handle population of tenants before-fork
if (populateTenants(server, args) < 0) {
return -1;
}
}
}
const auto pid_main = getpid();
/* create the shared memory for stats */
const auto shmpath = fmt::format("mako{}", pid_main);

View File

@ -31,8 +31,9 @@
#include <chrono>
#include <list>
#include <map>
#include <vector>
#include <string>
#include <string_view>
#include <vector>
#include <fdb_api.hpp>
#include <pthread.h>
#include <sys/types.h>
@ -84,7 +85,9 @@ enum ArgKind {
ARG_TLS_CERTIFICATE_FILE,
ARG_TLS_KEY_FILE,
ARG_TLS_CA_FILE,
ARG_AUTHORIZATION_TOKEN_FILE,
ARG_AUTHORIZATION_KEYPAIR_ID,
ARG_AUTHORIZATION_PRIVATE_KEY_PEM_FILE,
ARG_ENABLE_TOKEN_BASED_AUTHORIZATION,
ARG_TRANSACTION_TIMEOUT_TX,
ARG_TRANSACTION_TIMEOUT_DB,
};
@ -140,6 +143,11 @@ constexpr const int MAX_REPORT_FILES = 200;
struct Arguments {
Arguments();
int validate();
bool isAuthorizationEnabled() const noexcept;
void generateAuthorizationTokens();
// Needs to be called once per fdb-accessing process
int setGlobalOptions() const;
bool isAnyTimeoutEnabled() const;
int api_version;
@ -191,9 +199,12 @@ struct Arguments {
char report_files[MAX_REPORT_FILES][PATH_MAX];
int num_report_files;
int distributed_tracer_client;
bool enable_token_based_authorization;
std::optional<std::string> tls_certificate_file;
std::optional<std::string> tls_key_file;
std::optional<std::string> tls_ca_file;
std::optional<std::string> keypair_id;
std::optional<std::string> private_key_pem;
std::map<std::string, std::string> authorization_tokens; // maps tenant name to token string
int transaction_timeout_db;
int transaction_timeout_tx;

View File

@ -21,6 +21,6 @@
#ifndef MAKO_PROCESS_HPP
#define MAKO_PROCESS_HPP
enum class ProcKind { MAIN, WORKER, STATS };
enum class ProcKind { MAIN, WORKER, STATS, ADMIN };
#endif /*MAKO_PROCESS_HPP*/

View File

@ -0,0 +1,56 @@
/*
* tenant.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include <chrono>
#include "tenant.hpp"
#include "time.hpp"
#include "utils.hpp"
#include "fdbrpc/TokenSignStdTypes.h"
namespace mako {
std::map<std::string, std::string> generateAuthorizationTokenMap(int num_tenants,
std::string public_key_id,
std::string private_key_pem) {
std::map<std::string, std::string> m;
auto t = authz::jwt::stdtypes::TokenSpec{};
auto const now = toIntegerSeconds(std::chrono::system_clock::now().time_since_epoch());
t.algorithm = authz::Algorithm::ES256;
t.keyId = public_key_id;
t.issuer = "mako";
t.subject = "benchmark";
t.audience = std::vector<std::string>{ "fdb_benchmark_server" };
t.issuedAtUnixTime = now;
t.expiresAtUnixTime = now + 60 * 60 * 12; // Good for 12 hours
t.notBeforeUnixTime = now - 60 * 5; // activated 5 mins ago
const int tokenIdLen = 36; // UUID length
auto tokenId = std::string(tokenIdLen, '\0');
for (auto i = 0; i < num_tenants; i++) {
std::string tenant_name = getTenantNameByIndex(i);
// swap out only the token ids and tenant names
randomAlphanumString(tokenId.data(), tokenIdLen);
t.tokenId = tokenId;
t.tenants = std::vector<std::string>{ tenant_name };
m[tenant_name] = authz::jwt::stdtypes::signToken(t, private_key_pem);
}
return m;
}
} // namespace mako

View File

@ -0,0 +1,44 @@
/*
* tenant.hpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include <cassert>
#include <map>
#include <string>
#include "fdb_api.hpp"
#include "utils.hpp"
namespace mako {
std::map<std::string, std::string> generateAuthorizationTokenMap(int tenants,
std::string public_key_id,
std::string private_key_pem);
inline std::string getTenantNameByIndex(int index) {
assert(index >= 0);
return "tenant" + std::to_string(index);
}
inline void computeTenantPrefix(fdb::ByteString& s, uint64_t id) {
uint64_t swapped = byteswapHelper(id);
fdb::BytesRef temp = reinterpret_cast<const uint8_t*>(&swapped);
memcpy(&s[0], temp.data(), 8);
}
} // namespace mako

View File

@ -35,6 +35,16 @@
namespace mako {
inline uint64_t byteswapHelper(uint64_t input) {
uint64_t output = 0;
for (int i = 0; i < 8; ++i) {
output <<= 8;
output += input & 0xFF;
input >>= 8;
}
return output;
}
/* uniform-distribution random */
/* return a uniform random number between low and high, both inclusive */
force_inline int urand(int low, int high) {
@ -53,6 +63,21 @@ force_inline int intSize(std::string_view sv) {
return static_cast<int>(sv.size());
}
template <typename Char>
inline void randomAlphanumString(Char* str, int len) {
constexpr auto chars_per_alpha = 26;
constexpr auto range = chars_per_alpha * 2 + 10; // uppercase, lowercase, digits
for (auto i = 0; i < len; i++) {
auto value = urand(0, range - 1);
if (value < chars_per_alpha)
str[i] = 'a' + value;
else if (value < 2 * chars_per_alpha)
str[i] = 'A' + value - chars_per_alpha;
else
str[i] = '0' + value - 2 * chars_per_alpha;
}
}
/* random string */
template <typename Char>
force_inline void randomString(Char* str, int len) {

View File

@ -468,6 +468,69 @@ function(package_bindingtester)
add_dependencies(bindingtester copy_bindingtester_binaries)
endfunction()
# Test for setting up Python venv for client tests.
# Adding this test as a fixture to another test allows the use of non-native Python packages within client test scripts
# by installing dependencies from requirements.txt
set(test_venv_dir ${CMAKE_BINARY_DIR}/tests/test_venv)
if (WIN32)
set(shell_cmd "cmd" CACHE INTERNAL "")
set(shell_opt "/c" CACHE INTERNAL "")
set(test_venv_activate "${test_venv_dir}/Scripts/activate.bat" CACHE INTERNAL "")
else()
set(shell_cmd "bash" CACHE INTERNAL "")
set(shell_opt "-c" CACHE INTERNAL "")
set(test_venv_activate ". ${test_venv_dir}/bin/activate" CACHE INTERNAL "")
endif()
set(test_venv_cmd "")
string(APPEND test_venv_cmd "${Python3_EXECUTABLE} -m venv ${test_venv_dir} ")
string(APPEND test_venv_cmd "&& ${test_venv_activate} ")
string(APPEND test_venv_cmd "&& pip install --upgrade pip ")
string(APPEND test_venv_cmd "&& pip install -r ${CMAKE_SOURCE_DIR}/tests/TestRunner/requirements.txt")
add_test(
NAME test_venv_setup
COMMAND bash -c ${test_venv_cmd}
WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR})
set_tests_properties(test_venv_setup PROPERTIES FIXTURES_SETUP test_virtual_env_setup TIMEOUT 120)
set_tests_properties(test_venv_setup PROPERTIES RESOURCE_LOCK TEST_VENV_SETUP)
# Run the test command under Python venv as a cmd (Windows) or bash (Linux/Apple) script, which allows && or || chaining.
function(add_python_venv_test)
set(oneValueArgs NAME WORKING_DIRECTORY TEST_TIMEOUT)
set(multiValueArgs COMMAND)
cmake_parse_arguments(T "" "${oneValueArgs}" "${multiValueArgs}" "${ARGN}")
if(OPEN_FOR_IDE)
return()
endif()
if(NOT T_NAME)
message(FATAL_ERROR "NAME is a required argument for add_fdbclient_test")
endif()
if(NOT T_COMMAND)
message(FATAL_ERROR "COMMAND is a required argument for add_fdbclient_test")
endif()
if(NOT T_WORKING_DIRECTORY)
set(T_WORKING_DIRECTORY ${CMAKE_BINARY_DIR})
endif()
if(NOT T_TEST_TIMEOUT)
if(USE_SANITIZER)
set(T_TEST_TIMEOUT 1200)
else()
set(T_TEST_TIMEOUT 300)
endif()
endif()
# expand list of command arguments to space-separated string so that we can pass to shell
string(REPLACE ";" " " T_COMMAND "${T_COMMAND}")
add_test(
NAME ${T_NAME}
WORKING_DIRECTORY ${T_WORKING_DIRECTORY}
COMMAND ${shell_cmd} ${shell_opt} "${test_venv_activate} && ${T_COMMAND}")
set_tests_properties(${T_NAME} PROPERTIES FIXTURES_REQUIRED test_virtual_env_setup TIMEOUT ${T_TEST_TIMEOUT})
set(test_env_vars "PYTHONPATH=${CMAKE_SOURCE_DIR}/tests/TestRunner:${CMAKE_BINARY_DIR}/tests/TestRunner")
if(USE_SANITIZER)
set(test_env_vars "${test_env_vars};${SANITIZER_OPTIONS}")
endif()
set_tests_properties("${T_NAME}" PROPERTIES ENVIRONMENT "${test_env_vars}")
endfunction()
# Creates a single cluster before running the specified command (usually a ctest test)
function(add_fdbclient_test)
set(options DISABLED ENABLED DISABLE_TENANTS DISABLE_LOG_DUMP API_TEST_BLOB_GRANULES_ENABLED TLS_ENABLED)
@ -489,8 +552,7 @@ function(add_fdbclient_test)
if(NOT T_COMMAND)
message(FATAL_ERROR "COMMAND is a required argument for add_fdbclient_test")
endif()
set(TMP_CLUSTER_CMD ${CMAKE_SOURCE_DIR}/tests/TestRunner/tmp_cluster.py
--build-dir ${CMAKE_BINARY_DIR})
set(TMP_CLUSTER_CMD python ${CMAKE_SOURCE_DIR}/tests/TestRunner/tmp_cluster.py --build-dir ${CMAKE_BINARY_DIR})
if(T_PROCESS_NUMBER)
list(APPEND TMP_CLUSTER_CMD --process-number ${T_PROCESS_NUMBER})
endif()
@ -506,23 +568,21 @@ function(add_fdbclient_test)
if(T_TLS_ENABLED)
list(APPEND TMP_CLUSTER_CMD --tls-enabled)
endif()
list(APPEND TMP_CLUSTER_CMD -- ${T_COMMAND})
message(STATUS "Adding Client test ${T_NAME}")
add_test(NAME "${T_NAME}"
WORKING_DIRECTORY ${T_WORKING_DIRECTORY}
COMMAND ${Python3_EXECUTABLE} ${TMP_CLUSTER_CMD}
--
${T_COMMAND})
if (T_TEST_TIMEOUT)
set_tests_properties("${T_NAME}" PROPERTIES TIMEOUT ${T_TEST_TIMEOUT})
else()
if (NOT T_TEST_TIMEOUT)
# default timeout
if(USE_SANITIZER)
set_tests_properties("${T_NAME}" PROPERTIES TIMEOUT 1200)
set(T_TEST_TIMEOUT 1200)
else()
set_tests_properties("${T_NAME}" PROPERTIES TIMEOUT 300)
set(T_TEST_TIMEOUT 300)
endif()
endif()
set_tests_properties("${T_NAME}" PROPERTIES ENVIRONMENT "${SANITIZER_OPTIONS}")
add_python_venv_test(
NAME ${T_NAME}
WORKING_DIRECTORY ${T_WORKING_DIRECTORY}
COMMAND ${TMP_CLUSTER_CMD}
TEST_TIMEOUT ${T_TEST_TIMEOUT})
endfunction()
# Creates a cluster file for a nonexistent cluster before running the specified command
@ -544,19 +604,16 @@ function(add_unavailable_fdbclient_test)
if(NOT T_COMMAND)
message(FATAL_ERROR "COMMAND is a required argument for add_unavailable_fdbclient_test")
endif()
message(STATUS "Adding unavailable client test ${T_NAME}")
add_test(NAME "${T_NAME}"
COMMAND ${Python3_EXECUTABLE} ${CMAKE_SOURCE_DIR}/tests/TestRunner/fake_cluster.py
--output-dir ${CMAKE_BINARY_DIR}
--
${T_COMMAND})
if (T_TEST_TIMEOUT)
set_tests_properties("${T_NAME}" PROPERTIES TIMEOUT ${T_TEST_TIMEOUT})
else()
if (NOT T_TEST_TIMEOUT)
# default timeout
set_tests_properties("${T_NAME}" PROPERTIES TIMEOUT 60)
set(T_TEST_TIMEOUT 60)
endif()
set_tests_properties("${T_NAME}" PROPERTIES ENVIRONMENT "${SANITIZER_OPTIONS}")
message(STATUS "Adding unavailable client test ${T_NAME}")
add_python_venv_test(
NAME ${T_NAME}
COMMAND python ${CMAKE_SOURCE_DIR}/tests/TestRunner/fake_cluster.py
--output-dir ${CMAKE_BINARY_DIR} -- ${T_COMMAND}
TEST_TIMEOUT ${T_TEST_TIMEOUT})
endfunction()
# Creates 3 distinct clusters before running the specified command.
@ -580,13 +637,12 @@ function(add_multi_fdbclient_test)
message(FATAL_ERROR "COMMAND is a required argument for add_multi_fdbclient_test")
endif()
message(STATUS "Adding Client test ${T_NAME}")
add_test(NAME "${T_NAME}"
COMMAND ${Python3_EXECUTABLE} ${CMAKE_SOURCE_DIR}/tests/TestRunner/tmp_multi_cluster.py
add_python_venv_test(
NAME ${T_NAME}
COMMAND python ${CMAKE_SOURCE_DIR}/tests/TestRunner/tmp_multi_cluster.py
--build-dir ${CMAKE_BINARY_DIR}
--clusters 3
--
${T_COMMAND})
set_tests_properties("${T_NAME}" PROPERTIES TIMEOUT 60)
--clusters 3 -- ${T_COMMAND}
TEST_TIMEOUT 60)
endfunction()
function(add_java_test)
@ -623,39 +679,3 @@ function(add_java_test)
-Djava.library.path=${CMAKE_BINARY_DIR}/lib
${T_CLASS} "@CLUSTER_FILE@")
endfunction()
# Adds a FDB test implemented by a script that does the full setup, such as creating cluster
# and running client binaries as necessary
function(add_scripted_fdb_test)
set(options DISABLED ENABLED)
set(oneValueArgs NAME TEST_TIMEOUT)
set(multiValueArgs COMMAND)
cmake_parse_arguments(T "${options}" "${oneValueArgs}" "${multiValueArgs}" "${ARGN}")
if(OPEN_FOR_IDE)
return()
endif()
if(NOT T_ENABLED AND T_DISABLED)
return()
endif()
if(NOT T_NAME)
message(FATAL_ERROR "NAME is a required argument for add_scripted_fdb_test")
endif()
if(NOT T_COMMAND)
message(FATAL_ERROR "COMMAND is a required argument for add_scripted_fdb_test")
endif()
message(STATUS "Adding Scripted FDB test ${T_NAME}")
add_test(NAME "${T_NAME}"
COMMAND ${T_COMMAND})
set_tests_properties("${T_NAME}" PROPERTIES ENVIRONMENT
"${SANITIZER_OPTIONS};PYTHONPATH=${CMAKE_SOURCE_DIR}/tests/TestRunner:${CMAKE_BINARY_DIR}/tests/TestRunner")
if (T_TEST_TIMEOUT)
set_tests_properties("${T_NAME}" PROPERTIES TIMEOUT ${T_TEST_TIMEOUT})
else()
# default timeout
if(USE_SANITIZER)
set_tests_properties("${T_NAME}" PROPERTIES TIMEOUT 1200)
else()
set_tests_properties("${T_NAME}" PROPERTIES TIMEOUT 300)
endif()
endif()
endfunction()

View File

@ -9,7 +9,7 @@ function(compile_boost)
# Configure bootstrap command
set(BOOTSTRAP_COMMAND "./bootstrap.sh")
set(BOOTSTRAP_LIBRARIES "context,filesystem,iostreams")
set(BOOTSTRAP_LIBRARIES "context,filesystem,iostreams,system,serialization")
set(BOOST_CXX_COMPILER "${CMAKE_CXX_COMPILER}")
# Can't build Boost with Intel compiler, use clang instead.
@ -77,7 +77,9 @@ function(compile_boost)
BUILD_BYPRODUCTS "${BOOST_INSTALL_DIR}/include/boost/config.hpp"
"${BOOST_INSTALL_DIR}/lib/libboost_context.a"
"${BOOST_INSTALL_DIR}/lib/libboost_filesystem.a"
"${BOOST_INSTALL_DIR}/lib/libboost_iostreams.a")
"${BOOST_INSTALL_DIR}/lib/libboost_iostreams.a"
"${BOOST_INSTALL_DIR}/lib/libboost_serialization.a"
"${BOOST_INSTALL_DIR}/lib/libboost_system.a")
add_library(${COMPILE_BOOST_TARGET}_context STATIC IMPORTED)
add_dependencies(${COMPILE_BOOST_TARGET}_context ${COMPILE_BOOST_TARGET}Project)
@ -91,9 +93,17 @@ function(compile_boost)
add_dependencies(${COMPILE_BOOST_TARGET}_iostreams ${COMPILE_BOOST_TARGET}Project)
set_target_properties(${COMPILE_BOOST_TARGET}_iostreams PROPERTIES IMPORTED_LOCATION "${BOOST_INSTALL_DIR}/lib/libboost_iostreams.a")
add_library(${COMPILE_BOOST_TARGET}_serialization STATIC IMPORTED)
add_dependencies(${COMPILE_BOOST_TARGET}_serialization ${COMPILE_BOOST_TARGET}Project)
set_target_properties(${COMPILE_BOOST_TARGET}_serialization PROPERTIES IMPORTED_LOCATION "${BOOST_INSTALL_DIR}/lib/libboost_serialization.a")
add_library(${COMPILE_BOOST_TARGET}_system STATIC IMPORTED)
add_dependencies(${COMPILE_BOOST_TARGET}_system ${COMPILE_BOOST_TARGET}Project)
set_target_properties(${COMPILE_BOOST_TARGET}_system PROPERTIES IMPORTED_LOCATION "${BOOST_INSTALL_DIR}/lib/libboost_system.a")
add_library(${COMPILE_BOOST_TARGET} INTERFACE)
target_include_directories(${COMPILE_BOOST_TARGET} SYSTEM INTERFACE ${BOOST_INSTALL_DIR}/include)
target_link_libraries(${COMPILE_BOOST_TARGET} INTERFACE ${COMPILE_BOOST_TARGET}_context ${COMPILE_BOOST_TARGET}_filesystem ${COMPILE_BOOST_TARGET}_iostreams)
target_link_libraries(${COMPILE_BOOST_TARGET} INTERFACE ${COMPILE_BOOST_TARGET}_context ${COMPILE_BOOST_TARGET}_filesystem ${COMPILE_BOOST_TARGET}_iostreams ${COMPILE_BOOST_TARGET}_system ${COMPILE_BOOST_TARGET}_serialization)
endfunction(compile_boost)
@ -135,18 +145,18 @@ if(WIN32)
# properly for config mode. So we use the old way on Windows
# find_package(Boost 1.72.0 EXACT QUIET REQUIRED CONFIG PATHS ${BOOST_HINT_PATHS})
# I think depending on the cmake version this will cause weird warnings
find_package(Boost 1.72 COMPONENTS filesystem iostreams)
find_package(Boost 1.78 COMPONENTS filesystem iostreams serialization system)
add_library(boost_target INTERFACE)
target_link_libraries(boost_target INTERFACE Boost::boost Boost::filesystem Boost::iostreams)
target_link_libraries(boost_target INTERFACE Boost::boost Boost::filesystem Boost::iostreams Boost::serialization Boost::system)
return()
endif()
find_package(Boost 1.78.0 EXACT QUIET COMPONENTS context filesystem iostreams CONFIG PATHS ${BOOST_HINT_PATHS})
find_package(Boost 1.78.0 EXACT QUIET COMPONENTS context filesystem iostreams serialization system CONFIG PATHS ${BOOST_HINT_PATHS})
set(FORCE_BOOST_BUILD OFF CACHE BOOL "Forces cmake to build boost and ignores any installed boost")
if(Boost_FOUND AND Boost_filesystem_FOUND AND Boost_context_FOUND AND Boost_iostreams_FOUND AND NOT FORCE_BOOST_BUILD)
if(Boost_FOUND AND Boost_filesystem_FOUND AND Boost_context_FOUND AND Boost_iostreams_FOUND AND Boost_system_FOUND AND Boost_serialization_FOUND AND NOT FORCE_BOOST_BUILD)
add_library(boost_target INTERFACE)
target_link_libraries(boost_target INTERFACE Boost::boost Boost::context Boost::filesystem Boost::iostreams)
target_link_libraries(boost_target INTERFACE Boost::boost Boost::context Boost::filesystem Boost::iostreams Boost::serialization Boost::system)
elseif(WIN32)
message(FATAL_ERROR "Could not find Boost")
else()

View File

@ -178,6 +178,13 @@ else()
add_compile_options(-ggdb1)
endif()
if(CLANG)
# The default DWARF 5 format does not play nicely with GNU Binutils 2.39 and earlier, resulting
# in tools like addr2line omitting line numbers. We can consider removing this once we are able
# to use a version that has a fix.
add_compile_options(-gdwarf-4)
endif()
if(NOT FDB_RELEASE)
# Enable compression of the debug sections. This reduces the size of the binaries several times.
# We do not enable it release builds, because CPack fails to generate debuginfo packages when

View File

@ -1,5 +1,5 @@
--index-url https://pypi.python.org/simple
setuptools==65.3.0
setuptools==65.5.1
sphinx==5.1.1
sphinx-bootstrap-theme==0.8.1
docutils==0.19

View File

@ -902,6 +902,13 @@
"num_tenants" : 1, // on data cluster, local count; on management cluster, total metacluster count
"num_tenant_groups" : 10,
"tenant_group_capacity" : 20,
},
"idempotency_ids" : {
"size_bytes" : 0, // An estimate of the current number of bytes used in the database to store idempotency ids.
"expired_version" : 0, // The commit status of a transaction whose commit version could be <= expired_version can no longer be determined.
"expired_age" : 0, // The age in seconds of expired_version.
"oldest_id_version" : 0, // The version of the oldest idempotency id still stored in the database.
"oldest_id_age" : 0 // The age in seconds of the oldest_id_version.
}
},
"client":{

View File

@ -1338,6 +1338,10 @@ static RangeResult mergeDeltaStreams(const BlobGranuleChunkRef& chunk,
ASSERT(streams.size() < std::numeric_limits<int16_t>::max());
ASSERT(startClears.size() == streams.size());
if (streams.empty()) {
return RangeResult{};
}
int prefixLen = commonPrefixLength(chunk.keyRange.begin, chunk.keyRange.end);
// next element for each stream

View File

@ -647,12 +647,13 @@ struct EncryptedRangeFileWriter : public IRangeFileWriter {
Reference<BlobCipherKey> cipherKey = wait(refreshKey(self, self->cipherKeys.textCipherKey->getDomainId()));
self->cipherKeys.textCipherKey = cipherKey;
}
EncryptBlobCipherAes265Ctr encryptor(self->cipherKeys.textCipherKey,
self->cipherKeys.headerCipherKey,
self->cipherKeys.ivRef.begin(),
AES_256_IV_LENGTH,
ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE,
BlobCipherMetrics::BACKUP);
EncryptBlobCipherAes265Ctr encryptor(
self->cipherKeys.textCipherKey,
self->cipherKeys.headerCipherKey,
self->cipherKeys.ivRef.begin(),
AES_256_IV_LENGTH,
getEncryptAuthTokenMode(EncryptAuthTokenMode::ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE),
BlobCipherMetrics::BACKUP);
Arena arena;
int64_t payloadSize = self->wPtr - self->dataPayloadStart;
auto encryptedData = encryptor.encrypt(self->dataPayloadStart, payloadSize, self->encryptHeader, arena);

View File

@ -232,6 +232,43 @@ ACTOR static Future<Optional<Key>> getBoundary(Reference<ReadYourWritesTransacti
return result.front().key;
}
ACTOR Future<JsonBuilderObject> getIdmpKeyStatus(Database db) {
state Reference<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(db);
state int64_t size;
state IdempotencyIdsExpiredVersion expired;
state KeyBackedObjectProperty<IdempotencyIdsExpiredVersion, _Unversioned> expiredKey(idempotencyIdsExpiredVersion,
Unversioned());
state int64_t oldestIdVersion = 0;
state int64_t oldestIdTime = 0;
loop {
try {
tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::READ_LOCK_AWARE);
wait(store(size, tr->getEstimatedRangeSizeBytes(idempotencyIdKeys)) &&
store(expired, expiredKey.getD(tr)) &&
success(getBoundary(tr, idempotencyIdKeys, Oldest::True, &oldestIdVersion, &oldestIdTime)));
JsonBuilderObject result;
result["size_bytes"] = size;
if (expired.expired != 0) {
result["expired_version"] = expired.expired;
}
if (expired.expiredTime != 0) {
result["expired_age"] = int64_t(now()) - expired.expiredTime;
}
if (oldestIdVersion != 0) {
result["oldest_id_version"] = oldestIdVersion;
}
if (oldestIdTime != 0) {
result["oldest_id_age"] = int64_t(now()) - oldestIdTime;
}
return result;
} catch (Error& e) {
wait(tr->onError(e));
}
}
}
ACTOR Future<Void> cleanIdempotencyIds(Database db, double minAgeSeconds) {
state int64_t idmpKeySize;
state int64_t candidateDeleteSize;
@ -315,8 +352,10 @@ ACTOR Future<Void> cleanIdempotencyIds(Database db, double minAgeSeconds) {
if (!finalRange.empty()) {
tr->addReadConflictRange(finalRange);
tr->clear(finalRange);
tr->set(idempotencyIdsExpiredVersion,
ObjectWriter::toValue(IdempotencyIdsExpiredVersion{ candidateDeleteVersion }, Unversioned()));
tr->set(
idempotencyIdsExpiredVersion,
ObjectWriter::toValue(IdempotencyIdsExpiredVersion{ candidateDeleteVersion, candidateDeleteTime },
Unversioned()));
TraceEvent("IdempotencyIdsCleanerAttempt")
.detail("Range", finalRange.toString())
.detail("IdmpKeySizeEstimate", idmpKeySize)
@ -332,4 +371,4 @@ ACTOR Future<Void> cleanIdempotencyIds(Database db, double minAgeSeconds) {
}
}
return Void();
}
}

View File

@ -1537,13 +1537,11 @@ void MultiVersionTransaction::setOption(FDBTransactionOptions::Option option, Op
persistentOptions.emplace_back(option, value.castTo<Standalone<StringRef>>());
}
if (itr->first == FDBTransactionOptions::TIMEOUT) {
setTimeout(value);
}
auto tr = getTransaction();
if (tr.transaction) {
tr.transaction->setOption(option, value);
} else if (itr->first == FDBTransactionOptions::TIMEOUT) {
setTimeout(value);
}
}

View File

@ -11026,11 +11026,12 @@ ACTOR Future<bool> blobRestoreActor(Reference<DatabaseContext> cx, KeyRange rang
try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
state Key key = blobRestoreCommandKeyFor(range);
Optional<Value> value = wait(tr->get(key));
if (value.present()) {
Standalone<BlobRestoreStatus> status = decodeBlobRestoreStatus(value.get());
if (status.progress < 100) {
if (status.phase != BlobRestorePhase::DONE) {
return false; // stop if there is in-progress restore.
}
}

View File

@ -602,8 +602,6 @@ const KeyRef JSONSchemas::statusSchema = R"statusSchema(
"description":"abc"
}
],
)statusSchema"
R"statusSchema(
"recovery_state":{
"seconds_since_last_recovered":1,
"required_resolvers":1,
@ -976,6 +974,13 @@ const KeyRef JSONSchemas::statusSchema = R"statusSchema(
"num_tenants":0,
"num_tenant_groups":10,
"tenant_group_capacity":20
},
"idempotency_ids":{
"size_bytes": 0,
"expired_version": 0,
"expired_age": 0,
"oldest_id_version": 0,
"oldest_id_age": 0
}
},
"client":{

View File

@ -245,7 +245,11 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( ALL_DATA_REMOVED_DELAY, 1.0 );
init( INITIAL_FAILURE_REACTION_DELAY, 30.0 ); if( randomize && BUGGIFY ) INITIAL_FAILURE_REACTION_DELAY = 0.0;
init( CHECK_TEAM_DELAY, 30.0 );
init( PERPETUAL_WIGGLE_DELAY, 50.0 );
// This is a safety knob to avoid busy spinning and the case a small cluster don't have enough space when excluding and including too fast. The basic idea is let PW wait for the re-included storage to take on data before wiggling the next one.
// This knob's ideal value would vary by cluster based on its size and disk type. In the meanwhile, the wiggle will also wait until the storage load is almost (85%) balanced.
init( PERPETUAL_WIGGLE_DELAY, 60 );
init( PERPETUAL_WIGGLE_SMALL_LOAD_RATIO, 10 );
init( PERPETUAL_WIGGLE_MIN_BYTES_BALANCE_RATIO, 0.85 );
init( PERPETUAL_WIGGLE_DISABLE_REMOVER, true );
init( LOG_ON_COMPLETION_DELAY, DD_QUEUE_LOGGING_INTERVAL );
init( BEST_TEAM_MAX_TEAM_TRIES, 10 );
@ -303,6 +307,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( TENANT_CACHE_STORAGE_QUOTA_REFRESH_INTERVAL, 10 ); if( randomize && BUGGIFY ) TENANT_CACHE_STORAGE_QUOTA_REFRESH_INTERVAL = deterministicRandom()->randomInt(1, 10);
init( TENANT_CACHE_STORAGE_USAGE_TRACE_INTERVAL, 300 );
init( CP_FETCH_TENANTS_OVER_STORAGE_QUOTA_INTERVAL, 5 ); if( randomize && BUGGIFY ) CP_FETCH_TENANTS_OVER_STORAGE_QUOTA_INTERVAL = deterministicRandom()->randomInt(1, 10);
init( DD_BUILD_EXTRA_TEAMS_OVERRIDE, 10 ); if( randomize && BUGGIFY ) DD_BUILD_EXTRA_TEAMS_OVERRIDE = 2;
// TeamRemover
init( TR_FLAG_DISABLE_MACHINE_TEAM_REMOVER, false ); if( randomize && BUGGIFY ) TR_FLAG_DISABLE_MACHINE_TEAM_REMOVER = deterministicRandom()->random01() < 0.1 ? true : false; // false by default. disable the consistency check when it's true
@ -388,6 +393,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( ROCKSDB_WRITER_THREAD_PRIORITY, 0 );
init( ROCKSDB_BACKGROUND_PARALLELISM, 4 );
init( ROCKSDB_READ_PARALLELISM, 4 );
init( ROCKSDB_CHECKPOINT_READER_PARALLELISM, 4 );
// If true, do not process and store RocksDB logs
init( ROCKSDB_MUTE_LOGS, true );
// Use a smaller memtable in simulation to avoid OOMs.
@ -407,6 +413,8 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( ROCKSDB_READ_VALUE_TIMEOUT, 5.0 ); if (isSimulated) ROCKSDB_READ_VALUE_TIMEOUT = 5 * 60;
init( ROCKSDB_READ_VALUE_PREFIX_TIMEOUT, 5.0 ); if (isSimulated) ROCKSDB_READ_VALUE_PREFIX_TIMEOUT = 5 * 60;
init( ROCKSDB_READ_RANGE_TIMEOUT, 5.0 ); if (isSimulated) ROCKSDB_READ_RANGE_TIMEOUT = 5 * 60;
init( ROCKSDB_READ_CHECKPOINT_TIMEOUT, 60.0 ); if (isSimulated) ROCKSDB_READ_CHECKPOINT_TIMEOUT = 5 * 60;
init( ROCKSDB_CHECKPOINT_READ_AHEAD_SIZE, 2 << 20 ); // 2M
init( ROCKSDB_READ_QUEUE_WAIT, 1.0 );
init( ROCKSDB_READ_QUEUE_HARD_MAX, 1000 );
init( ROCKSDB_READ_QUEUE_SOFT_MAX, 500 );
@ -1031,7 +1039,8 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( BLOB_FULL_RESTORE_MODE, false );
init( BLOB_MIGRATOR_CHECK_INTERVAL, isSimulated ? 1.0 : 5.0);
init( BLOB_MANIFEST_RW_ROWS, isSimulated ? 10 : 1000);
init( BLOB_RESTORE_MLOGS_URL, isSimulated ? "file://simfdb/fdbblob/mlogs" : "");
init( BLOB_RESTORE_MLOGS_URL, isSimulated ? "file://simfdb/backups/" : "");
init( BLOB_MIGRATOR_ERROR_RETRIES, 20);
init( BGCC_TIMEOUT, isSimulated ? 10.0 : 120.0 );
init( BGCC_MIN_INTERVAL, isSimulated ? 1.0 : 10.0 );

View File

@ -314,19 +314,28 @@ struct BlobManifest {
};
// Defines blob restore status
enum BlobRestorePhase { INIT = 0, LOAD_MANIFEST = 1, MANIFEST_DONE = 2, MIGRATE = 3, APPLY_MLOGS = 4, DONE = 5 };
enum BlobRestorePhase {
INIT = 0,
STARTING_MIGRATOR = 1,
LOADING_MANIFEST = 2,
LOADED_MANIFEST = 3,
COPYING_DATA = 4,
APPLYING_MLOGS = 5,
DONE = 6,
ERROR = 7
};
struct BlobRestoreStatus {
constexpr static FileIdentifier file_identifier = 378657;
BlobRestorePhase phase;
int progress;
int status;
BlobRestoreStatus() : phase(BlobRestorePhase::INIT){};
BlobRestoreStatus(BlobRestorePhase pha) : phase(pha), progress(0){};
BlobRestoreStatus(BlobRestorePhase pha, int prog) : phase(pha), progress(prog){};
BlobRestoreStatus(BlobRestorePhase pha) : phase(pha), status(0){};
BlobRestoreStatus(BlobRestorePhase pha, int prog) : phase(pha), status(prog){};
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, phase, progress);
serializer(ar, phase, status);
}
};

View File

@ -29,6 +29,7 @@
#pragma once
#include "fdbclient/FDBTypes.h"
#include "fdbclient/JsonBuilder.h"
#include "fdbclient/PImpl.h"
#include "flow/Arena.h"
#include "flow/IRandom.h"
@ -46,10 +47,11 @@ struct IdempotencyIdsExpiredVersion {
// Any version at or below expired might have had its idempotency id expired. Any version greater than `expired`
// definitely has not had it's idempotency id expired.
Version expired = 0;
int64_t expiredTime = 0;
template <class Archive>
void serialize(Archive& ar) {
serializer(ar, expired);
serializer(ar, expired, expiredTime);
}
};
@ -187,6 +189,8 @@ KeyRangeRef makeIdempotencySingleKeyRange(Arena& arena, Version version, uint8_t
void decodeIdempotencyKey(KeyRef key, Version& commitVersion, uint8_t& highOrderBatchIndex);
ACTOR Future<JsonBuilderObject> getIdmpKeyStatus(Database db);
// Delete zero or more idempotency ids older than minAgeSeconds
//
// Normally idempotency ids are deleted as part of the normal commit process, so this only needs to clean ids that

View File

@ -1440,7 +1440,8 @@ struct DeleteTenantImpl {
Reference<typename DB::TransactionT> tr) {
state Optional<TenantMapEntry> tenantEntry = wait(tryGetTenantTransaction(tr, self->tenantName));
if (!tenantEntry.present() || tenantEntry.get().id != self->tenantId) {
if (!tenantEntry.present() || tenantEntry.get().id != self->tenantId ||
tenantEntry.get().tenantState == TenantState::RENAMING_TO) {
throw tenant_not_found();
}
@ -1452,11 +1453,6 @@ struct DeleteTenantImpl {
}
if (tenantEntry.get().tenantState != TenantState::REMOVING) {
// Disallow removing the "new" name of a renamed tenant before it completes
if (tenantEntry.get().tenantState == TenantState::RENAMING_TO) {
throw tenant_not_found();
}
state TenantMapEntry updatedEntry = tenantEntry.get();
// Check if we are deleting a tenant in the middle of a rename
updatedEntry.tenantState = TenantState::REMOVING;

View File

@ -186,9 +186,14 @@ public:
double METRIC_DELAY;
double ALL_DATA_REMOVED_DELAY;
double INITIAL_FAILURE_REACTION_DELAY;
double CHECK_TEAM_DELAY;
double PERPETUAL_WIGGLE_DELAY;
bool PERPETUAL_WIGGLE_DISABLE_REMOVER;
double CHECK_TEAM_DELAY; // Perpetual wiggle check cluster team healthy
double PERPETUAL_WIGGLE_SMALL_LOAD_RATIO; // If the average load of storage server is less than this ratio * average
// shard bytes, the perpetual wiggle won't consider the available space
// load balance in the cluster
double PERPETUAL_WIGGLE_MIN_BYTES_BALANCE_RATIO; // target min : average space load balance ratio after re-include
// before perpetual wiggle will start the next wiggle
double PERPETUAL_WIGGLE_DELAY; // The max interval between the last wiggle finish and the next wiggle start
bool PERPETUAL_WIGGLE_DISABLE_REMOVER; // Whether the start of perpetual wiggle replace team remover
double LOG_ON_COMPLETION_DELAY;
int BEST_TEAM_MAX_TEAM_TRIES;
int BEST_TEAM_OPTION_COUNT;
@ -264,6 +269,7 @@ public:
double DD_FAILURE_TIME;
double DD_ZERO_HEALTHY_TEAM_DELAY;
int DD_BUILD_EXTRA_TEAMS_OVERRIDE; // build extra teams to allow data movement to progress. must be larger than 0
// Run storage enginee on a child process on the same machine with storage process
bool REMOTE_KV_STORE;
@ -321,6 +327,7 @@ public:
int ROCKSDB_WRITER_THREAD_PRIORITY;
int ROCKSDB_BACKGROUND_PARALLELISM;
int ROCKSDB_READ_PARALLELISM;
int ROCKSDB_CHECKPOINT_READER_PARALLELISM;
int64_t ROCKSDB_MEMTABLE_BYTES;
bool ROCKSDB_LEVEL_STYLE_COMPACTION;
bool ROCKSDB_UNSAFE_AUTO_FSYNC;
@ -332,6 +339,8 @@ public:
double ROCKSDB_READ_VALUE_TIMEOUT;
double ROCKSDB_READ_VALUE_PREFIX_TIMEOUT;
double ROCKSDB_READ_RANGE_TIMEOUT;
double ROCKSDB_READ_CHECKPOINT_TIMEOUT;
int64_t ROCKSDB_CHECKPOINT_READ_AHEAD_SIZE;
double ROCKSDB_READ_QUEUE_WAIT;
int ROCKSDB_READ_QUEUE_SOFT_MAX;
int ROCKSDB_READ_QUEUE_HARD_MAX;
@ -1000,6 +1009,7 @@ public:
double BLOB_MIGRATOR_CHECK_INTERVAL;
int BLOB_MANIFEST_RW_ROWS;
std::string BLOB_RESTORE_MLOGS_URL;
int BLOB_MIGRATOR_ERROR_RETRIES;
// Blob metadata
int64_t BLOB_METADATA_CACHE_TTL;

View File

@ -31,6 +31,8 @@ enum CheckpointFormat {
DataMoveRocksCF = 1,
// For RocksDB, checkpoint generated via rocksdb::Checkpoint::CreateCheckpoint().
RocksDB = 2,
// Checkpoint fetched as key-value pairs.
RocksDBKeyValues = 3,
};
// Metadata of a FDB checkpoint.

View File

@ -245,8 +245,7 @@ Future<Void> AsyncFileCached::changeFileSize(int64_t size) {
// Wait for the page truncations to finish, then truncate the underlying file
// Template types are being provided explicitly because they can't be automatically deduced for some reason.
return mapAsync<Void, std::function<Future<Void>(Void)>, Void>(
waitForAll(actors), [=](Void _) -> Future<Void> { return uncached->truncate(size); });
return mapAsync(waitForAll(actors), [=](Void _) -> Future<Void> { return uncached->truncate(size); });
}
Future<Void> AsyncFileCached::flush() {

View File

@ -24,6 +24,10 @@ add_flow_target(STATIC_LIBRARY NAME fdbrpc
add_flow_target(STATIC_LIBRARY NAME fdbrpc_sampling
SRCS ${FDBRPC_SRCS}
DISABLE_ACTOR_DIAGNOSTICS ${FDBRPC_SRCS_DISABLE_ACTOR_DIAGNOSTICS})
add_flow_target(STATIC_LIBRARY NAME tokensign
SRCS TokenSign/TokenSignStdTypes.cpp
DISABLE_ACTOR_DIAGNOSTICS ${FDBRPC_SRCS_DISABLE_ACTOR_DIAGNOSTICS})
target_link_libraries(tokensign PRIVATE fdbrpc flow)
add_flow_target(LINK_TEST NAME fdbrpclinktest SRCS LinkTest.cpp)
target_link_libraries(fdbrpclinktest PRIVATE fdbrpc rapidjson)

View File

@ -216,7 +216,7 @@ struct SubNetTest {
return IPAddress(arr[0]);
} else {
std::array<unsigned char, 16> res;
memcpy(res.data(), arr, 4);
memcpy(res.data(), arr, 16);
return IPAddress(res);
}
}

View File

@ -289,6 +289,8 @@ ProcessClass::Fitness ProcessClass::machineClassFitness(ClusterRole role) const
return ProcessClass::GoodFit;
case ProcessClass::MasterClass:
return ProcessClass::OkayFit;
case ProcessClass::BlobWorkerClass:
return ProcessClass::OkayFit;
default:
return ProcessClass::NeverAssign;
}

View File

@ -216,7 +216,7 @@ bool TokenCache::validate(TenantNameRef name, StringRef token) {
.detail("From", peer) \
.detail("Reason", reason) \
.detail("CurrentTime", currentTime) \
.detail("Token", token.toStringRef(arena).toStringView())
.detail("Token", toStringRef(arena, token).toStringView())
bool TokenCacheImpl::validateAndAdd(double currentTime, StringRef token, NetworkAddress const& peer) {
Arena arena;
@ -406,7 +406,7 @@ TEST_CASE("/fdbrpc/authz/TokenCache/BadTokens") {
if (TokenCache::instance().validate(validTokenSpec.tenants.get()[0], signedToken)) {
fmt::print("Unexpected successful validation at mutation {}, token spec: {}\n",
mutationDesc,
mutatedTokenSpec.toStringRef(tmpArena).toStringView());
toStringRef(tmpArena, mutatedTokenSpec).toStringView());
ASSERT(false);
}
} else if (i == numBadMutations) {
@ -475,7 +475,7 @@ TEST_CASE("/fdbrpc/authz/TokenCache/GoodTokens") {
signedToken = authz::jwt::signToken(arena, tokenSpec, privateKey);
if (!TokenCache::instance().validate(tokenSpec.tenants.get()[0], signedToken)) {
fmt::print("Unexpected failed token validation, token spec: {}, now: {}\n",
tokenSpec.toStringRef(arena).toStringView(),
toStringRef(arena, tokenSpec).toStringView(),
g_network->timer());
ASSERT(false);
}
@ -483,7 +483,7 @@ TEST_CASE("/fdbrpc/authz/TokenCache/GoodTokens") {
if (TokenCache::instance().validate(tokenSpec.tenants.get()[0], signedToken)) {
fmt::print(
"Unexpected successful token validation after supposedly expiring in cache, token spec: {}, now: {}\n",
tokenSpec.toStringRef(arena).toStringView(),
toStringRef(arena, tokenSpec).toStringView(),
g_network->timer());
ASSERT(false);
}

View File

@ -139,15 +139,6 @@ namespace authz {
using MessageDigestMethod = const EVP_MD*;
Algorithm algorithmFromString(StringRef s) noexcept {
if (s == "RS256"_sr)
return Algorithm::RS256;
else if (s == "ES256"_sr)
return Algorithm::ES256;
else
return Algorithm::UNKNOWN;
}
std::pair<PKeyAlgorithm, MessageDigestMethod> getMethod(Algorithm alg) {
if (alg == Algorithm::RS256) {
return { PKeyAlgorithm::RSA, ::EVP_sha256() };
@ -169,36 +160,6 @@ std::string_view getAlgorithmName(Algorithm alg) {
} // namespace authz
namespace authz::flatbuffers {
SignedTokenRef signToken(Arena& arena, TokenRef token, StringRef keyName, PrivateKey privateKey) {
auto ret = SignedTokenRef{};
auto writer = ObjectWriter([&arena](size_t len) { return new (arena) uint8_t[len]; }, IncludeVersion());
writer.serialize(token);
auto tokenStr = writer.toStringRef();
auto sig = privateKey.sign(arena, tokenStr, *::EVP_sha256());
ret.token = tokenStr;
ret.signature = sig;
ret.keyName = StringRef(arena, keyName);
return ret;
}
bool verifyToken(SignedTokenRef signedToken, PublicKey publicKey) {
return publicKey.verify(signedToken.token, signedToken.signature, *::EVP_sha256());
}
TokenRef makeRandomTokenSpec(Arena& arena, IRandom& rng) {
auto token = TokenRef{};
token.expiresAt = timer_monotonic() * (0.5 + rng.random01());
const auto numTenants = rng.randomInt(1, 3);
for (auto i = 0; i < numTenants; i++) {
token.tenants.push_back(arena, genRandomAlphanumStringRef(arena, rng, MinTenantNameLen, MaxTenantNameLenPlus1));
}
return token;
}
} // namespace authz::flatbuffers
namespace authz::jwt {
template <class FieldType, size_t NameLen>
@ -222,17 +183,20 @@ void appendField(fmt::memory_buffer& b, char const (&name)[NameLen], Optional<Fi
}
}
StringRef TokenRef::toStringRef(Arena& arena) {
StringRef toStringRef(Arena& arena, const TokenRef& tokenSpec) {
auto buf = fmt::memory_buffer();
fmt::format_to(std::back_inserter(buf), "alg={} kid={}", getAlgorithmName(algorithm), keyId.toStringView());
appendField(buf, "iss", issuer);
appendField(buf, "sub", subject);
appendField(buf, "aud", audience);
appendField(buf, "iat", issuedAtUnixTime);
appendField(buf, "exp", expiresAtUnixTime);
appendField(buf, "nbf", notBeforeUnixTime);
appendField(buf, "jti", tokenId);
appendField(buf, "tenants", tenants);
fmt::format_to(std::back_inserter(buf),
"alg={} kid={}",
getAlgorithmName(tokenSpec.algorithm),
tokenSpec.keyId.toStringView());
appendField(buf, "iss", tokenSpec.issuer);
appendField(buf, "sub", tokenSpec.subject);
appendField(buf, "aud", tokenSpec.audience);
appendField(buf, "iat", tokenSpec.issuedAtUnixTime);
appendField(buf, "exp", tokenSpec.expiresAtUnixTime);
appendField(buf, "nbf", tokenSpec.notBeforeUnixTime);
appendField(buf, "jti", tokenSpec.tokenId);
appendField(buf, "tenants", tokenSpec.tenants);
auto str = new (arena) uint8_t[buf.size()];
memcpy(str, buf.data(), buf.size());
return StringRef(str, buf.size());
@ -375,7 +339,7 @@ Optional<StringRef> parseHeaderPart(Arena& arena, TokenRef& token, StringRef b64
if (typValue != "JWT"_sr)
return "'typ' is not 'JWT'"_sr;
auto algValue = StringRef(reinterpret_cast<const uint8_t*>(alg.GetString()), alg.GetStringLength());
auto algType = algorithmFromString(algValue);
auto algType = algorithmFromString(algValue.toStringView());
if (algType == Algorithm::UNKNOWN)
return "Unsupported algorithm"_sr;
token.algorithm = algType;
@ -579,29 +543,6 @@ TokenRef makeRandomTokenSpec(Arena& arena, IRandom& rng, Algorithm alg) {
void forceLinkTokenSignTests() {}
TEST_CASE("/fdbrpc/TokenSign/FlatBuffer") {
const auto numIters = 100;
for (auto i = 0; i < numIters; i++) {
auto arena = Arena();
auto privateKey = mkcert::makeEcP256();
auto publicKey = privateKey.toPublic();
auto& rng = *deterministicRandom();
auto tokenSpec = authz::flatbuffers::makeRandomTokenSpec(arena, rng);
auto keyName = genRandomAlphanumStringRef(arena, rng, MinKeyNameLen, MaxKeyNameLenPlus1);
auto signedToken = authz::flatbuffers::signToken(arena, tokenSpec, keyName, privateKey);
ASSERT(authz::flatbuffers::verifyToken(signedToken, publicKey));
// try tampering with signed token by adding one more tenant
tokenSpec.tenants.push_back(arena,
genRandomAlphanumStringRef(arena, rng, MinTenantNameLen, MaxTenantNameLenPlus1));
auto writer = ObjectWriter([&arena](size_t len) { return new (arena) uint8_t[len]; }, IncludeVersion());
writer.serialize(tokenSpec);
signedToken.token = writer.toStringRef();
ASSERT(!authz::flatbuffers::verifyToken(signedToken, publicKey));
}
printf("%d runs OK\n", numIters);
return Void();
}
TEST_CASE("/fdbrpc/TokenSign/JWT") {
const auto numIters = 100;
for (auto i = 0; i < numIters; i++) {
@ -670,7 +611,7 @@ TEST_CASE("/fdbrpc/TokenSign/JWT/ToStringRef") {
StringRef tenants[2]{ "tenant1"_sr, "tenant2"_sr };
t.tenants = VectorRef<StringRef>(tenants, 2);
auto arena = Arena();
auto tokenStr = t.toStringRef(arena);
auto tokenStr = toStringRef(arena, t);
auto tokenStrExpected =
"alg=ES256 kid=keyId iss=issuer sub=subject aud=[aud1,aud2,aud3] iat=123 exp=456 nbf=789 jti=tokenId tenants=[tenant1,tenant2]"_sr;
if (tokenStr != tokenStrExpected) {
@ -682,73 +623,42 @@ TEST_CASE("/fdbrpc/TokenSign/JWT/ToStringRef") {
return Void();
}
// This unit test takes too long to run in RandomUnitTests.toml
// FIXME: Move this to benchmark to flowbench
/*
TEST_CASE("/fdbrpc/TokenSign/bench") {
auto keyTypes = std::array<StringRef, 2>{ "EC"_sr, "RSA"_sr };
for (auto kty : keyTypes) {
constexpr auto repeat = 5;
constexpr auto numSamples = 10000;
fmt::print("=== {} keys case\n", kty.toString());
auto key = kty == "EC"_sr ? mkcert::makeEcP256() : mkcert::makeRsa4096Bit();
auto pubKey = key.toPublic();
auto& rng = *deterministicRandom();
auto arena = Arena();
auto jwtSpecs = new (arena) authz::jwt::TokenRef[numSamples];
auto fbSpecs = new (arena) authz::flatbuffers::TokenRef[numSamples];
auto jwts = new (arena) StringRef[numSamples];
auto fbs = new (arena) StringRef[numSamples];
for (auto i = 0; i < numSamples; i++) {
jwtSpecs[i] = authz::jwt::makeRandomTokenSpec(
arena, rng, kty == "EC"_sr ? authz::Algorithm::ES256 : authz::Algorithm::RS256);
fbSpecs[i] = authz::flatbuffers::makeRandomTokenSpec(arena, rng);
}
{
auto const jwtSignBegin = timer_monotonic();
for (auto i = 0; i < numSamples; i++) {
jwts[i] = authz::jwt::signToken(arena, jwtSpecs[i], key);
}
auto const jwtSignEnd = timer_monotonic();
fmt::print("JWT Sign : {:.2f} OPS\n", numSamples / (jwtSignEnd - jwtSignBegin));
}
{
auto const jwtVerifyBegin = timer_monotonic();
for (auto rep = 0; rep < repeat; rep++) {
for (auto i = 0; i < numSamples; i++) {
auto verifyOk = authz::jwt::verifyToken(jwts[i], pubKey);
ASSERT(verifyOk);
}
}
auto const jwtVerifyEnd = timer_monotonic();
fmt::print("JWT Verify : {:.2f} OPS\n", repeat * numSamples / (jwtVerifyEnd - jwtVerifyBegin));
}
{
auto tmpArena = Arena();
auto const fbSignBegin = timer_monotonic();
for (auto i = 0; i < numSamples; i++) {
auto fbToken = authz::flatbuffers::signToken(tmpArena, fbSpecs[i], "defaultKey"_sr, key);
auto wr = ObjectWriter([&arena](size_t len) { return new (arena) uint8_t[len]; }, Unversioned());
wr.serialize(fbToken);
fbs[i] = wr.toStringRef();
}
auto const fbSignEnd = timer_monotonic();
fmt::print("FlatBuffers Sign : {:.2f} OPS\n", numSamples / (fbSignEnd - fbSignBegin));
}
{
auto const fbVerifyBegin = timer_monotonic();
for (auto rep = 0; rep < repeat; rep++) {
for (auto i = 0; i < numSamples; i++) {
auto signedToken = ObjectReader::fromStringRef<Standalone<authz::flatbuffers::SignedTokenRef>>(
fbs[i], Unversioned());
auto verifyOk = authz::flatbuffers::verifyToken(signedToken, pubKey);
ASSERT(verifyOk);
}
}
auto const fbVerifyEnd = timer_monotonic();
fmt::print("FlatBuffers Verify : {:.2f} OPS\n", repeat * numSamples / (fbVerifyEnd - fbVerifyBegin));
}
}
return Void();
auto keyTypes = std::array<StringRef, 1>{ "EC"_sr };
for (auto kty : keyTypes) {
constexpr auto repeat = 5;
constexpr auto numSamples = 10000;
fmt::print("=== {} keys case\n", kty.toString());
auto key = kty == "EC"_sr ? mkcert::makeEcP256() : mkcert::makeRsa4096Bit();
auto pubKey = key.toPublic();
auto& rng = *deterministicRandom();
auto arena = Arena();
auto jwtSpecs = new (arena) authz::jwt::TokenRef[numSamples];
auto jwts = new (arena) StringRef[numSamples];
for (auto i = 0; i < numSamples; i++) {
jwtSpecs[i] = authz::jwt::makeRandomTokenSpec(
arena, rng, kty == "EC"_sr ? authz::Algorithm::ES256 : authz::Algorithm::RS256);
}
{
auto const jwtSignBegin = timer_monotonic();
for (auto i = 0; i < numSamples; i++) {
jwts[i] = authz::jwt::signToken(arena, jwtSpecs[i], key);
}
auto const jwtSignEnd = timer_monotonic();
fmt::print("JWT Sign : {:.2f} OPS\n", numSamples / (jwtSignEnd - jwtSignBegin));
}
{
auto const jwtVerifyBegin = timer_monotonic();
for (auto rep = 0; rep < repeat; rep++) {
for (auto i = 0; i < numSamples; i++) {
auto [verifyOk, errorMsg] = authz::jwt::verifyToken(jwts[i], pubKey);
ASSERT(!errorMsg.present());
ASSERT(verifyOk);
}
}
auto const jwtVerifyEnd = timer_monotonic();
fmt::print("JWT Verify : {:.2f} OPS\n", repeat * numSamples / (jwtVerifyEnd - jwtVerifyBegin));
}
}
return Void();
}
*/

View File

@ -0,0 +1,109 @@
/*
* TokenSignStlTypes.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "fdbrpc/TokenSign.h"
#include "fdbrpc/TokenSignStdTypes.h"
#include "flow/PKey.h"
#include "flow/MkCert.h"
#include <stdexcept>
namespace {
// converts std::optional<STANDARD_TYPE(S)> to Optional<FLOW_TYPE(T)>
template <class T, class S>
void convertAndAssign(Arena& arena, Optional<T>& to, const std::optional<S>& from) {
if constexpr (std::is_same_v<S, std::vector<std::string>>) {
static_assert(std::is_same_v<T, VectorRef<StringRef>>,
"Source type std::vector<std::string> must convert to VectorRef<StringRef>");
if (from.has_value()) {
const auto& value = from.value();
if (value.empty()) {
to = VectorRef<StringRef>();
} else {
// no need to deep copy string because we have the underlying memory for the duration of token signing.
auto buf = new (arena) StringRef[value.size()];
for (auto i = 0u; i < value.size(); i++) {
buf[i] = StringRef(value[i]);
}
to = VectorRef<StringRef>(buf, value.size());
}
}
} else if constexpr (std::is_same_v<S, std::string>) {
static_assert(std::is_same_v<T, StringRef>, "Source type std::string must convert to StringRef");
if (from.has_value()) {
const auto& value = from.value();
// no need to deep copy string because we have the underlying memory for the duration of token signing.
to = StringRef(value);
}
} else {
static_assert(
std::is_same_v<S, T>,
"Source types that aren't std::vector<std::string> or std::string must have the same destination type");
static_assert(std::is_trivially_copy_assignable_v<S>,
"Source types that aren't std::vector<std::string> or std::string must not use heap memory");
if (from.has_value()) {
to = from.value();
}
}
}
} // anonymous namespace
namespace authz::jwt::stdtypes {
std::string makeEcP256PrivateKeyPem() {
try {
PrivateKey pk = mkcert::makeEcP256();
auto tmpArena = Arena();
auto pem = pk.writePem(tmpArena);
return pem.toString();
} catch (Error& e) {
throw std::runtime_error(e.name());
}
}
std::string signToken(const TokenSpec& tokenSpec, const std::string& privateKeyPem) {
try {
auto arena = Arena();
auto privateKey = PrivateKey(PemEncoded{}, StringRef(privateKeyPem));
// translate TokenSpec (uses STL types) to TokenRef (uses Flow types)
auto t = authz::jwt::TokenRef{};
t.algorithm = tokenSpec.algorithm;
t.keyId = StringRef(tokenSpec.keyId);
convertAndAssign(arena, t.issuer, tokenSpec.issuer);
convertAndAssign(arena, t.subject, tokenSpec.subject);
convertAndAssign(arena, t.audience, tokenSpec.audience);
convertAndAssign(arena, t.issuedAtUnixTime, tokenSpec.issuedAtUnixTime);
convertAndAssign(arena, t.expiresAtUnixTime, tokenSpec.expiresAtUnixTime);
convertAndAssign(arena, t.notBeforeUnixTime, tokenSpec.notBeforeUnixTime);
convertAndAssign(arena, t.tokenId, tokenSpec.tokenId);
convertAndAssign(arena, t.tenants, tokenSpec.tenants);
return signToken(arena, t, privateKey).toString();
} catch (Error& e) {
if (e.code() == error_code_pkey_decode_error) {
// bad PEM
throw std::invalid_argument(e.name());
} else {
throw std::runtime_error(e.name());
}
}
}
} // namespace authz::jwt::stdtypes

View File

@ -74,8 +74,8 @@ public:
// Wait for diskDelay before submitting the I/O
// Template types are being provided explicitly because they can't be automatically deduced for some reason.
// Capture file by value in case this is destroyed during the delay
return mapAsync<Void, std::function<Future<int>(Void)>, int>(
delay(diskDelay), [=, file = file](Void _) -> Future<int> { return file->read(data, length, offset); });
return mapAsync(delay(diskDelay),
[=, file = file](Void _) -> Future<int> { return file->read(data, length, offset); });
}
Future<Void> write(void const* data, int length, int64_t offset) override {
@ -115,20 +115,19 @@ public:
// Wait for diskDelay before submitting the I/O
// Capture file by value in case this is destroyed during the delay
return mapAsync<Void, std::function<Future<Void>(Void)>, Void>(
delay(getDelay()), [=, file = file](Void _) -> Future<Void> {
if (pdata) {
return map(holdWhile(arena, file->write(pdata, length, offset)),
[corruptedBlock, file = file](auto res) {
if (g_network->isSimulated()) {
g_simulator->corruptedBlocks.emplace(file->getFilename(), corruptedBlock);
}
return res;
});
}
return mapAsync(delay(getDelay()), [=, file = file](Void _) -> Future<Void> {
if (pdata) {
return map(holdWhile(arena, file->write(pdata, length, offset)),
[corruptedBlock, file = file](auto res) {
if (g_network->isSimulated()) {
g_simulator->corruptedBlocks.emplace(file->getFilename(), corruptedBlock);
}
return res;
});
}
return file->write(data, length, offset);
});
return file->write(data, length, offset);
});
}
Future<Void> truncate(int64_t size) override {
@ -138,17 +137,16 @@ public:
// Wait for diskDelay before submitting the I/O
// Capture file by value in case this is destroyed during the delay
return mapAsync<Void, std::function<Future<Void>(Void)>, Void>(
delay(diskDelay), [size, file = file](Void _) -> Future<Void> {
constexpr auto maxBlockValue =
std::numeric_limits<decltype(g_simulator->corruptedBlocks)::key_type::second_type>::max();
auto firstDeletedBlock =
g_simulator->corruptedBlocks.lower_bound(std::make_pair(file->getFilename(), size / 4096));
auto lastFileBlock =
g_simulator->corruptedBlocks.upper_bound(std::make_pair(file->getFilename(), maxBlockValue));
g_simulator->corruptedBlocks.erase(firstDeletedBlock, lastFileBlock);
return file->truncate(size);
});
return mapAsync(delay(diskDelay), [size, file = file](Void _) -> Future<Void> {
constexpr auto maxBlockValue =
std::numeric_limits<decltype(g_simulator->corruptedBlocks)::key_type::second_type>::max();
auto firstDeletedBlock =
g_simulator->corruptedBlocks.lower_bound(std::make_pair(file->getFilename(), size / 4096));
auto lastFileBlock =
g_simulator->corruptedBlocks.upper_bound(std::make_pair(file->getFilename(), maxBlockValue));
g_simulator->corruptedBlocks.erase(firstDeletedBlock, lastFileBlock);
return file->truncate(size);
});
}
Future<Void> sync() override {
@ -158,8 +156,7 @@ public:
// Wait for diskDelay before submitting the I/O
// Capture file by value in case this is destroyed during the delay
return mapAsync<Void, std::function<Future<Void>(Void)>, Void>(
delay(diskDelay), [=, file = file](Void _) -> Future<Void> { return file->sync(); });
return mapAsync(delay(diskDelay), [=, file = file](Void _) -> Future<Void> { return file->sync(); });
}
Future<int64_t> size() const override {
@ -169,8 +166,7 @@ public:
// Wait for diskDelay before submitting the I/O
// Capture file by value in case this is destroyed during the delay
return mapAsync<Void, std::function<Future<int64_t>(Void)>, int64_t>(
delay(diskDelay), [=, file = file](Void _) -> Future<int64_t> { return file->size(); });
return mapAsync(delay(diskDelay), [=, file = file](Void _) -> Future<int64_t> { return file->size(); });
}
int64_t debugFD() const override { return file->debugFD(); }

View File

@ -299,14 +299,13 @@ struct RequestData : NonCopyable {
requestStarted = false;
if (backoff > 0) {
response = mapAsync<Void, std::function<Future<Reply>(Void)>, Reply>(
delay(backoff), [this, stream, &request, model, alternatives, channel](Void _) {
requestStarted = true;
modelHolder = Reference<ModelHolder>(new ModelHolder(model, stream->getEndpoint().token.first()));
Future<Reply> resp = stream->tryGetReply(request);
maybeDuplicateTSSRequest(stream, request, model, resp, alternatives, channel);
return resp;
});
response = mapAsync(delay(backoff), [this, stream, &request, model, alternatives, channel](Void _) {
requestStarted = true;
modelHolder = Reference<ModelHolder>(new ModelHolder(model, stream->getEndpoint().token.first()));
Future<Reply> resp = stream->tryGetReply(request);
maybeDuplicateTSSRequest(stream, request, model, resp, alternatives, channel);
return resp;
});
} else {
requestStarted = true;
modelHolder = Reference<ModelHolder>(new ModelHolder(model, stream->getEndpoint().token.first()));

View File

@ -18,89 +18,32 @@
* limitations under the License.
*/
#pragma once
#ifndef FDBRPC_TOKEN_SIGN_H
#define FDBRPC_TOKEN_SIGN_H
#pragma once
#include "flow/network.h"
#include "flow/Arena.h"
#include "flow/FastRef.h"
#include "flow/FileIdentifier.h"
#include "fdbrpc/TenantInfo.h"
#include "flow/PKey.h"
namespace authz {
enum class Algorithm : int {
RS256,
ES256,
UNKNOWN,
};
Algorithm algorithmFromString(StringRef s) noexcept;
} // namespace authz
namespace authz::flatbuffers {
struct TokenRef {
static constexpr FileIdentifier file_identifier = 1523118;
double expiresAt;
VectorRef<StringRef> tenants;
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, expiresAt, tenants);
}
};
struct SignedTokenRef {
static constexpr FileIdentifier file_identifier = 5916732;
StringRef token;
StringRef keyName;
StringRef signature;
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, token, keyName, signature);
}
int expectedSize() const { return token.size() + keyName.size() + signature.size(); }
};
SignedTokenRef signToken(Arena& arena, TokenRef token, StringRef keyName, PrivateKey privateKey);
bool verifyToken(SignedTokenRef signedToken, PublicKey publicKey);
} // namespace authz::flatbuffers
#include "fdbrpc/TokenSpec.h"
#include <string>
#include <vector>
namespace authz::jwt {
// Given S = concat(B64UrlEnc(headerJson), ".", B64UrlEnc(payloadJson)),
// JWT is concat(S, ".", B64UrlEnc(sign(S, PrivateKey))).
// Below we refer to S as "sign input"
namespace detail {
// This struct is not meant to be flatbuffer-serialized
// This is a parsed, flattened view of T and signature
struct TokenRef {
// header part ("typ": "JWT" implicitly enforced)
Algorithm algorithm; // alg
StringRef keyId; // kid
// payload part
Optional<StringRef> issuer; // iss
Optional<StringRef> subject; // sub
Optional<VectorRef<StringRef>> audience; // aud
Optional<uint64_t> issuedAtUnixTime; // iat
Optional<uint64_t> expiresAtUnixTime; // exp
Optional<uint64_t> notBeforeUnixTime; // nbf
Optional<StringRef> tokenId; // jti
Optional<VectorRef<StringRef>> tenants; // tenants
// signature part
StringRef signature;
// work around the fact that VectorRef takes more than one template parameter
template <typename T>
using VectorRefAlias = VectorRef<T>;
// print each non-signature field in non-JSON, human-readable format e.g. for trace
StringRef toStringRef(Arena& arena);
};
} // namespace detail
using TokenRef = BasicTokenSpec<StringRef, detail::VectorRefAlias, Optional>;
// print each non-signature field in non-JSON, human-readable format e.g. for trace
StringRef toStringRef(Arena& arena, const TokenRef& tokenSpec);
StringRef makeSignInput(Arena& arena, const TokenRef& tokenSpec);

View File

@ -0,0 +1,54 @@
/*
* TokenSignStlTypes.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
#ifndef FDBRPC_TOKEN_SIGN_STD_TYPES_H
#define FDBRPC_TOKEN_SIGN_STD_TYPES_H
#include "fdbrpc/TokenSpec.h"
#include <string>
#include <vector>
// Below functions build as a library separate from fdbrpc
// The intent is to re-use the key/token generation part in a way that the input, the output,
// and possible exceptions are all standard types, such that it can be used outside the FDB/Flow world,
// especially for testing and benchmarking purposes
namespace authz::jwt::stdtypes {
namespace detail {
// work around the fact that std::vector takes more than one template parameter
template <class T>
using VectorAlias = std::vector<T>;
} // namespace detail
using TokenSpec = BasicTokenSpec<std::string, detail::VectorAlias>;
// Generate an elliptic curve private key on a P-256 curve, and serialize it as PEM.
std::string makeEcP256PrivateKeyPem();
// If this function was to take PrivateKey class as parameter,
// users of this library would need to link to flow in order to use it.
// To avoid that, keep the key input as PEM and suffer the parsing overhead.
std::string signToken(const TokenSpec& tokenSpec, const std::string& privateKeyPem);
} // namespace authz::jwt::stdtypes
#endif // FDBRPC_TOKEN_SIGN_STD_TYPES_H

View File

@ -0,0 +1,77 @@
/*
* TokenSpec.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef FDBRPC_TOKEN_SPEC_H
#define FDBRPC_TOKEN_SPEC_H
#pragma once
#include <cstdint>
#include <optional>
#include <string_view>
namespace authz {
enum class Algorithm : int {
RS256,
ES256,
UNKNOWN,
};
inline Algorithm algorithmFromString(std::string_view s) noexcept {
if (s == "RS256")
return Algorithm::RS256;
else if (s == "ES256")
return Algorithm::ES256;
else
return Algorithm::UNKNOWN;
}
} // namespace authz
namespace authz::jwt {
// Given S = concat(B64UrlEnc(headerJson), ".", B64UrlEnc(payloadJson)),
// JWT is concat(S, ".", B64UrlEnc(sign(S, PrivateKey))).
// Below we refer to S as "sign input"
// This struct is not meant to be flatbuffer-serialized
// This is a parsed, flattened view of S and signature
template <class StringType, template <class> class VectorType, template <class> class OptionalType = std::optional>
struct BasicTokenSpec {
// header part ("typ": "JWT" implicitly enforced)
Algorithm algorithm; // alg
StringType keyId; // kid
// payload part
OptionalType<StringType> issuer; // iss
OptionalType<StringType> subject; // sub
OptionalType<VectorType<StringType>> audience; // aud
OptionalType<uint64_t> issuedAtUnixTime; // iat
OptionalType<uint64_t> expiresAtUnixTime; // exp
OptionalType<uint64_t> notBeforeUnixTime; // nbf
OptionalType<StringType> tokenId; // jti
OptionalType<VectorType<StringType>> tenants; // tenants
// signature part
StringType signature;
};
} // namespace authz::jwt
#endif /*FDBRPC_TOKEN_SPEC_H*/

View File

@ -1174,8 +1174,6 @@ ACTOR Future<Void> checkManagerLock(Transaction* tr, Reference<BlobManagerData>
ASSERT(currentLockValue.present());
int64_t currentEpoch = decodeBlobManagerEpochValue(currentLockValue.get());
if (currentEpoch != bmData->epoch) {
ASSERT(currentEpoch > bmData->epoch);
if (BM_DEBUG) {
fmt::print(
"BM {0} found new epoch {1} > {2} in lock check\n", bmData->id.toString(), currentEpoch, bmData->epoch);
@ -3547,16 +3545,27 @@ ACTOR Future<Void> recoverBlobManager(Reference<BlobManagerData> bmData) {
bool isFullRestore = wait(isFullRestoreMode(bmData->db, normalKeys));
bmData->isFullRestoreMode = isFullRestore;
if (bmData->isFullRestoreMode) {
BlobRestoreStatus initStatus(BlobRestorePhase::LOAD_MANIFEST);
wait(updateRestoreStatus(bmData->db, normalKeys, initStatus));
wait(loadManifest(bmData->db, bmData->bstore));
int64_t epoc = wait(lastBlobEpoc(bmData->db, bmData->bstore));
wait(updateEpoch(bmData, epoc + 1));
BlobRestoreStatus completedStatus(BlobRestorePhase::MANIFEST_DONE);
wait(updateRestoreStatus(bmData->db, normalKeys, completedStatus));
Optional<BlobRestoreStatus> status = wait(getRestoreStatus(bmData->db, normalKeys));
ASSERT(status.present());
state BlobRestorePhase phase = status.get().phase;
if (phase == BlobRestorePhase::STARTING_MIGRATOR || phase == BlobRestorePhase::LOADING_MANIFEST) {
wait(updateRestoreStatus(bmData->db, normalKeys, BlobRestoreStatus(LOADING_MANIFEST), {}));
try {
wait(loadManifest(bmData->db, bmData->bstore));
int64_t epoc = wait(lastBlobEpoc(bmData->db, bmData->bstore));
wait(updateEpoch(bmData, epoc + 1));
BlobRestoreStatus completedStatus(BlobRestorePhase::LOADED_MANIFEST);
wait(updateRestoreStatus(bmData->db, normalKeys, completedStatus, BlobRestorePhase::LOADING_MANIFEST));
} catch (Error& e) {
if (e.code() != error_code_restore_missing_data) {
throw e; // retryable errors
}
// terminate blob restore for non-retryable errors
TraceEvent("ManifestLoadError", bmData->id).error(e).detail("Phase", phase);
BlobRestoreStatus error(BlobRestorePhase::ERROR, e.code());
wait(updateRestoreStatus(bmData->db, normalKeys, error, {}));
}
}
}
state Reference<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(bmData->db);
@ -5310,10 +5319,6 @@ ACTOR Future<Void> bgConsistencyCheck(Reference<BlobManagerData> bmData) {
}
ACTOR Future<Void> backupManifest(Reference<BlobManagerData> bmData) {
if (g_network->isSimulated() && g_simulator->speedUpSimulation) {
return Void();
}
bmData->initBStore();
loop {
wait(dumpManifest(bmData->db, bmData->bstore, bmData->epoch, bmData->manifestDumperSeqNo));
@ -5322,8 +5327,8 @@ ACTOR Future<Void> backupManifest(Reference<BlobManagerData> bmData) {
}
}
// Simulation validation that multiple blob managers aren't started with the same epoch
static std::map<int64_t, UID> managerEpochsSeen;
// Simulation validation that multiple blob managers aren't started with the same epoch within same cluster
static std::map<std::pair<std::string, int64_t>, UID> managerEpochsSeen;
ACTOR Future<Void> checkBlobManagerEpoch(Reference<AsyncVar<ServerDBInfo> const> dbInfo, int64_t epoch, UID dbgid) {
loop {
@ -5338,15 +5343,18 @@ ACTOR Future<Void> blobManager(BlobManagerInterface bmInterf,
Reference<AsyncVar<ServerDBInfo> const> dbInfo,
int64_t epoch) {
if (g_network->isSimulated()) {
bool managerEpochAlreadySeen = managerEpochsSeen.count(epoch);
std::string clusterId = dbInfo->get().clusterInterface.id().shortString();
auto clusterEpoc = std::make_pair(clusterId, epoch);
bool managerEpochAlreadySeen = managerEpochsSeen.count(clusterEpoc);
if (managerEpochAlreadySeen) {
TraceEvent(SevError, "DuplicateBlobManagersAtEpoch")
.detail("ClusterId", clusterId)
.detail("Epoch", epoch)
.detail("BMID1", bmInterf.id())
.detail("BMID2", managerEpochsSeen.at(epoch));
.detail("BMID2", managerEpochsSeen.at(clusterEpoc));
}
ASSERT(!managerEpochAlreadySeen);
managerEpochsSeen[epoch] = bmInterf.id();
managerEpochsSeen[clusterEpoc] = bmInterf.id();
}
state Reference<BlobManagerData> self =
makeReference<BlobManagerData>(bmInterf.id(),
@ -5390,7 +5398,7 @@ ACTOR Future<Void> blobManager(BlobManagerInterface bmInterf,
self->addActor.send(backupManifest(self));
}
if (BUGGIFY) {
if (BUGGIFY && !self->isFullRestoreMode) {
self->addActor.send(chaosRangeMover(self));
}

View File

@ -87,13 +87,6 @@ struct BlobManifestFile {
std::sort(result.begin(), result.end());
return result;
}
// Find the last manifest file
ACTOR static Future<std::string> last(Reference<BackupContainerFileSystem> reader) {
std::vector<BlobManifestFile> files = wait(list(reader));
ASSERT(!files.empty());
return files.front().fileName;
}
};
// This class dumps blob manifest to external blob storage.
@ -135,6 +128,7 @@ private:
blobGranuleMappingKeys, // Map granule to workers. Track the active granules
blobGranuleFileKeys, // Map a granule version to granule files. Track files for a granule
blobGranuleHistoryKeys, // Map granule to its parents and parent bundaries. for time-travel read
blobGranuleSplitKeys, // Granule split state to recover from a splitting granule
blobRangeKeys // Key ranges managed by blob
};
for (auto range : ranges) {
@ -233,11 +227,16 @@ public:
ACTOR static Future<Void> execute(Reference<BlobManifestLoader> self) {
try {
Value data = wait(readFromFile(self));
if (data.empty()) {
throw restore_missing_data();
}
state Standalone<BlobManifest> manifest = decode(data);
wait(writeSystemKeys(self, manifest.rows));
BlobGranuleRestoreVersionVector _ = wait(listGranules(self));
} catch (Error& e) {
dprint("WARNING: unexpected manifest loader error {}\n", e.what()); // skip error handling so far
dprint("WARNING: unexpected manifest loader error {}\n", e.what());
TraceEvent("BlobManfiestError").error(e).log();
throw;
}
return Void();
}
@ -286,9 +285,8 @@ public:
if (e.code() == error_code_restore_missing_data) {
dprint("missing data for key range {} \n", granuleRange.toString());
TraceEvent("BlobRestoreMissingData").detail("KeyRange", granuleRange.toString());
} else {
throw;
}
throw;
}
}
return results;
@ -319,7 +317,12 @@ private:
// Read data from a manifest file
ACTOR static Future<Value> readFromFile(Reference<BlobManifestLoader> self) {
state Reference<BackupContainerFileSystem> container = self->blobConn_->getForRead(MANIFEST_FOLDER);
std::string fileName = wait(BlobManifestFile::last(container));
std::vector<BlobManifestFile> files = wait(BlobManifestFile::list(container));
if (files.empty()) {
dprint("No blob manifest files for restore\n");
return Value();
}
std::string fileName = files.front().fileName;
state Reference<IAsyncFile> reader = wait(container->readFile(fileName));
state int64_t fileSize = wait(reader->size());
state Arena arena;
@ -376,7 +379,7 @@ private:
// Find the newest granule for a key range. The newest granule has the max version and relevant files
ACTOR static Future<Standalone<BlobGranuleRestoreVersion>> getGranule(Transaction* tr, KeyRangeRef range) {
state Standalone<BlobGranuleRestoreVersion> granuleVersion;
KeyRange historyKeyRange = blobGranuleHistoryKeyRangeFor(range);
state KeyRange historyKeyRange = blobGranuleHistoryKeyRangeFor(range);
// reverse lookup so that the first row is the newest version
state RangeResult results =
wait(tr->getRange(historyKeyRange, GetRangeLimits::BYTE_LIMIT_UNLIMITED, Snapshot::True, Reverse::True));
@ -529,13 +532,13 @@ ACTOR Future<int64_t> lastBlobEpoc(Database db, Reference<BlobConnectionProvider
// Return true if the given key range is restoring
ACTOR Future<bool> isFullRestoreMode(Database db, KeyRangeRef keys) {
KeyRange range = wait(getRestoringRange(db, keys));
return !range.empty();
std::pair<KeyRange, BlobRestoreStatus> rangeStatus = wait(getRestoreRangeStatus(db, keys));
return !rangeStatus.first.empty() && rangeStatus.second.phase != BlobRestorePhase::DONE;
}
// Check the given key range and return subrange that is doing restore. Returns empty range if no restoring
// for any portion of the given range.
ACTOR Future<KeyRange> getRestoringRange(Database db, KeyRangeRef keys) {
ACTOR Future<std::pair<KeyRange, BlobRestoreStatus>> getRestoreRangeStatus(Database db, KeyRangeRef keys) {
state Transaction tr(db);
loop {
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
@ -552,9 +555,8 @@ ACTOR Future<KeyRange> getRestoringRange(Database db, KeyRangeRef keys) {
KeyRange keyRange = decodeBlobRestoreCommandKeyFor(r.key);
if (keys.intersects(keyRange)) {
Standalone<BlobRestoreStatus> status = decodeBlobRestoreStatus(r.value);
if (status.phase < BlobRestorePhase::DONE) {
return KeyRangeRef(std::max(keys.begin, keyRange.begin), std::min(keys.end, keyRange.end));
}
KeyRangeRef intersected(std::max(keys.begin, keyRange.begin), std::min(keys.end, keyRange.end));
return std::make_pair(intersected, status);
}
}
if (!ranges.more) {
@ -566,7 +568,7 @@ ACTOR Future<KeyRange> getRestoringRange(Database db, KeyRangeRef keys) {
begin = firstGreaterThan(ranges.end()[-1].key);
}
}
return KeyRangeRef();
return std::make_pair(KeyRangeRef(), BlobRestoreStatus(BlobRestorePhase::DONE));
} catch (Error& e) {
wait(tr.onError(e));
}
@ -574,14 +576,29 @@ ACTOR Future<KeyRange> getRestoringRange(Database db, KeyRangeRef keys) {
}
// Update restore status
ACTOR Future<Void> updateRestoreStatus(Database db, KeyRangeRef range, BlobRestoreStatus status) {
ACTOR Future<Void> updateRestoreStatus(Database db,
KeyRangeRef range,
BlobRestoreStatus status,
Optional<BlobRestorePhase> expectedPhase) {
state Transaction tr(db);
loop {
try {
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
Key key = blobRestoreCommandKeyFor(range);
state Key key = blobRestoreCommandKeyFor(range);
// check if current phase is expected
if (expectedPhase.present()) {
Optional<Value> oldValue = wait(tr.get(key));
if (oldValue.present()) {
Standalone<BlobRestoreStatus> status = decodeBlobRestoreStatus(oldValue.get());
if (status.phase != expectedPhase.get()) {
throw restore_error();
}
}
}
Value value = blobRestoreCommandValueFor(status);
tr.set(key, value);
wait(tr.commit());
@ -593,23 +610,11 @@ ACTOR Future<Void> updateRestoreStatus(Database db, KeyRangeRef range, BlobResto
}
// Get restore status
ACTOR Future<Optional<BlobRestoreStatus>> getRestoreStatus(Database db, KeyRangeRef range) {
state Transaction tr(db);
loop {
try {
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
Key key = blobRestoreCommandKeyFor(range);
Optional<Value> value = wait(tr.get(key));
Optional<BlobRestoreStatus> result;
if (value.present()) {
Standalone<BlobRestoreStatus> status = decodeBlobRestoreStatus(value.get());
result = status;
}
return result;
} catch (Error& e) {
wait(tr.onError(e));
}
ACTOR Future<Optional<BlobRestoreStatus>> getRestoreStatus(Database db, KeyRangeRef keys) {
state Optional<BlobRestoreStatus> result;
std::pair<KeyRange, BlobRestoreStatus> rangeStatus = wait(getRestoreRangeStatus(db, keys));
if (!rangeStatus.first.empty()) {
result = rangeStatus.second;
}
return result;
}

View File

@ -67,18 +67,26 @@ public:
// Start migration
ACTOR static Future<Void> start(Reference<BlobMigrator> self) {
wait(checkIfReadyForMigration(self));
wait(lockDatabase(self->db_, self->interf_.id()));
wait(prepare(self, normalKeys));
wait(advanceVersion(self));
wait(serverLoop(self));
return Void();
}
ACTOR static Future<Void> updateStatus(Reference<BlobMigrator> self, KeyRange keys, BlobRestoreStatus status) {
wait(updateRestoreStatus(self->db_, keys, status, {}));
return Void();
}
private:
// Check if blob manifest is loaded so that blob migration can start
ACTOR static Future<Void> checkIfReadyForMigration(Reference<BlobMigrator> self) {
loop {
Optional<BlobRestoreStatus> status = wait(getRestoreStatus(self->db_, normalKeys));
if (canStartMigration(status)) {
ASSERT(status.present());
BlobRestorePhase phase = status.get().phase;
if (phase == BlobRestorePhase::LOADED_MANIFEST) {
BlobGranuleRestoreVersionVector granules = wait(listBlobGranules(self->db_, self->blobConn_));
if (!granules.empty()) {
self->blobGranules_ = granules;
@ -89,27 +97,24 @@ private:
.detail("Version", granule.version)
.detail("SizeInBytes", granule.sizeInBytes);
}
BlobRestoreStatus status(BlobRestorePhase::MIGRATE, 0);
wait(updateRestoreStatus(self->db_, normalKeys, status));
wait(updateRestoreStatus(self->db_,
normalKeys,
BlobRestoreStatus(BlobRestorePhase::COPYING_DATA),
BlobRestorePhase::LOADED_MANIFEST));
return Void();
}
} else if (phase >= BlobRestorePhase::COPYING_DATA) {
TraceEvent("BlobMigratorUnexpectedPhase", self->interf_.id()).detail("Phase", status.get().phase);
throw restore_error();
}
wait(delay(SERVER_KNOBS->BLOB_MIGRATOR_CHECK_INTERVAL));
}
}
// Check if we should start migration. Migration can be started after manifest is fully loaded
static bool canStartMigration(Optional<BlobRestoreStatus> status) {
if (status.present()) {
BlobRestoreStatus value = status.get();
return value.phase == BlobRestorePhase::MANIFEST_DONE; // manifest is loaded successfully
}
return false;
}
// Prepare for data migration for given key range.
ACTOR static Future<Void> prepare(Reference<BlobMigrator> self, KeyRangeRef keys) {
wait(waitForDataMover(self));
state int oldMode = wait(setDDMode(self->db_, 0));
// Register as a storage server, so that DataDistributor could start data movement after
std::pair<Version, Tag> verAndTag = wait(addStorageServer(self->db_, self->interf_.ssi));
dprint("Started storage server interface {} {}\n", verAndTag.first, verAndTag.second.toString());
@ -118,14 +123,38 @@ private:
// It'll restart DataDistributor so that internal data structures like ShardTracker, ShardsAffectedByTeamFailure
// could be re-initialized. Ideally it should be done within DataDistributor, then we don't need to
// restart DataDistributor
state int oldMode = wait(setDDMode(self->db_, 0));
wait(unassignServerKeys(self, keys));
wait(assignKeysToServer(self, keys, self->interf_.ssi.id()));
wait(success(setDDMode(self->db_, oldMode)));
return Void();
}
// Assign given key range to specified storage server. Subsquent
// Wait until all pending data moving is done before doing full restore.
ACTOR static Future<Void> waitForDataMover(Reference<BlobMigrator> self) {
state int retries = 0;
loop {
state Transaction tr(self->db_);
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
try {
RangeResult dms = wait(tr.getRange(dataMoveKeys, CLIENT_KNOBS->TOO_MANY));
if (dms.size() == 0) {
return Void();
} else {
dprint("Wait pending data moving {}\n", dms.size());
wait(delay(2));
if (++retries > SERVER_KNOBS->BLOB_MIGRATOR_ERROR_RETRIES) {
throw restore_error();
}
}
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
// Assign given key range to specified storage server.
ACTOR static Future<Void> assignKeysToServer(Reference<BlobMigrator> self, KeyRangeRef keys, UID serverUID) {
state Transaction tr(self->db_);
loop {
@ -147,20 +176,31 @@ private:
// Unassign given key range from its current storage servers
ACTOR static Future<Void> unassignServerKeys(Reference<BlobMigrator> self, KeyRangeRef keys) {
state Transaction tr(self->db_);
state int retries = 0;
loop {
state Transaction tr(self->db_);
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
try {
state RangeResult serverList = wait(tr.getRange(serverListKeys, CLIENT_KNOBS->TOO_MANY));
state RangeResult serverList =
wait(tr.getRange(serverListKeys, CLIENT_KNOBS->TOO_MANY, Snapshot::True));
ASSERT(!serverList.more && serverList.size() < CLIENT_KNOBS->TOO_MANY);
for (auto& server : serverList) {
state UID id = decodeServerListValue(server.value).id();
Optional<Value> tag = wait(tr.get(serverTagKeyFor(id)));
if (!tag.present()) {
dprint("Server {} no tag\n", id.shortString());
continue;
}
if (id == self->interf_.id()) {
continue;
}
RangeResult ranges = wait(krmGetRanges(&tr, serverKeysPrefixFor(id), keys));
bool owning = false;
for (auto& r : ranges) {
if (r.value == serverKeysTrue) {
if (r.value != serverKeysFalse) {
owning = true;
break;
}
@ -175,6 +215,9 @@ private:
return Void();
} catch (Error& e) {
wait(tr.onError(e));
if (++retries > SERVER_KNOBS->BLOB_MIGRATOR_ERROR_RETRIES) {
throw restore_error();
}
}
}
}
@ -184,9 +227,17 @@ private:
loop {
bool done = wait(checkProgress(self));
if (done) {
wait(updateRestoreStatus(self->db_, normalKeys, BlobRestoreStatus(BlobRestorePhase::APPLY_MLOGS)));
wait(updateRestoreStatus(self->db_,
normalKeys,
BlobRestoreStatus(BlobRestorePhase::APPLYING_MLOGS),
BlobRestorePhase::COPYING_DATA));
wait(unlockDatabase(self->db_, self->interf_.id()));
wait(applyMutationLogs(self));
wait(updateRestoreStatus(self->db_, normalKeys, BlobRestoreStatus(BlobRestorePhase::DONE)));
wait(updateRestoreStatus(self->db_,
normalKeys,
BlobRestoreStatus(BlobRestorePhase::DONE),
BlobRestorePhase::APPLYING_MLOGS));
return Void();
}
wait(delay(SERVER_KNOBS->BLOB_MIGRATOR_CHECK_INTERVAL));
@ -223,8 +274,8 @@ private:
state bool done = incompleted == 0;
dprint("Migration progress :{}%. done {}\n", progress, done);
TraceEvent("BlobMigratorProgress", self->interf_.id()).detail("Progress", progress);
BlobRestoreStatus status(BlobRestorePhase::MIGRATE, progress);
wait(updateRestoreStatus(self->db_, normalKeys, status));
BlobRestoreStatus status(BlobRestorePhase::COPYING_DATA, progress);
wait(updateRestoreStatus(self->db_, normalKeys, status, BlobRestorePhase::COPYING_DATA));
return done;
} catch (Error& e) {
wait(tr.onError(e));
@ -247,6 +298,9 @@ private:
dprint("Advance version from {} to {}\n", current, expected);
TraceEvent("AdvanceVersion", self->interf_.id()).detail("From", current).detail("To", expected);
wait(tr.commit());
} else {
dprint("Skip advancing version {}. current {}\n", expected, current);
TraceEvent("SkipAdvanceVersion", self->interf_.id()).detail("From", current).detail("To", expected);
}
return Void();
} catch (Error& e) {
@ -287,23 +341,49 @@ private:
// check last version in mutation logs
Optional<std::string> proxy; // unused
Optional<std::string> encryptionKeyFile; // unused
Reference<IBackupContainer> bc = IBackupContainer::openContainer(mutationLogsUrl, proxy, encryptionKeyFile);
state Reference<IBackupContainer> bc =
IBackupContainer::openContainer(mutationLogsUrl, proxy, encryptionKeyFile);
BackupDescription desc = wait(bc->describeBackup());
if (!desc.contiguousLogEnd.present()) {
TraceEvent(SevError, "BlobMigratorInvalidMutationLogs").detail("Url", mutationLogsUrl);
TraceEvent("InvalidMutationLogs").detail("Url", mutationLogsUrl);
throw restore_missing_data();
}
Version targetVersion = desc.contiguousLogEnd.get() - 1;
TraceEvent("ApplyMutationLogs", self->interf_.id()).detail("Version", targetVersion);
if (!desc.minLogBegin.present()) {
TraceEvent("InvalidMutationLogs").detail("Url", mutationLogsUrl);
throw restore_missing_data();
}
state Version minLogVersion = desc.minLogBegin.get();
state Version maxLogVersion = desc.contiguousLogEnd.get() - 1;
// restore to target version
Standalone<VectorRef<KeyRangeRef>> ranges;
Standalone<VectorRef<Version>> beginVersions;
state Standalone<VectorRef<KeyRangeRef>> ranges;
state Standalone<VectorRef<Version>> beginVersions;
for (auto& granule : self->blobGranules_) {
ranges.push_back(ranges.arena(), granule.keyRange);
beginVersions.push_back(beginVersions.arena(), granule.version);
if (granule.version < minLogVersion || granule.version > maxLogVersion) {
TraceEvent("InvalidMutationLogs")
.detail("Granule", granule.granuleID)
.detail("GranuleVersion", granule.version)
.detail("MinLogVersion", minLogVersion)
.detail("MaxLogVersion", maxLogVersion);
throw restore_missing_data();
}
// no need to apply mutation logs if granule is already on that version
if (granule.version < maxLogVersion) {
ranges.push_back(ranges.arena(), granule.keyRange);
beginVersions.push_back(beginVersions.arena(), granule.version + 1);
}
}
Optional<RestorableFileSet> restoreSet =
wait(bc->getRestoreSet(maxLogVersion, self->db_, ranges, OnlyApplyMutationLogs::True, minLogVersion));
if (!restoreSet.present()) {
TraceEvent("InvalidMutationLogs")
.detail("MinLogVersion", minLogVersion)
.detail("MaxLogVersion", maxLogVersion);
throw restore_missing_data();
}
std::string tagName = "blobrestore-" + self->interf_.id().shortString();
TraceEvent("ApplyMutationLogs", self->interf_.id()).detail("Version", minLogVersion);
wait(submitRestore(self, KeyRef(tagName), KeyRef(mutationLogsUrl), ranges, beginVersions));
return Void();
}
@ -327,11 +407,11 @@ private:
beginVersions,
WaitForComplete::True,
invalidVersion,
Verbose::False,
Verbose::True,
""_sr, // addPrefix
""_sr, // removePrefix
LockDB::False,
UnlockDB::False,
LockDB::True,
UnlockDB::True,
OnlyApplyMutationLogs::True));
TraceEvent("ApplyMutationLogsComplete", self->interf_.id()).detail("Version", version);
return Void();
@ -396,6 +476,11 @@ private:
metrics.bytes = sizeInBytes(self);
GetStorageMetricsReply resp;
resp.load = metrics;
resp.available = StorageMetrics();
resp.capacity = StorageMetrics();
resp.bytesInputRate = 0;
resp.versionLag = 0;
resp.lastUpdate = now();
req.reply.send(resp);
}
when(ReplyPromise<KeyValueStoreType> reply = waitNext(ssi.getKeyValueStoreType.getFuture())) {
@ -436,11 +521,14 @@ private:
req.reply.sendError(unsupported_operation());
}
when(GetKeyValuesRequest req = waitNext(ssi.getKeyValues.getFuture())) {
/* dprint("Unsupported GetKeyValuesRequest {} - {} @ {}\n",
dprint("Unsupported GetKeyValuesRequest {} - {} @ {}\n",
req.begin.getKey().printable(),
req.end.getKey().printable(),
req.version); */
req.reply.sendError(unsupported_operation());
req.version);
// A temp fix to send back broken promise error so that fetchKey can switch to another
// storage server. We should remove the storage server interface after
// restore is done
req.reply.sendError(broken_promise());
}
when(GetValueRequest req = waitNext(ssi.getValue.getFuture())) {
dprint("Unsupported GetValueRequest\n");
@ -525,12 +613,13 @@ private:
ACTOR Future<Void> blobMigrator(BlobMigratorInterface interf, Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
TraceEvent("StartBlobMigrator", interf.id()).detail("Interface", interf.id().toString());
dprint("Starting blob migrator {}\n", interf.id().toString());
state Reference<BlobMigrator> self = makeReference<BlobMigrator>(dbInfo, interf);
try {
Reference<BlobMigrator> self = makeReference<BlobMigrator>(dbInfo, interf);
wait(BlobMigrator::start(self));
} catch (Error& e) {
dprint("Unexpected blob migrator error {}\n", e.what());
TraceEvent("BlobMigratorError", interf.id()).error(e);
wait(BlobMigrator::updateStatus(self, normalKeys, BlobRestoreStatus(BlobRestorePhase::ERROR, e.code())));
}
return Void();
}

View File

@ -284,7 +284,7 @@ struct BlobWorkerData : NonCopyable, ReferenceCounted<BlobWorkerData> {
int changeFeedStreamReplyBufferSize = SERVER_KNOBS->BG_DELTA_FILE_TARGET_BYTES / 4;
bool isEncryptionEnabled = false;
EncryptionAtRestMode encryptMode;
bool buggifyFull = false;
int64_t memoryFullThreshold =
@ -307,7 +307,7 @@ struct BlobWorkerData : NonCopyable, ReferenceCounted<BlobWorkerData> {
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
SERVER_KNOBS->FILE_LATENCY_SKETCH_ACCURACY,
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY),
isEncryptionEnabled(isEncryptionOpSupported(EncryptOperationType::BLOB_GRANULE_ENCRYPTION)) {}
encryptMode(EncryptionAtRestMode::DISABLED) {}
bool managerEpochOk(int64_t epoch) {
if (epoch < currentManagerEpoch) {
@ -438,6 +438,10 @@ void checkGranuleLock(int64_t epoch, int64_t seqno, int64_t ownerEpoch, int64_t
throw granule_assignment_conflict();
}
}
bool isEncryptionOpSupported(EncryptionAtRestMode encryptMode) {
return encryptMode.isEncryptionEnabled() && SERVER_KNOBS->BG_METADATA_SOURCE == "tenant";
}
} // namespace
// Below actors asssit in fetching/lookup desired encryption keys. Following steps are done for an encryption key
@ -455,16 +459,19 @@ ACTOR Future<BlobGranuleCipherKeysCtx> getLatestGranuleCipherKeys(Reference<Blob
KeyRange keyRange,
Arena* arena) {
state BlobGranuleCipherKeysCtx cipherKeysCtx;
state Reference<GranuleTenantData> tenantData = wait(bwData->tenantData.getDataForGranule(keyRange));
ASSERT(tenantData.isValid());
state EncryptCipherDomainId domainId = FDB_DEFAULT_ENCRYPT_DOMAIN_ID;
if (bwData->encryptMode.mode == EncryptionAtRestMode::DOMAIN_AWARE) {
state Reference<GranuleTenantData> tenantData = wait(bwData->tenantData.getDataForGranule(keyRange));
ASSERT(tenantData.isValid());
domainId = tenantData->entry.id;
}
std::unordered_set<EncryptCipherDomainId> domainIds;
domainIds.emplace(tenantData->entry.id);
domainIds.emplace(domainId);
std::unordered_map<EncryptCipherDomainId, Reference<BlobCipherKey>> domainKeyMap =
wait(getLatestEncryptCipherKeys(bwData->dbInfo, domainIds, BlobCipherMetrics::BLOB_GRANULE));
auto domainKeyItr = domainKeyMap.find(tenantData->entry.id);
auto domainKeyItr = domainKeyMap.find(domainId);
ASSERT(domainKeyItr != domainKeyMap.end());
cipherKeysCtx.textCipherKey = BlobGranuleCipherKey::fromBlobCipherKey(domainKeyItr->second, *arena);
@ -822,7 +829,7 @@ ACTOR Future<BlobFileIndex> writeDeltaFile(Reference<BlobWorkerData> bwData,
state Optional<BlobGranuleCipherKeysMeta> cipherKeysMeta;
state Arena arena;
if (bwData->isEncryptionEnabled) {
if (isEncryptionOpSupported(bwData->encryptMode)) {
BlobGranuleCipherKeysCtx ciphKeysCtx = wait(getLatestGranuleCipherKeys(bwData, keyRange, &arena));
cipherKeysCtx = std::move(ciphKeysCtx);
cipherKeysMeta = BlobGranuleCipherKeysCtx::toCipherKeysMeta(cipherKeysCtx.get());
@ -1035,7 +1042,7 @@ ACTOR Future<BlobFileIndex> writeSnapshot(Reference<BlobWorkerData> bwData,
state Optional<BlobGranuleCipherKeysMeta> cipherKeysMeta;
state Arena arena;
if (bwData->isEncryptionEnabled) {
if (isEncryptionOpSupported(bwData->encryptMode)) {
BlobGranuleCipherKeysCtx ciphKeysCtx = wait(getLatestGranuleCipherKeys(bwData, keyRange, &arena));
cipherKeysCtx = std::move(ciphKeysCtx);
cipherKeysMeta = BlobGranuleCipherKeysCtx::toCipherKeysMeta(cipherKeysCtx.get());
@ -1281,13 +1288,12 @@ ACTOR Future<BlobFileIndex> compactFromBlob(Reference<BlobWorkerData> bwData,
ASSERT(snapshotVersion < version);
state Optional<BlobGranuleCipherKeysCtx> snapCipherKeysCtx;
if (g_network && g_network->isSimulated() &&
isEncryptionOpSupported(EncryptOperationType::BLOB_GRANULE_ENCRYPTION) &&
if (g_network && g_network->isSimulated() && isEncryptionOpSupported(bwData->encryptMode) &&
!snapshotF.cipherKeysMeta.present()) {
ASSERT(false);
}
if (snapshotF.cipherKeysMeta.present()) {
ASSERT(isEncryptionOpSupported(EncryptOperationType::BLOB_GRANULE_ENCRYPTION));
ASSERT(isEncryptionOpSupported(bwData->encryptMode));
CODE_PROBE(true, "fetching cipher keys for blob snapshot file");
BlobGranuleCipherKeysCtx keysCtx =
wait(getGranuleCipherKeysFromKeysMeta(bwData, snapshotF.cipherKeysMeta.get(), &filenameArena));
@ -1314,14 +1320,13 @@ ACTOR Future<BlobFileIndex> compactFromBlob(Reference<BlobWorkerData> bwData,
deltaF = files.deltaFiles[deltaIdx];
if (g_network && g_network->isSimulated() &&
isEncryptionOpSupported(EncryptOperationType::BLOB_GRANULE_ENCRYPTION) &&
if (g_network && g_network->isSimulated() && isEncryptionOpSupported(bwData->encryptMode) &&
!deltaF.cipherKeysMeta.present()) {
ASSERT(false);
}
if (deltaF.cipherKeysMeta.present()) {
ASSERT(isEncryptionOpSupported(EncryptOperationType::BLOB_GRANULE_ENCRYPTION));
ASSERT(isEncryptionOpSupported(bwData->encryptMode));
CODE_PROBE(true, "fetching cipher keys for delta file");
BlobGranuleCipherKeysCtx keysCtx =
wait(getGranuleCipherKeysFromKeysMeta(bwData, deltaF.cipherKeysMeta.get(), &filenameArena));
@ -2129,7 +2134,6 @@ ACTOR Future<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> bwData,
}
}
}
metadata->files = startState.existingFiles.get();
snapshotEligible = true;
}
@ -2173,8 +2177,29 @@ ACTOR Future<Void> blobGranuleUpdateFiles(Reference<BlobWorkerData> bwData,
}
// No need to start Change Feed in full restore mode
if (bwData->isFullRestoreMode)
if (bwData->isFullRestoreMode) {
while (inFlightFiles.size() > 0) {
if (inFlightFiles.front().future.isReady()) {
BlobFileIndex completedFile = wait(inFlightFiles.front().future);
if (inFlightFiles.front().snapshot) {
if (metadata->files.deltaFiles.empty()) {
ASSERT(completedFile.version == metadata->initialSnapshotVersion);
} else {
ASSERT(completedFile.version == metadata->files.deltaFiles.back().version);
}
metadata->files.snapshotFiles.push_back(completedFile);
metadata->durableSnapshotVersion.set(completedFile.version);
pendingSnapshots--;
}
inFlightFiles.pop_front();
wait(yield(TaskPriority::BlobWorkerUpdateStorage));
} else {
wait(yield(TaskPriority::BlobWorkerUpdateStorage));
}
}
metadata->readable.send(Void());
return Void();
}
checkMergeCandidate = granuleCheckMergeCandidate(bwData,
metadata,
@ -3621,12 +3646,10 @@ ACTOR Future<Void> doBlobGranuleFileRequest(Reference<BlobWorkerData> bwData, Bl
state Reference<GranuleMetadata> metadata = m;
// state Version granuleBeginVersion = req.beginVersion;
// skip waiting for CF ready for recovery mode
if (!bwData->isFullRestoreMode) {
choose {
when(wait(metadata->readable.getFuture())) {}
when(wait(metadata->cancelled.getFuture())) {
throw wrong_shard_server();
}
choose {
when(wait(metadata->readable.getFuture())) {}
when(wait(metadata->cancelled.getFuture())) {
throw wrong_shard_server();
}
}
@ -3803,12 +3826,12 @@ ACTOR Future<Void> doBlobGranuleFileRequest(Reference<BlobWorkerData> bwData, Bl
.detail("Encrypted", encrypted);
}
if (g_network && g_network->isSimulated() &&
isEncryptionOpSupported(EncryptOperationType::BLOB_GRANULE_ENCRYPTION) && !encrypted) {
if (g_network && g_network->isSimulated() && isEncryptionOpSupported(bwData->encryptMode) &&
!encrypted) {
ASSERT(false);
}
if (encrypted) {
ASSERT(isEncryptionOpSupported(EncryptOperationType::BLOB_GRANULE_ENCRYPTION));
ASSERT(isEncryptionOpSupported(bwData->encryptMode));
ASSERT(!chunk.snapshotFile.get().cipherKeysCtx.present());
CODE_PROBE(true, "fetching cipher keys from meta ref for snapshot file");
snapCipherKeysCtx = getGranuleCipherKeysFromKeysMetaRef(
@ -3825,12 +3848,12 @@ ACTOR Future<Void> doBlobGranuleFileRequest(Reference<BlobWorkerData> bwData, Bl
.detail("Encrypted", encrypted);
}
if (g_network && g_network->isSimulated() &&
isEncryptionOpSupported(EncryptOperationType::BLOB_GRANULE_ENCRYPTION) && !encrypted) {
if (g_network && g_network->isSimulated() && isEncryptionOpSupported(bwData->encryptMode) &&
!encrypted) {
ASSERT(false);
}
if (encrypted) {
ASSERT(isEncryptionOpSupported(EncryptOperationType::BLOB_GRANULE_ENCRYPTION));
ASSERT(isEncryptionOpSupported(bwData->encryptMode));
ASSERT(!chunk.deltaFiles[deltaIdx].cipherKeysCtx.present());
CODE_PROBE(true, "fetching cipher keys from meta ref for delta files");
deltaCipherKeysCtxs.emplace(
@ -4030,7 +4053,6 @@ ACTOR Future<GranuleStartState> openGranule(Reference<BlobWorkerData> bwData, As
state Future<Optional<Value>> fLockValue = tr.get(lockKey);
state Future<ForcedPurgeState> fForcedPurgeState = getForcePurgedState(&tr, req.keyRange);
Future<Optional<GranuleHistory>> fHistory = getLatestGranuleHistory(&tr, req.keyRange);
Optional<GranuleHistory> history = wait(fHistory);
info.history = history;
@ -4132,6 +4154,7 @@ ACTOR Future<GranuleStartState> openGranule(Reference<BlobWorkerData> bwData, As
if (info.existingFiles.get().snapshotFiles.empty()) {
ASSERT(info.existingFiles.get().deltaFiles.empty());
info.previousDurableVersion = invalidVersion;
info.doSnapshot = true;
} else if (info.existingFiles.get().deltaFiles.empty()) {
info.previousDurableVersion = info.existingFiles.get().snapshotFiles.back().version;
} else {
@ -4157,8 +4180,7 @@ ACTOR Future<GranuleStartState> openGranule(Reference<BlobWorkerData> bwData, As
// If anything in previousGranules, need to do the handoff logic and set
// ret.previousChangeFeedId, and the previous durable version will come from the previous
// granules
if (info.history.present() && info.history.get().value.parentVersions.size() > 0 &&
!bwData->isFullRestoreMode) {
if (info.history.present() && info.history.get().value.parentVersions.size() > 0) {
CODE_PROBE(true, "Granule open found parent");
if (info.history.get().value.parentVersions.size() == 1) { // split
state KeyRangeRef parentRange(info.history.get().value.parentBoundaries[0],
@ -4190,11 +4212,13 @@ ACTOR Future<GranuleStartState> openGranule(Reference<BlobWorkerData> bwData, As
info.changeFeedStartVersion = granuleSplitState.second;
} else if (granuleSplitState.first == BlobGranuleSplitState::Initialized) {
CODE_PROBE(true, "Granule open found granule in initialized state");
wait(updateGranuleSplitState(&tr,
info.splitParentGranule.get().first,
info.splitParentGranule.get().second,
info.granuleID,
BlobGranuleSplitState::Assigned));
if (!bwData->isFullRestoreMode) {
wait(updateGranuleSplitState(&tr,
info.splitParentGranule.get().first,
info.splitParentGranule.get().second,
info.granuleID,
BlobGranuleSplitState::Assigned));
}
// change feed was created as part of this transaction, changeFeedStartVersion
// will be set later
} else {
@ -5099,10 +5123,15 @@ ACTOR Future<Void> simForceFullMemory(Reference<BlobWorkerData> bwData) {
ACTOR Future<Void> blobWorker(BlobWorkerInterface bwInterf,
ReplyPromise<InitializeBlobWorkerReply> recruitReply,
Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
state Reference<BlobWorkerData> self(new BlobWorkerData(
bwInterf.id(), dbInfo, openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::True)));
state Database cx = openDBOnServer(dbInfo, TaskPriority::DefaultEndpoint, LockAware::True);
state Reference<BlobWorkerData> self(new BlobWorkerData(bwInterf.id(), dbInfo, cx));
self->id = bwInterf.id();
self->locality = bwInterf.locality;
// Since the blob worker gets initalized through the blob manager it is easier to fetch the encryption state using
// the DB Config rather than passing it through the initalization request for the blob manager and blob worker
DatabaseConfiguration config = wait(getDatabaseConfiguration(cx));
self->encryptMode = config.encryptionAtRestMode;
TraceEvent("BWEncryptionAtRestMode").detail("Mode", self->encryptMode.toString());
state Future<Void> collection = actorCollection(self->addActor.getFuture());

View File

@ -61,6 +61,7 @@
#include "fdbserver/RatekeeperInterface.h"
#include "fdbserver/BlobManagerInterface.h"
#include "fdbserver/ServerDBInfo.h"
#include "fdbserver/SingletonRoles.h"
#include "fdbserver/Status.actor.h"
#include "fdbserver/LatencyBandConfig.h"
#include "fdbclient/GlobalConfig.actor.h"
@ -75,187 +76,6 @@
#include "flow/Util.h"
#include "flow/actorcompiler.h" // This must be the last #include.
void failAfter(Future<Void> trigger, Endpoint e);
// This is used to artificially amplify the used count for processes
// occupied by non-singletons. This ultimately makes it less desirable
// for singletons to use those processes as well. This constant should
// be increased if we ever have more than 100 singletons (unlikely).
static const int PID_USED_AMP_FOR_NON_SINGLETON = 100;
// Wrapper for singleton interfaces
template <class Interface>
struct Singleton {
const Optional<Interface>& interface;
Singleton(const Optional<Interface>& interface) : interface(interface) {}
virtual Role getRole() const = 0;
virtual ProcessClass::ClusterRole getClusterRole() const = 0;
virtual void setInterfaceToDbInfo(ClusterControllerData* cc) const = 0;
virtual void halt(ClusterControllerData* cc, Optional<Standalone<StringRef>> pid) const = 0;
virtual void recruit(ClusterControllerData* cc) const = 0;
};
struct RatekeeperSingleton : Singleton<RatekeeperInterface> {
RatekeeperSingleton(const Optional<RatekeeperInterface>& interface) : Singleton(interface) {}
Role getRole() const { return Role::RATEKEEPER; }
ProcessClass::ClusterRole getClusterRole() const { return ProcessClass::Ratekeeper; }
void setInterfaceToDbInfo(ClusterControllerData* cc) const {
if (interface.present()) {
TraceEvent("CCRK_SetInf", cc->id).detail("Id", interface.get().id());
cc->db.setRatekeeper(interface.get());
}
}
void halt(ClusterControllerData* cc, Optional<Standalone<StringRef>> pid) const {
if (interface.present() && cc->id_worker.count(pid)) {
cc->id_worker[pid].haltRatekeeper =
brokenPromiseToNever(interface.get().haltRatekeeper.getReply(HaltRatekeeperRequest(cc->id)));
}
}
void recruit(ClusterControllerData* cc) const {
cc->lastRecruitTime = now();
cc->recruitRatekeeper.set(true);
}
};
struct DataDistributorSingleton : Singleton<DataDistributorInterface> {
DataDistributorSingleton(const Optional<DataDistributorInterface>& interface) : Singleton(interface) {}
Role getRole() const { return Role::DATA_DISTRIBUTOR; }
ProcessClass::ClusterRole getClusterRole() const { return ProcessClass::DataDistributor; }
void setInterfaceToDbInfo(ClusterControllerData* cc) const {
if (interface.present()) {
TraceEvent("CCDD_SetInf", cc->id).detail("Id", interface.get().id());
cc->db.setDistributor(interface.get());
}
}
void halt(ClusterControllerData* cc, Optional<Standalone<StringRef>> pid) const {
if (interface.present() && cc->id_worker.count(pid)) {
cc->id_worker[pid].haltDistributor =
brokenPromiseToNever(interface.get().haltDataDistributor.getReply(HaltDataDistributorRequest(cc->id)));
}
}
void recruit(ClusterControllerData* cc) const {
cc->lastRecruitTime = now();
cc->recruitDistributor.set(true);
}
};
struct ConsistencyScanSingleton : Singleton<ConsistencyScanInterface> {
ConsistencyScanSingleton(const Optional<ConsistencyScanInterface>& interface) : Singleton(interface) {}
Role getRole() const { return Role::CONSISTENCYSCAN; }
ProcessClass::ClusterRole getClusterRole() const { return ProcessClass::ConsistencyScan; }
void setInterfaceToDbInfo(ClusterControllerData* cc) const {
if (interface.present()) {
TraceEvent("CCCK_SetInf", cc->id).detail("Id", interface.get().id());
cc->db.setConsistencyScan(interface.get());
}
}
void halt(ClusterControllerData* cc, Optional<Standalone<StringRef>> pid) const {
if (interface.present()) {
cc->id_worker[pid].haltConsistencyScan =
brokenPromiseToNever(interface.get().haltConsistencyScan.getReply(HaltConsistencyScanRequest(cc->id)));
}
}
void recruit(ClusterControllerData* cc) const {
cc->lastRecruitTime = now();
cc->recruitConsistencyScan.set(true);
}
};
struct BlobManagerSingleton : Singleton<BlobManagerInterface> {
BlobManagerSingleton(const Optional<BlobManagerInterface>& interface) : Singleton(interface) {}
Role getRole() const { return Role::BLOB_MANAGER; }
ProcessClass::ClusterRole getClusterRole() const { return ProcessClass::BlobManager; }
void setInterfaceToDbInfo(ClusterControllerData* cc) const {
if (interface.present()) {
TraceEvent("CCBM_SetInf", cc->id).detail("Id", interface.get().id());
cc->db.setBlobManager(interface.get());
}
}
void halt(ClusterControllerData* cc, Optional<Standalone<StringRef>> pid) const {
if (interface.present() && cc->id_worker.count(pid)) {
cc->id_worker[pid].haltBlobManager =
brokenPromiseToNever(interface.get().haltBlobManager.getReply(HaltBlobManagerRequest(cc->id)));
}
}
void recruit(ClusterControllerData* cc) const {
cc->lastRecruitTime = now();
cc->recruitBlobManager.set(true);
}
void haltBlobGranules(ClusterControllerData* cc, Optional<Standalone<StringRef>> pid) const {
if (interface.present()) {
cc->id_worker[pid].haltBlobManager =
brokenPromiseToNever(interface.get().haltBlobGranules.getReply(HaltBlobGranulesRequest(cc->id)));
}
}
};
struct BlobMigratorSingleton : Singleton<BlobMigratorInterface> {
BlobMigratorSingleton(const Optional<BlobMigratorInterface>& interface) : Singleton(interface) {}
Role getRole() const { return Role::BLOB_MIGRATOR; }
ProcessClass::ClusterRole getClusterRole() const { return ProcessClass::BlobMigrator; }
void setInterfaceToDbInfo(ClusterControllerData* cc) const {
if (interface.present()) {
TraceEvent("CCMG_SetInf", cc->id).detail("Id", interface.get().id());
cc->db.setBlobMigrator(interface.get());
}
}
void halt(ClusterControllerData* cc, Optional<Standalone<StringRef>> pid) const {
if (interface.present()) {
TraceEvent("CCMG_Halt", cc->id).detail("Id", interface.get().id());
cc->id_worker[pid].haltBlobMigrator =
brokenPromiseToNever(interface.get().haltBlobMigrator.getReply(HaltBlobMigratorRequest(cc->id)));
}
}
void recruit(ClusterControllerData* cc) const {
cc->lastRecruitTime = now();
cc->recruitBlobMigrator.set(true);
}
};
struct EncryptKeyProxySingleton : Singleton<EncryptKeyProxyInterface> {
EncryptKeyProxySingleton(const Optional<EncryptKeyProxyInterface>& interface) : Singleton(interface) {}
Role getRole() const { return Role::ENCRYPT_KEY_PROXY; }
ProcessClass::ClusterRole getClusterRole() const { return ProcessClass::EncryptKeyProxy; }
void setInterfaceToDbInfo(ClusterControllerData* cc) const {
if (interface.present()) {
TraceEvent("CCEKP_SetInf", cc->id).detail("Id", interface.get().id());
cc->db.setEncryptKeyProxy(interface.get());
}
}
void halt(ClusterControllerData* cc, Optional<Standalone<StringRef>> pid) const {
if (interface.present() && cc->id_worker.count(pid)) {
cc->id_worker[pid].haltEncryptKeyProxy =
brokenPromiseToNever(interface.get().haltEncryptKeyProxy.getReply(HaltEncryptKeyProxyRequest(cc->id)));
}
}
void recruit(ClusterControllerData* cc) const {
cc->lastRecruitTime = now();
cc->recruitEncryptKeyProxy.set(true);
}
};
ACTOR Future<Optional<Value>> getPreviousCoordinators(ClusterControllerData* self) {
state ReadYourWritesTransaction tr(self->db.db);
loop {
@ -613,22 +433,22 @@ ProcessClass::Fitness findBestFitnessForSingleton(const ClusterControllerData* s
// Returns true iff the singleton is healthy. "Healthy" here means that
// the singleton is stable (see below) and doesn't need to be rerecruited.
// Side effects: (possibly) initiates recruitment
template <class Interface>
template <class SingletonClass>
bool isHealthySingleton(ClusterControllerData* self,
const WorkerDetails& newWorker,
const Singleton<Interface>& singleton,
const SingletonClass& singleton,
const ProcessClass::Fitness& bestFitness,
const Optional<UID> recruitingID) {
// A singleton is stable if it exists in cluster, has not been killed off of proc and is not being recruited
bool isStableSingleton = singleton.interface.present() &&
self->id_worker.count(singleton.interface.get().locality.processId()) &&
(!recruitingID.present() || (recruitingID.get() == singleton.interface.get().id()));
bool isStableSingleton = singleton.isPresent() &&
self->id_worker.count(singleton.getInterface().locality.processId()) &&
(!recruitingID.present() || (recruitingID.get() == singleton.getInterface().id()));
if (!isStableSingleton) {
return false; // not healthy because unstable
}
auto& currWorker = self->id_worker[singleton.interface.get().locality.processId()];
auto& currWorker = self->id_worker[singleton.getInterface().locality.processId()];
auto currFitness = currWorker.details.processClass.machineClassFitness(singleton.getClusterRole());
if (currWorker.priorityInfo.isExcluded) {
currFitness = ProcessClass::ExcludeFit;
@ -644,11 +464,11 @@ bool isHealthySingleton(ClusterControllerData* self,
if (shouldRerecruit) {
std::string roleAbbr = singleton.getRole().abbreviation;
TraceEvent(("CCHalt" + roleAbbr).c_str(), self->id)
.detail(roleAbbr + "ID", singleton.interface.get().id())
.detail(roleAbbr + "ID", singleton.getInterface().id())
.detail("Excluded", currWorker.priorityInfo.isExcluded)
.detail("Fitness", currFitness)
.detail("BestFitness", bestFitness);
singleton.recruit(self); // SIDE EFFECT: initiating recruitment
singleton.recruit(*self); // SIDE EFFECT: initiating recruitment
return false; // not healthy since needed to be rerecruited
} else {
return true; // healthy because doesn't need to be rerecruited
@ -736,29 +556,29 @@ void checkBetterSingletons(ClusterControllerData* self) {
// Check if the singletons are healthy.
// side effect: try to rerecruit the singletons to more optimal processes
bool rkHealthy = isHealthySingleton<RatekeeperInterface>(
bool rkHealthy = isHealthySingleton<RatekeeperSingleton>(
self, newRKWorker, rkSingleton, bestFitnessForRK, self->recruitingRatekeeperID);
bool ddHealthy = isHealthySingleton<DataDistributorInterface>(
bool ddHealthy = isHealthySingleton<DataDistributorSingleton>(
self, newDDWorker, ddSingleton, bestFitnessForDD, self->recruitingDistributorID);
bool csHealthy = isHealthySingleton<ConsistencyScanInterface>(
bool csHealthy = isHealthySingleton<ConsistencyScanSingleton>(
self, newCSWorker, csSingleton, bestFitnessForCS, self->recruitingConsistencyScanID);
bool bmHealthy = true;
bool mgHealthy = true;
if (self->db.blobGranulesEnabled.get()) {
bmHealthy = isHealthySingleton<BlobManagerInterface>(
bmHealthy = isHealthySingleton<BlobManagerSingleton>(
self, newBMWorker, bmSingleton, bestFitnessForBM, self->recruitingBlobManagerID);
if (self->db.blobRestoreEnabled.get()) {
mgHealthy = isHealthySingleton<BlobMigratorInterface>(
mgHealthy = isHealthySingleton<BlobMigratorSingleton>(
self, newMGWorker, mgSingleton, bestFitnessForMG, self->recruitingBlobMigratorID);
}
}
bool ekpHealthy = true;
if (SERVER_KNOBS->ENABLE_ENCRYPTION) {
ekpHealthy = isHealthySingleton<EncryptKeyProxyInterface>(
ekpHealthy = isHealthySingleton<EncryptKeyProxySingleton>(
self, newEKPWorker, ekpSingleton, bestFitnessForEKP, self->recruitingEncryptKeyProxyID);
}
// if any of the singletons are unhealthy (rerecruited or not stable), then do not
@ -769,9 +589,9 @@ void checkBetterSingletons(ClusterControllerData* self) {
// if we reach here, we know that the singletons are healthy so let's
// check if we can colocate the singletons in a more optimal way
Optional<Standalone<StringRef>> currRKProcessId = rkSingleton.interface.get().locality.processId();
Optional<Standalone<StringRef>> currDDProcessId = ddSingleton.interface.get().locality.processId();
Optional<Standalone<StringRef>> currCSProcessId = csSingleton.interface.get().locality.processId();
Optional<Standalone<StringRef>> currRKProcessId = rkSingleton.getInterface().locality.processId();
Optional<Standalone<StringRef>> currDDProcessId = ddSingleton.getInterface().locality.processId();
Optional<Standalone<StringRef>> currCSProcessId = csSingleton.getInterface().locality.processId();
Optional<Standalone<StringRef>> newRKProcessId = newRKWorker.interf.locality.processId();
Optional<Standalone<StringRef>> newDDProcessId = newDDWorker.interf.locality.processId();
Optional<Standalone<StringRef>> newCSProcessId = newCSWorker.interf.locality.processId();
@ -779,17 +599,17 @@ void checkBetterSingletons(ClusterControllerData* self) {
Optional<Standalone<StringRef>> currBMProcessId, newBMProcessId;
Optional<Standalone<StringRef>> currMGProcessId, newMGProcessId;
if (self->db.blobGranulesEnabled.get()) {
currBMProcessId = bmSingleton.interface.get().locality.processId();
currBMProcessId = bmSingleton.getInterface().locality.processId();
newBMProcessId = newBMWorker.interf.locality.processId();
if (self->db.blobRestoreEnabled.get()) {
currMGProcessId = mgSingleton.interface.get().locality.processId();
currMGProcessId = mgSingleton.getInterface().locality.processId();
newMGProcessId = newMGWorker.interf.locality.processId();
}
}
Optional<Standalone<StringRef>> currEKPProcessId, newEKPProcessId;
if (SERVER_KNOBS->ENABLE_ENCRYPTION) {
currEKPProcessId = ekpSingleton.interface.get().locality.processId();
currEKPProcessId = ekpSingleton.getInterface().locality.processId();
newEKPProcessId = newEKPWorker.interf.locality.processId();
}
@ -837,18 +657,18 @@ void checkBetterSingletons(ClusterControllerData* self) {
newColocMap[newCSProcessId] <= currColocMap[currCSProcessId]) {
// rerecruit the singleton for which we have found a better process, if any
if (newColocMap[newRKProcessId] < currColocMap[currRKProcessId]) {
rkSingleton.recruit(self);
rkSingleton.recruit(*self);
} else if (newColocMap[newDDProcessId] < currColocMap[currDDProcessId]) {
ddSingleton.recruit(self);
ddSingleton.recruit(*self);
} else if (self->db.blobGranulesEnabled.get() && newColocMap[newBMProcessId] < currColocMap[currBMProcessId]) {
bmSingleton.recruit(self);
bmSingleton.recruit(*self);
} else if (self->db.blobGranulesEnabled.get() && self->db.blobRestoreEnabled.get() &&
newColocMap[newMGProcessId] < currColocMap[currMGProcessId]) {
mgSingleton.recruit(self);
mgSingleton.recruit(*self);
} else if (SERVER_KNOBS->ENABLE_ENCRYPTION && newColocMap[newEKPProcessId] < currColocMap[currEKPProcessId]) {
ekpSingleton.recruit(self);
ekpSingleton.recruit(*self);
} else if (newColocMap[newCSProcessId] < currColocMap[currCSProcessId]) {
csSingleton.recruit(self);
csSingleton.recruit(*self);
}
}
}
@ -1191,14 +1011,14 @@ void clusterRegisterMaster(ClusterControllerData* self, RegisterMasterRequest co
// Halts the registering (i.e. requesting) singleton if one is already in the process of being recruited
// or, halts the existing singleton in favour of the requesting one
template <class Interface>
template <class SingletonClass>
void haltRegisteringOrCurrentSingleton(ClusterControllerData* self,
const WorkerInterface& worker,
const Singleton<Interface>& currSingleton,
const Singleton<Interface>& registeringSingleton,
const SingletonClass& currSingleton,
const SingletonClass& registeringSingleton,
const Optional<UID> recruitingID) {
ASSERT(currSingleton.getRole() == registeringSingleton.getRole());
const UID registeringID = registeringSingleton.interface.get().id();
const UID registeringID = registeringSingleton.getInterface().id();
const std::string roleName = currSingleton.getRole().roleName;
const std::string roleAbbr = currSingleton.getRole().abbreviation;
@ -1210,22 +1030,22 @@ void haltRegisteringOrCurrentSingleton(ClusterControllerData* self,
.detail("DcID", printable(self->clusterControllerDcId))
.detail("ReqDcID", printable(worker.locality.dcId()))
.detail("Recruiting" + roleAbbr + "ID", recruitingID.present() ? recruitingID.get() : UID());
registeringSingleton.halt(self, worker.locality.processId());
registeringSingleton.halt(*self, worker.locality.processId());
} else if (!recruitingID.present()) {
// if not currently recruiting, then halt previous one in favour of requesting one
TraceEvent(("CCRegister" + roleName).c_str(), self->id).detail(roleAbbr + "ID", registeringID);
if (currSingleton.interface.present() && currSingleton.interface.get().id() != registeringID &&
self->id_worker.count(currSingleton.interface.get().locality.processId())) {
if (currSingleton.isPresent() && currSingleton.getInterface().id() != registeringID &&
self->id_worker.count(currSingleton.getInterface().locality.processId())) {
TraceEvent(("CCHaltPrevious" + roleName).c_str(), self->id)
.detail(roleAbbr + "ID", currSingleton.interface.get().id())
.detail(roleAbbr + "ID", currSingleton.getInterface().id())
.detail("DcID", printable(self->clusterControllerDcId))
.detail("ReqDcID", printable(worker.locality.dcId()))
.detail("Recruiting" + roleAbbr + "ID", recruitingID.present() ? recruitingID.get() : UID());
currSingleton.halt(self, currSingleton.interface.get().locality.processId());
currSingleton.halt(*self, currSingleton.getInterface().locality.processId());
}
// set the curr singleton if it doesn't exist or its different from the requesting one
if (!currSingleton.interface.present() || currSingleton.interface.get().id() != registeringID) {
registeringSingleton.setInterfaceToDbInfo(self);
if (!currSingleton.isPresent() || currSingleton.getInterface().id() != registeringID) {
registeringSingleton.setInterfaceToDbInfo(*self);
}
}
}
@ -1399,41 +1219,41 @@ ACTOR Future<Void> registerWorker(RegisterWorkerRequest req,
if (req.distributorInterf.present()) {
auto currSingleton = DataDistributorSingleton(self->db.serverInfo->get().distributor);
auto registeringSingleton = DataDistributorSingleton(req.distributorInterf);
haltRegisteringOrCurrentSingleton<DataDistributorInterface>(
haltRegisteringOrCurrentSingleton<DataDistributorSingleton>(
self, w, currSingleton, registeringSingleton, self->recruitingDistributorID);
}
if (req.ratekeeperInterf.present()) {
auto currSingleton = RatekeeperSingleton(self->db.serverInfo->get().ratekeeper);
auto registeringSingleton = RatekeeperSingleton(req.ratekeeperInterf);
haltRegisteringOrCurrentSingleton<RatekeeperInterface>(
haltRegisteringOrCurrentSingleton<RatekeeperSingleton>(
self, w, currSingleton, registeringSingleton, self->recruitingRatekeeperID);
}
if (self->db.blobGranulesEnabled.get() && req.blobManagerInterf.present()) {
auto currSingleton = BlobManagerSingleton(self->db.serverInfo->get().blobManager);
auto registeringSingleton = BlobManagerSingleton(req.blobManagerInterf);
haltRegisteringOrCurrentSingleton<BlobManagerInterface>(
haltRegisteringOrCurrentSingleton<BlobManagerSingleton>(
self, w, currSingleton, registeringSingleton, self->recruitingBlobManagerID);
}
if (req.blobMigratorInterf.present() && self->db.blobRestoreEnabled.get()) {
auto currSingleton = BlobMigratorSingleton(self->db.serverInfo->get().blobMigrator);
auto registeringSingleton = BlobMigratorSingleton(req.blobMigratorInterf);
haltRegisteringOrCurrentSingleton<BlobMigratorInterface>(
haltRegisteringOrCurrentSingleton<BlobMigratorSingleton>(
self, w, currSingleton, registeringSingleton, self->recruitingBlobMigratorID);
}
if (SERVER_KNOBS->ENABLE_ENCRYPTION && req.encryptKeyProxyInterf.present()) {
auto currSingleton = EncryptKeyProxySingleton(self->db.serverInfo->get().encryptKeyProxy);
auto registeringSingleton = EncryptKeyProxySingleton(req.encryptKeyProxyInterf);
haltRegisteringOrCurrentSingleton<EncryptKeyProxyInterface>(
haltRegisteringOrCurrentSingleton<EncryptKeyProxySingleton>(
self, w, currSingleton, registeringSingleton, self->recruitingEncryptKeyProxyID);
}
if (req.consistencyScanInterf.present()) {
auto currSingleton = ConsistencyScanSingleton(self->db.serverInfo->get().consistencyScan);
auto registeringSingleton = ConsistencyScanSingleton(req.consistencyScanInterf);
haltRegisteringOrCurrentSingleton<ConsistencyScanInterface>(
haltRegisteringOrCurrentSingleton<ConsistencyScanSingleton>(
self, w, currSingleton, registeringSingleton, self->recruitingConsistencyScanID);
}
@ -2146,20 +1966,6 @@ ACTOR Future<Void> handleTriggerAuditStorage(ClusterControllerData* self, Cluste
}
}
struct SingletonRecruitThrottler {
double lastRecruitStart;
SingletonRecruitThrottler() : lastRecruitStart(-1) {}
double newRecruitment() {
double n = now();
double waitTime =
std::max(0.0, (lastRecruitStart + SERVER_KNOBS->CC_THROTTLE_SINGLETON_RERECRUIT_INTERVAL - n));
lastRecruitStart = n;
return waitTime;
}
};
ACTOR Future<Void> startDataDistributor(ClusterControllerData* self, double waitTime) {
// If master fails at the same time, give it a chance to clear master PID.
// Also wait to avoid too many consecutive recruits in a small time window.
@ -2213,7 +2019,7 @@ ACTOR Future<Void> startDataDistributor(ClusterControllerData* self, double wait
.detail("DDID", distributor.get().id())
.detail("DcID", printable(self->clusterControllerDcId));
DataDistributorSingleton(distributor).halt(self, distributor.get().locality.processId());
DataDistributorSingleton(distributor).halt(*self, distributor.get().locality.processId());
}
if (!distributor.present() || distributor.get().id() != ddInterf.get().id()) {
self->db.setDistributor(ddInterf.get());
@ -2244,7 +2050,7 @@ ACTOR Future<Void> monitorDataDistributor(ClusterControllerData* self) {
SERVER_KNOBS->DD_FAILURE_TIME))) {
const auto& distributor = self->db.serverInfo->get().distributor;
TraceEvent("CCDataDistributorDied", self->id).detail("DDID", distributor.get().id());
DataDistributorSingleton(distributor).halt(self, distributor.get().locality.processId());
DataDistributorSingleton(distributor).halt(*self, distributor.get().locality.processId());
self->db.clearInterf(ProcessClass::DataDistributorClass);
}
when(wait(self->recruitDistributor.onChange())) {}
@ -2305,7 +2111,7 @@ ACTOR Future<Void> startRatekeeper(ClusterControllerData* self, double waitTime)
TraceEvent("CCHaltRatekeeperAfterRecruit", self->id)
.detail("RKID", ratekeeper.get().id())
.detail("DcID", printable(self->clusterControllerDcId));
RatekeeperSingleton(ratekeeper).halt(self, ratekeeper.get().locality.processId());
RatekeeperSingleton(ratekeeper).halt(*self, ratekeeper.get().locality.processId());
}
if (!ratekeeper.present() || ratekeeper.get().id() != interf.get().id()) {
self->db.setRatekeeper(interf.get());
@ -2336,7 +2142,7 @@ ACTOR Future<Void> monitorRatekeeper(ClusterControllerData* self) {
SERVER_KNOBS->RATEKEEPER_FAILURE_TIME))) {
const auto& ratekeeper = self->db.serverInfo->get().ratekeeper;
TraceEvent("CCRatekeeperDied", self->id).detail("RKID", ratekeeper.get().id());
RatekeeperSingleton(ratekeeper).halt(self, ratekeeper.get().locality.processId());
RatekeeperSingleton(ratekeeper).halt(*self, ratekeeper.get().locality.processId());
self->db.clearInterf(ProcessClass::RatekeeperClass);
}
when(wait(self->recruitRatekeeper.onChange())) {}
@ -2395,7 +2201,7 @@ ACTOR Future<Void> startConsistencyScan(ClusterControllerData* self) {
TraceEvent("CCHaltConsistencyScanAfterRecruit", self->id)
.detail("CKID", consistencyScan.get().id())
.detail("DcID", printable(self->clusterControllerDcId));
ConsistencyScanSingleton(consistencyScan).halt(self, consistencyScan.get().locality.processId());
ConsistencyScanSingleton(consistencyScan).halt(*self, consistencyScan.get().locality.processId());
}
if (!consistencyScan.present() || consistencyScan.get().id() != interf.get().id()) {
self->db.setConsistencyScan(interf.get());
@ -2420,7 +2226,6 @@ ACTOR Future<Void> monitorConsistencyScan(ClusterControllerData* self) {
TraceEvent("CCMonitorConsistencyScanWaitingForRecovery", self->id).log();
wait(self->db.serverInfo->onChange());
}
TraceEvent("CCMonitorConsistencyScan", self->id).log();
loop {
if (self->db.serverInfo->get().consistencyScan.present() && !self->recruitConsistencyScan.get()) {
@ -2498,7 +2303,7 @@ ACTOR Future<Void> startEncryptKeyProxy(ClusterControllerData* self, double wait
TraceEvent("CCEKP_HaltAfterRecruit", self->id)
.detail("Id", encryptKeyProxy.get().id())
.detail("DcId", printable(self->clusterControllerDcId));
EncryptKeyProxySingleton(encryptKeyProxy).halt(self, encryptKeyProxy.get().locality.processId());
EncryptKeyProxySingleton(encryptKeyProxy).halt(*self, encryptKeyProxy.get().locality.processId());
}
if (!encryptKeyProxy.present() || encryptKeyProxy.get().id() != interf.get().id()) {
self->db.setEncryptKeyProxy(interf.get());
@ -2527,7 +2332,7 @@ ACTOR Future<Void> monitorEncryptKeyProxy(ClusterControllerData* self) {
SERVER_KNOBS->ENCRYPT_KEY_PROXY_FAILURE_TIME))) {
TraceEvent("CCEKP_Died", self->id);
const auto& encryptKeyProxy = self->db.serverInfo->get().encryptKeyProxy;
EncryptKeyProxySingleton(encryptKeyProxy).halt(self, encryptKeyProxy.get().locality.processId());
EncryptKeyProxySingleton(encryptKeyProxy).halt(*self, encryptKeyProxy.get().locality.processId());
self->db.clearInterf(ProcessClass::EncryptKeyProxyClass);
}
when(wait(self->recruitEncryptKeyProxy.onChange())) {}
@ -2569,22 +2374,27 @@ ACTOR Future<Void> watchBlobRestoreCommand(ClusterControllerData* self) {
tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
Optional<Value> blobRestoreCommand = wait(tr->get(blobRestoreCommandKey));
if (blobRestoreCommand.present()) {
Standalone<BlobRestoreStatus> status = decodeBlobRestoreStatus(blobRestoreCommand.get());
TraceEvent("WatchBlobRestoreCommand").detail("Progress", status.progress).detail("Phase", status.phase);
state Standalone<BlobRestoreStatus> status = decodeBlobRestoreStatus(blobRestoreCommand.get());
TraceEvent("WatchBlobRestore", self->id).detail("Phase", status.phase);
if (status.phase == BlobRestorePhase::INIT) {
self->db.blobRestoreEnabled.set(true);
if (self->db.blobGranulesEnabled.get()) {
wait(updateRestoreStatus(
self->cx, normalKeys, BlobRestoreStatus(BlobRestorePhase::STARTING_MIGRATOR), {}));
const auto& blobManager = self->db.serverInfo->get().blobManager;
if (blobManager.present()) {
BlobManagerSingleton(blobManager)
.haltBlobGranules(self, blobManager.get().locality.processId());
.haltBlobGranules(*self, blobManager.get().locality.processId());
}
const auto& blobMigrator = self->db.serverInfo->get().blobMigrator;
if (blobMigrator.present()) {
BlobMigratorSingleton(blobMigrator).halt(self, blobMigrator.get().locality.processId());
BlobMigratorSingleton(blobMigrator).halt(*self, blobMigrator.get().locality.processId());
}
} else {
TraceEvent("SkipBlobRestoreInitCommand", self->id).log();
wait(updateRestoreStatus(self->cx, normalKeys, BlobRestoreStatus(BlobRestorePhase::ERROR), {}));
}
}
self->db.blobRestoreEnabled.set(status.phase < BlobRestorePhase::DONE);
}
state Future<Void> watch = tr->watch(blobRestoreCommandKey);
@ -2647,7 +2457,7 @@ ACTOR Future<Void> startBlobMigrator(ClusterControllerData* self, double waitTim
TraceEvent("CCHaltBlobMigratorAfterRecruit", self->id)
.detail("MGID", blobMigrator.get().id())
.detail("DcID", printable(self->clusterControllerDcId));
BlobMigratorSingleton(blobMigrator).halt(self, blobMigrator.get().locality.processId());
BlobMigratorSingleton(blobMigrator).halt(*self, blobMigrator.get().locality.processId());
}
if (!blobMigrator.present() || blobMigrator.get().id() != interf.get().id()) {
self->db.setBlobMigrator(interf.get());
@ -2677,8 +2487,8 @@ ACTOR Future<Void> monitorBlobMigrator(ClusterControllerData* self) {
loop {
choose {
when(wait(wfClient)) {
TraceEvent("CCBlobMigratorDied", self->id)
.detail("MGID", self->db.serverInfo->get().blobMigrator.get().id());
UID mgID = self->db.serverInfo->get().blobMigrator.get().id();
TraceEvent("CCBlobMigratorDied", self->id).detail("MGID", mgID);
self->db.clearInterf(ProcessClass::BlobMigratorClass);
break;
}
@ -2752,7 +2562,7 @@ ACTOR Future<Void> startBlobManager(ClusterControllerData* self, double waitTime
TraceEvent("CCHaltBlobManagerAfterRecruit", self->id)
.detail("BMID", blobManager.get().id())
.detail("DcID", printable(self->clusterControllerDcId));
BlobManagerSingleton(blobManager).halt(self, blobManager.get().locality.processId());
BlobManagerSingleton(blobManager).halt(*self, blobManager.get().locality.processId());
}
if (!blobManager.present() || blobManager.get().id() != interf.get().id()) {
self->db.setBlobManager(interf.get());
@ -2809,7 +2619,7 @@ ACTOR Future<Void> monitorBlobManager(ClusterControllerData* self) {
when(wait(wfClient)) {
const auto& blobManager = self->db.serverInfo->get().blobManager;
TraceEvent("CCBlobManagerDied", self->id).detail("BMID", blobManager.get().id());
BlobManagerSingleton(blobManager).halt(self, blobManager.get().locality.processId());
BlobManagerSingleton(blobManager).halt(*self, blobManager.get().locality.processId());
self->db.clearInterf(ProcessClass::BlobManagerClass);
break;
}
@ -2821,10 +2631,11 @@ ACTOR Future<Void> monitorBlobManager(ClusterControllerData* self) {
if (!self->db.blobGranulesEnabled.get()) {
const auto& blobManager = self->db.serverInfo->get().blobManager;
BlobManagerSingleton(blobManager)
.haltBlobGranules(self, blobManager.get().locality.processId());
.haltBlobGranules(*self, blobManager.get().locality.processId());
if (self->db.blobRestoreEnabled.get()) {
const auto& blobMigrator = self->db.serverInfo->get().blobMigrator;
BlobMigratorSingleton(blobMigrator).halt(self, blobMigrator.get().locality.processId());
BlobMigratorSingleton(blobMigrator)
.halt(*self, blobMigrator.get().locality.processId());
}
break;
}

View File

@ -208,11 +208,13 @@ int64_t getMaxShardSize(double dbSizeEstimate) {
(int64_t)SERVER_KNOBS->MAX_SHARD_BYTES);
}
ShardSizeBounds calculateShardSizeBounds(const KeyRange& keys,
const Reference<AsyncVar<Optional<ShardMetrics>>>& shardMetrics,
const BandwidthStatus& bandwidthStatus,
PromiseStream<KeyRange> readHotShard) {
// Returns the shard size bounds as well as whether `keys` a read hot shard.
std::pair<ShardSizeBounds, bool> calculateShardSizeBounds(
const KeyRange& keys,
const Reference<AsyncVar<Optional<ShardMetrics>>>& shardMetrics,
const BandwidthStatus& bandwidthStatus) {
ShardSizeBounds bounds = ShardSizeBounds::shardSizeBoundsBeforeTrack();
bool readHotShard = false;
if (shardMetrics->get().present()) {
auto bytes = shardMetrics->get().get().metrics.bytes;
auto readBandwidthStatus = getReadBandwidthStatus(shardMetrics->get().get().metrics);
@ -252,15 +254,13 @@ ShardSizeBounds calculateShardSizeBounds(const KeyRange& keys,
SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS *
(1.0 - SERVER_KNOBS->SHARD_MAX_BYTES_READ_PER_KSEC_JITTER);
bounds.permittedError.bytesReadPerKSecond = bounds.min.bytesReadPerKSecond / 4;
// TraceEvent("RHDTriggerReadHotLoggingForShard")
// .detail("ShardBegin", keys.begin.printable().c_str())
// .detail("ShardEnd", keys.end.printable().c_str());
readHotShard.send(keys);
readHotShard = true;
} else {
ASSERT(false);
}
}
return bounds;
return { bounds, readHotShard };
}
ACTOR Future<Void> trackShardMetrics(DataDistributionTracker::SafeAccessor self,
@ -285,7 +285,15 @@ ACTOR Future<Void> trackShardMetrics(DataDistributionTracker::SafeAccessor self,
try {
loop {
state ShardSizeBounds bounds;
bounds = calculateShardSizeBounds(keys, shardMetrics, bandwidthStatus, self()->readHotShard);
bool readHotShard;
std::tie(bounds, readHotShard) = calculateShardSizeBounds(keys, shardMetrics, bandwidthStatus);
if (readHotShard) {
// TraceEvent("RHDTriggerReadHotLoggingForShard")
// .detail("ShardBegin", keys.begin.printable().c_str())
// .detail("ShardEnd", keys.end.printable().c_str());
self()->readHotShard.send(keys);
}
loop {
// metrics.second is the number of key-ranges (i.e., shards) in the 'keys' key-range
@ -327,6 +335,7 @@ ACTOR Future<Void> trackShardMetrics(DataDistributionTracker::SafeAccessor self,
keys, metrics.first.get(), shardMetrics->get().get().metrics, initWithNewMetrics);
if (needToMove) {
// Do we need to update shardsAffectedByTeamFailure here?
// TODO(zhewu): move this to physical shard tracker that does shard split based on size.
self()->output.send(
RelocateShard(keys,
DataMovementReason::ENFORCE_MOVE_OUT_OF_PHYSICAL_SHARD,
@ -1544,6 +1553,83 @@ FDB_DEFINE_BOOLEAN_PARAM(InOverSizePhysicalShard);
FDB_DEFINE_BOOLEAN_PARAM(PhysicalShardAvailable);
FDB_DEFINE_BOOLEAN_PARAM(MoveKeyRangeOutPhysicalShard);
// Tracks storage metrics for `keys` and updates `physicalShardStats` which is the stats for the physical shard owning
// this key range. This function is similar to `trackShardMetrics()` and altered for physical shard. This meant to be
// temporary. Eventually, we want a new interface to track physical shard metrics more efficiently.
ACTOR Future<Void> trackKeyRangeInPhysicalShardMetrics(
Reference<IDDTxnProcessor> db,
KeyRange keys,
Reference<AsyncVar<Optional<ShardMetrics>>> shardMetrics,
Reference<AsyncVar<Optional<StorageMetrics>>> physicalShardStats) {
state BandwidthStatus bandwidthStatus =
shardMetrics->get().present() ? getBandwidthStatus(shardMetrics->get().get().metrics) : BandwidthStatusNormal;
state double lastLowBandwidthStartTime =
shardMetrics->get().present() ? shardMetrics->get().get().lastLowBandwidthStartTime : now();
state int shardCount = shardMetrics->get().present() ? shardMetrics->get().get().shardCount : 1;
wait(delay(0, TaskPriority::DataDistribution));
/*TraceEvent("trackKeyRangeInPhysicalShardMetricsStarting")
.detail("Keys", keys)
.detail("TrackedBytesInitiallyPresent", shardMetrics->get().present())
.detail("StartingMetrics", shardMetrics->get().present() ? shardMetrics->get().get().metrics.bytes : 0)
.detail("StartingMerges", shardMetrics->get().present() ? shardMetrics->get().get().merges : 0);*/
loop {
state ShardSizeBounds bounds;
bool readHotShard;
std::tie(bounds, readHotShard) = calculateShardSizeBounds(keys, shardMetrics, bandwidthStatus);
loop {
// metrics.second is the number of key-ranges (i.e., shards) in the 'keys' key-range
std::pair<Optional<StorageMetrics>, int> metrics =
wait(db->waitStorageMetrics(keys,
bounds.min,
bounds.max,
bounds.permittedError,
CLIENT_KNOBS->STORAGE_METRICS_SHARD_LIMIT,
shardCount));
if (metrics.first.present()) {
BandwidthStatus newBandwidthStatus = getBandwidthStatus(metrics.first.get());
if (newBandwidthStatus == BandwidthStatusLow && bandwidthStatus != BandwidthStatusLow) {
lastLowBandwidthStartTime = now();
}
bandwidthStatus = newBandwidthStatus;
// Update current physical shard aggregated stats;
if (!physicalShardStats->get().present()) {
physicalShardStats->set(metrics.first.get());
} else {
if (!shardMetrics->get().present()) {
// We collect key range stats for the first time.
physicalShardStats->set(physicalShardStats->get().get() + metrics.first.get());
} else {
physicalShardStats->set(physicalShardStats->get().get() - shardMetrics->get().get().metrics +
metrics.first.get());
}
}
shardMetrics->set(ShardMetrics(metrics.first.get(), lastLowBandwidthStartTime, shardCount));
break;
} else {
shardCount = metrics.second;
if (shardMetrics->get().present()) {
auto newShardMetrics = shardMetrics->get().get();
newShardMetrics.shardCount = shardCount;
shardMetrics->set(newShardMetrics);
}
}
}
}
}
void PhysicalShardCollection::PhysicalShard::insertNewRangeData(const KeyRange& newRange) {
RangeData data;
data.stats = makeReference<AsyncVar<Optional<ShardMetrics>>>();
data.trackMetrics = trackKeyRangeInPhysicalShardMetrics(txnProcessor, newRange, data.stats, stats);
auto it = rangeData.emplace(newRange, data);
ASSERT(it.second);
}
void PhysicalShardCollection::PhysicalShard::addRange(const KeyRange& newRange) {
if (g_network->isSimulated()) {
// Test that new range must not overlap with any existing range in this shard.
@ -1552,9 +1638,7 @@ void PhysicalShardCollection::PhysicalShard::addRange(const KeyRange& newRange)
}
}
// TODO(zhewu): add metrics tracking actor.
RangeData data;
rangeData.emplace(newRange, data);
insertNewRangeData(newRange);
}
void PhysicalShardCollection::PhysicalShard::removeRange(const KeyRange& outRange) {
@ -1569,9 +1653,7 @@ void PhysicalShardCollection::PhysicalShard::removeRange(const KeyRange& outRang
std::vector<KeyRangeRef> remainingRanges = range - outRange;
for (auto& r : remainingRanges) {
ASSERT(r != range);
// TODO(zhewu): add metrics tracking actor.
RangeData data;
rangeData.emplace(r, data);
insertNewRangeData(r);
}
// Must erase last since `remainingRanges` uses data in `range`.
rangeData.erase(range);
@ -1623,7 +1705,7 @@ void PhysicalShardCollection::insertPhysicalShardToCollection(uint64_t physicalS
ASSERT(physicalShardID != anonymousShardId.first() && physicalShardID != UID().first());
ASSERT(physicalShardInstances.count(physicalShardID) == 0);
physicalShardInstances.insert(
std::make_pair(physicalShardID, PhysicalShard(physicalShardID, metrics, teams, whenCreated)));
std::make_pair(physicalShardID, PhysicalShard(txnProcessor, physicalShardID, metrics, teams, whenCreated)));
return;
}

View File

@ -21,6 +21,7 @@
#include "fdbserver/DDTeamCollection.h"
#include "flow/Trace.h"
#include "flow/actorcompiler.h" // This must be the last #include.
#include <climits>
FDB_DEFINE_BOOLEAN_PARAM(IsPrimary);
FDB_DEFINE_BOOLEAN_PARAM(IsInitialTeam);
@ -575,6 +576,14 @@ public:
state int teamsToBuild;
teamsToBuild = std::max(0, std::min(desiredTeams - teamCount, maxTeams - totalTeamCount));
if (teamCount == 0 && teamsToBuild == 0 && SERVER_KNOBS->DD_BUILD_EXTRA_TEAMS_OVERRIDE > 0) {
// Use DD_BUILD_EXTRA_TEAMS_OVERRIDE > 0 as the feature flag: Set to 0 to disable it
TraceEvent(SevWarnAlways, "BuildServerTeamsHaveTooManyUnhealthyTeams")
.detail("Hint", "Build teams may stuck and prevent DD from relocating data")
.detail("BuildExtraServerTeamsOverride", SERVER_KNOBS->DD_BUILD_EXTRA_TEAMS_OVERRIDE);
teamsToBuild = SERVER_KNOBS->DD_BUILD_EXTRA_TEAMS_OVERRIDE;
}
TraceEvent("BuildTeamsBegin", self->distributorId)
.detail("TeamsToBuild", teamsToBuild)
.detail("DesiredTeams", desiredTeams)
@ -583,7 +592,8 @@ public:
.detail("PerpetualWigglingTeams", wigglingTeams)
.detail("UniqueMachines", uniqueMachines)
.detail("TeamSize", self->configuration.storageTeamSize)
.detail("Servers", serverCount)
.detail("Servers", self->server_info.size())
.detail("HealthyServers", serverCount)
.detail("CurrentTrackedServerTeams", self->teams.size())
.detail("HealthyTeamCount", teamCount)
.detail("TotalTeamCount", totalTeamCount)
@ -640,6 +650,10 @@ public:
}
} else {
self->lastBuildTeamsFailed = true;
TraceEvent(SevWarnAlways, "BuildTeamsNotEnoughUniqueMachines", self->distributorId)
.detail("Primary", self->primary)
.detail("UniqueMachines", uniqueMachines)
.detail("Replication", self->configuration.storageTeamSize);
}
self->evaluateTeamQuality();
@ -1957,6 +1971,40 @@ public:
return Void();
}
ACTOR static Future<Void> perpetualStorageWiggleRest(DDTeamCollection* self) {
state bool takeRest = true;
state Promise<int64_t> avgShardBytes;
while (takeRest) {
// a minimal delay to avoid excluding and including SS too fast
wait(delay(SERVER_KNOBS->PERPETUAL_WIGGLE_DELAY));
avgShardBytes.reset();
self->getAverageShardBytes.send(avgShardBytes);
int64_t avgBytes = wait(avgShardBytes.getFuture());
double ratio = self->loadBytesBalanceRatio(avgBytes * SERVER_KNOBS->PERPETUAL_WIGGLE_SMALL_LOAD_RATIO);
bool imbalance = ratio < SERVER_KNOBS->PERPETUAL_WIGGLE_MIN_BYTES_BALANCE_RATIO;
CODE_PROBE(imbalance, "Perpetual Wiggle pause because cluster is imbalance.");
// there must not have other teams to place wiggled data
takeRest = self->server_info.size() <= self->configuration.storageTeamSize ||
self->machine_info.size() < self->configuration.storageTeamSize || imbalance;
// log the extra delay and change the wiggler state
if (takeRest) {
self->storageWiggler->setWiggleState(StorageWiggler::PAUSE);
if (self->configuration.storageMigrationType == StorageMigrationType::GRADUAL) {
TraceEvent(SevWarn, "PerpetualStorageWiggleSleep", self->distributorId)
.suppressFor(SERVER_KNOBS->PERPETUAL_WIGGLE_DELAY * 4)
.detail("BytesBalanceRatio", ratio)
.detail("ServerSize", self->server_info.size())
.detail("MachineSize", self->machine_info.size())
.detail("StorageTeamSize", self->configuration.storageTeamSize);
}
}
}
return Void();
}
ACTOR static Future<Void> perpetualStorageWiggleIterator(DDTeamCollection* teamCollection,
AsyncVar<bool>* stopSignal,
FutureStream<Void> finishStorageWiggleSignal) {
@ -1964,24 +2012,9 @@ public:
choose {
when(wait(stopSignal->onChange())) {}
when(waitNext(finishStorageWiggleSignal)) {
state bool takeRest = true; // delay to avoid delete and update ServerList too frequently
while (takeRest) {
wait(delayJittered(SERVER_KNOBS->PERPETUAL_WIGGLE_DELAY));
// there must not have other teams to place wiggled data
takeRest =
teamCollection->server_info.size() <= teamCollection->configuration.storageTeamSize ||
teamCollection->machine_info.size() < teamCollection->configuration.storageTeamSize;
if (takeRest) {
teamCollection->storageWiggler->setWiggleState(StorageWiggler::PAUSE);
if (teamCollection->configuration.storageMigrationType == StorageMigrationType::GRADUAL) {
TraceEvent(SevWarn, "PerpetualStorageWiggleSleep", teamCollection->distributorId)
.suppressFor(SERVER_KNOBS->PERPETUAL_WIGGLE_DELAY * 4)
.detail("ServerSize", teamCollection->server_info.size())
.detail("MachineSize", teamCollection->machine_info.size())
.detail("StorageTeamSize", teamCollection->configuration.storageTeamSize);
}
}
}
// delay to avoid delete and update ServerList too frequently, which could result busy loop or over
// utilize the disk of other active SS
wait(perpetualStorageWiggleRest(teamCollection));
wait(updateNextWigglingStorageID(teamCollection));
}
}
@ -3072,6 +3105,7 @@ public:
TraceEvent e("ServerStatus", self->getDistributorId());
e.detail("ServerUID", uid)
.detail("MachineIsValid", server_info[uid]->machine.isValid())
.detail("IsMachineHealthy", self->isMachineHealthy(server_info[uid]->machine))
.detail("MachineTeamSize",
server_info[uid]->machine.isValid() ? server_info[uid]->machine->machineTeams.size() : -1)
.detail("Primary", self->isPrimary());
@ -3116,13 +3150,13 @@ public:
TraceEvent("MachineInfo", self->getDistributorId())
.detail("Size", machine_info.size())
.detail("Primary", self->isPrimary());
state std::map<Standalone<StringRef>, Reference<TCMachineInfo>>::iterator machine = machine_info.begin();
state bool isMachineHealthy = false;
for (i = 0; i < machine_info.size(); i++) {
Reference<TCMachineInfo> _machine = machine->second;
if (!_machine.isValid() || machine_info.find(_machine->machineID) == machine_info.end() ||
_machine->serversOnMachine.empty()) {
bool machineIDFound = machine_info.find(_machine->machineID) != machine_info.end();
bool zeroHealthyServersOnMachine = true;
if (!_machine.isValid() || !machineIDFound || _machine->serversOnMachine.empty()) {
isMachineHealthy = false;
}
@ -3134,13 +3168,17 @@ public:
auto it = server_status.find(server->getId());
if (it != server_status.end() && !it->second.isUnhealthy()) {
isMachineHealthy = true;
zeroHealthyServersOnMachine = false;
break;
}
}
isMachineHealthy = false;
TraceEvent("MachineInfo", self->getDistributorId())
.detail("MachineInfoIndex", i)
.detail("Healthy", isMachineHealthy)
.detail("MachineIDFound", machineIDFound)
.detail("ZeroServersOnMachine", _machine->serversOnMachine.empty())
.detail("ZeroHealthyServersOnMachine", zeroHealthyServersOnMachine)
.detail("MachineID", machine->first.contents().toString())
.detail("MachineTeamOwned", machine->second->machineTeams.size())
.detail("ServerNumOnMachine", machine->second->serversOnMachine.size())
@ -3255,10 +3293,15 @@ void DDTeamCollection::traceServerInfo() const {
.detail("StoreType", server->getStoreType().toString())
.detail("InDesiredDC", server->isInDesiredDC());
}
for (auto& [serverID, server] : server_info) {
i = 0;
for (auto& server : server_info) {
const UID& serverID = server.first;
const ServerStatus& status = server_status.get(serverID);
TraceEvent("ServerStatus", distributorId)
.detail("ServerInfoIndex", i++)
.detail("ServerID", serverID)
.detail("Healthy", !server_status.get(serverID).isUnhealthy())
.detail("Healthy", !status.isUnhealthy())
.detail("StatusString", status.toString())
.detail("MachineIsValid", get(server_info, serverID)->machine.isValid())
.detail("MachineTeamSize",
get(server_info, serverID)->machine.isValid()
@ -3394,6 +3437,39 @@ Future<Void> DDTeamCollection::removeBadTeams() {
return DDTeamCollectionImpl::removeBadTeams(this);
}
double DDTeamCollection::loadBytesBalanceRatio(int64_t smallLoadThreshold) const {
double minLoadBytes = std::numeric_limits<double>::max();
double totalLoadBytes = 0;
int count = 0;
for (auto& [id, s] : server_info) {
// If a healthy SS don't have storage metrics, skip this round
if (server_status.get(s->getId()).isUnhealthy() || !s->metricsPresent()) {
TraceEvent(SevDebug, "LoadBytesBalanceRatioNoMetrics").detail("Server", id);
return 0;
}
double load = s->loadBytes();
totalLoadBytes += load;
++count;
minLoadBytes = std::min(minLoadBytes, load);
}
TraceEvent(SevDebug, "LoadBytesBalanceRatioMetrics")
.detail("TotalLoad", totalLoadBytes)
.detail("MinLoadBytes", minLoadBytes)
.detail("SmallLoadThreshold", smallLoadThreshold)
.detail("Count", count);
// avoid division-by-zero
double avgLoad = totalLoadBytes / count;
if (totalLoadBytes == 0 || avgLoad < smallLoadThreshold) {
CODE_PROBE(true, "The cluster load is small enough to ignore load bytes balance.");
return 1;
}
return minLoadBytes / avgLoad;
}
Future<Void> DDTeamCollection::storageServerFailureTracker(TCServerInfo* server,
ServerStatus* status,
Version addedVersion) {
@ -3542,7 +3618,8 @@ DDTeamCollection::DDTeamCollection(DDTeamCollectionInitParams const& params)
restartRecruiting(SERVER_KNOBS->DEBOUNCE_RECRUITING_DELAY), healthyTeamCount(0),
zeroHealthyTeams(params.zeroHealthyTeams), optimalTeamCount(0), zeroOptimalTeams(true), isTssRecruiting(false),
includedDCs(params.includedDCs), otherTrackedDCs(params.otherTrackedDCs),
processingUnhealthy(params.processingUnhealthy), readyToStart(params.readyToStart),
processingUnhealthy(params.processingUnhealthy), getAverageShardBytes(params.getAverageShardBytes),
readyToStart(params.readyToStart),
checkTeamDelay(delay(SERVER_KNOBS->CHECK_TEAM_DELAY, TaskPriority::DataDistribution)), badTeamRemover(Void()),
checkInvalidLocalities(Void()), wrongStoreTypeRemover(Void()), clearHealthyZoneFuture(true),
medianAvailableSpace(SERVER_KNOBS->MIN_AVAILABLE_SPACE_RATIO), lastMedianAvailableSpaceUpdate(0),
@ -3995,6 +4072,7 @@ void DDTeamCollection::traceAllInfo(bool shouldPrint) const {
}
}
// TODO: flush trace log to avoid trace buffer overflow when DD has too many servers and teams
TraceEvent("TraceAllInfo", distributorId).detail("Primary", primary);
traceConfigInfo();
traceServerInfo();
@ -4162,10 +4240,13 @@ int DDTeamCollection::addBestMachineTeams(int machineTeamsToBuild) {
addMachineTeam(machines);
addedMachineTeams++;
} else {
traceAllInfo(true);
// When too many teams exist in simulation, traceAllInfo will buffer too many trace logs before
// trace has a chance to flush its buffer, which causes assertion failure.
traceAllInfo(!g_network->isSimulated());
TraceEvent(SevWarn, "DataDistributionBuildTeams", distributorId)
.detail("Primary", primary)
.detail("Reason", "Unable to make desired machine Teams");
.detail("Reason", "Unable to make desired machine Teams")
.detail("Hint", "Check TraceAllInfo event");
lastBuildTeamsFailed = true;
break;
}
@ -4454,13 +4535,34 @@ int DDTeamCollection::addTeamsBestOf(int teamsToBuild, int desiredTeams, int max
// machineTeamsToBuild mimics how the teamsToBuild is calculated in buildTeams()
int machineTeamsToBuild =
std::max(0, std::min(desiredMachineTeams - healthyMachineTeamCount, maxMachineTeams - totalMachineTeamCount));
if (healthyMachineTeamCount == 0 && machineTeamsToBuild == 0 && SERVER_KNOBS->DD_BUILD_EXTRA_TEAMS_OVERRIDE > 0) {
// Use DD_BUILD_EXTRA_TEAMS_OVERRIDE > 0 as the feature flag: Set to 0 to disable it
TraceEvent(SevWarnAlways, "BuildMachineTeamsHaveTooManyUnhealthyMachineTeams")
.detail("Hint", "Build teams may stuck and prevent DD from relocating data")
.detail("BuildExtraMachineTeamsOverride", SERVER_KNOBS->DD_BUILD_EXTRA_TEAMS_OVERRIDE);
machineTeamsToBuild = SERVER_KNOBS->DD_BUILD_EXTRA_TEAMS_OVERRIDE;
}
if (g_network->isSimulated() && deterministicRandom()->random01() < 0.1) {
// Test when the system has lots of unhealthy machine teams, which may prevent TC from building new teams.
// The scenario creates a deadlock situation that DD cannot relocate data.
int totalMachineTeams = nChooseK(machine_info.size(), configuration.storageTeamSize);
TraceEvent("BuildMachineTeams")
.detail("Primary", primary)
.detail("CalculatedMachineTeamsToBuild", machineTeamsToBuild)
.detail("OverwriteMachineTeamsToBuildForTesting", totalMachineTeams);
machineTeamsToBuild = totalMachineTeams;
}
{
TraceEvent te("BuildMachineTeams");
te.detail("TotalHealthyMachine", totalHealthyMachineCount)
te.detail("Primary", primary)
.detail("TotalMachines", machine_info.size())
.detail("TotalHealthyMachine", totalHealthyMachineCount)
.detail("HealthyMachineTeamCount", healthyMachineTeamCount)
.detail("DesiredMachineTeams", desiredMachineTeams)
.detail("MaxMachineTeams", maxMachineTeams)
.detail("TotalMachineTeams", totalMachineTeamCount)
.detail("MachineTeamsToBuild", machineTeamsToBuild);
// Pre-build all machine teams until we have the desired number of machine teams
if (machineTeamsToBuild > 0 || notEnoughMachineTeamsForAMachine()) {
@ -4764,6 +4866,7 @@ Reference<TCMachineInfo> DDTeamCollection::checkAndCreateMachine(Reference<TCSer
machineInfo->serversOnMachine.push_back(server);
}
server->machine = machineInfo;
ASSERT(machineInfo->machineID == machine_id); // invariant for TC to work
return machineInfo;
}
@ -5104,7 +5207,8 @@ public:
makeReference<AsyncVar<bool>>(false),
PromiseStream<GetMetricsRequest>(),
Promise<UID>(),
PromiseStream<Promise<int>>() }));
PromiseStream<Promise<int>>(),
PromiseStream<Promise<int64_t>>() }));
for (int id = 1; id <= processCount; ++id) {
UID uid(id, 0);
@ -5148,7 +5252,8 @@ public:
makeReference<AsyncVar<bool>>(false),
PromiseStream<GetMetricsRequest>(),
Promise<UID>(),
PromiseStream<Promise<int>>() }));
PromiseStream<Promise<int>>(),
PromiseStream<Promise<int64_t>>() }));
for (int id = 1; id <= processCount; id++) {
UID uid(id, 0);

View File

@ -723,7 +723,8 @@ ACTOR Future<Void> dataDistribution(Reference<DataDistributor> self,
processingWiggle,
getShardMetrics,
removeFailedServer,
getUnhealthyRelocationCount });
getUnhealthyRelocationCount,
getAverageShardBytes });
teamCollectionsPtrs.push_back(primaryTeamCollection.getPtr());
auto recruitStorage = IAsyncListener<RequestStream<RecruitStorageRequest>>::create(
self->dbInfo, [](auto const& info) { return info.clusterInterface.recruitStorage; });
@ -744,7 +745,8 @@ ACTOR Future<Void> dataDistribution(Reference<DataDistributor> self,
processingWiggle,
getShardMetrics,
removeFailedServer,
getUnhealthyRelocationCount });
getUnhealthyRelocationCount,
getAverageShardBytes });
teamCollectionsPtrs.push_back(remoteTeamCollection.getPtr());
remoteTeamCollection->teamCollections = teamCollectionsPtrs;
actors.push_back(reportErrorsExcept(DDTeamCollection::run(remoteTeamCollection,

77
fdbserver/IPager.cpp Normal file
View File

@ -0,0 +1,77 @@
/*
* IPager.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "fdbserver/IPager.h"
#include "flow/IRandom.h"
#include "flow/UnitTest.h"
#include <limits>
TEST_CASE("/fdbserver/IPager/ArenaPage/PageContentChecksum") {
auto& g_knobs = IKnobCollection::getMutableGlobalKnobCollection();
for (uint8_t et = 0; et < EncodingType::MAX_ENCODING_TYPE; et++) {
constexpr int PAGE_SIZE = 8 * 1024;
EncodingType encodingType = (EncodingType)et;
Reference<ArenaPage> page = makeReference<ArenaPage>(PAGE_SIZE, PAGE_SIZE);
page->init(encodingType, PageType::BTreeNode, 1);
deterministicRandom()->randomBytes(page->mutateData(), page->dataSize());
PhysicalPageID pageID = deterministicRandom()->randomUInt32();
if (encodingType == AESEncryption || encodingType == AESEncryptionWithAuth) {
uint8_t cipherKeyBytes[AES_256_KEY_LENGTH];
deterministicRandom()->randomBytes(cipherKeyBytes, AES_256_KEY_LENGTH);
Reference<BlobCipherKey> cipherKey =
makeReference<BlobCipherKey>(0 /*domainId*/,
1 /*baseCipherId*/,
cipherKeyBytes,
AES_256_KEY_LENGTH,
std::numeric_limits<int64_t>::max() /*refreshAt*/,
std::numeric_limits<int64_t>::max() /*expireAt*/
);
page->encryptionKey.aesKey.cipherTextKey = cipherKey;
page->encryptionKey.aesKey.cipherHeaderKey = cipherKey;
if (encodingType == AESEncryption) {
g_knobs.setKnob("encrypt_header_auth_token_enabled", KnobValueRef::create(bool{ false }));
} else {
g_knobs.setKnob("encrypt_header_auth_token_enabled", KnobValueRef::create(bool{ true }));
g_knobs.setKnob("encrypt_header_auth_token_algo", KnobValueRef::create(int{ 1 }));
}
} else if (encodingType == XOREncryption_TestOnly) {
page->encryptionKey.xorKey = deterministicRandom()->randomInt(0, std::numeric_limits<uint8_t>::max());
page->encryptionKey.xorWith = deterministicRandom()->randomInt(0, std::numeric_limits<uint8_t>::max());
}
page->setWriteInfo(pageID, 1 /*version*/);
page->preWrite(pageID);
// Randomly corrupt the data.
uint8_t* byte = page->mutateData() + deterministicRandom()->randomInt(0, page->dataSize());
*byte = ~(*byte);
page->postReadHeader(pageID);
try {
// Assert checksum failure is thrown.
page->postReadPayload(pageID);
UNREACHABLE();
} catch (Error& e) {
if (encodingType == AESEncryptionWithAuth) {
ASSERT_EQ(e.code(), error_code_encrypt_header_authtoken_mismatch);
} else {
ASSERT_EQ(e.code(), error_code_page_decoding_failed);
}
}
}
return Void();
}

View File

@ -82,6 +82,9 @@ public:
LatencySample commitLatency;
LatencySample commitQueueLatency;
LatencySample dbWriteLatency;
std::vector<std::shared_ptr<LatencySample>> readLatency;
std::vector<std::shared_ptr<LatencySample>> scanLatency;
std::vector<std::shared_ptr<LatencySample>> readQueueLatency;
void setClosing() { this->closing = true; }
bool isClosing() const { return this->closing; }
@ -116,7 +119,22 @@ SharedRocksDBState::SharedRocksDBState(UID id)
dbWriteLatency(LatencySample("RocksDBWriteLatency",
id,
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY)) {}
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY)) {
for (int i = 0; i < SERVER_KNOBS->ROCKSDB_READ_PARALLELISM; i++) {
readLatency.push_back(std::make_shared<LatencySample>(format("RocksDBReadLatency-%d", i),
id,
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY));
scanLatency.push_back(std::make_shared<LatencySample>(format("RocksDBScanLatency-%d", i),
id,
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY));
readQueueLatency.push_back(std::make_shared<LatencySample>(format("RocksDBReadQueueLatency-%d", i),
id,
SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
SERVER_KNOBS->LATENCY_SKETCH_ACCURACY));
}
}
rocksdb::ColumnFamilyOptions SharedRocksDBState::initialCfOptions() {
rocksdb::ColumnFamilyOptions options;
@ -199,7 +217,7 @@ rocksdb::DBOptions SharedRocksDBState::initialDbOptions() {
options.statistics = rocksdb::CreateDBStatistics();
options.statistics->set_stats_level(rocksdb::StatsLevel(SERVER_KNOBS->ROCKSDB_STATS_LEVEL));
options.db_log_dir = SERVER_KNOBS->LOG_DIRECTORY;
options.db_log_dir = g_network->isSimulated() ? "" : SERVER_KNOBS->LOG_DIRECTORY;
if (SERVER_KNOBS->ROCKSDB_MUTE_LOGS) {
options.info_log = std::make_shared<NullRocksDBLogForwarder>();
@ -1461,7 +1479,8 @@ struct RocksDBKeyValueStore : IKeyValueStore {
if (doPerfContextMetrics) {
perfContextMetrics->reset();
}
double readBeginTime = timer_monotonic();
const double readBeginTime = timer_monotonic();
sharedState->readQueueLatency[threadIndex]->addMeasurement(readBeginTime - a.startTime);
if (a.getHistograms) {
metricPromiseStream->send(
std::make_pair(ROCKSDB_READVALUE_QUEUEWAIT_HISTOGRAM.toString(), readBeginTime - a.startTime));
@ -1515,16 +1534,17 @@ struct RocksDBKeyValueStore : IKeyValueStore {
a.result.sendError(statusToError(s));
}
const double endTime = timer_monotonic();
if (a.getHistograms) {
double currTime = timer_monotonic();
metricPromiseStream->send(
std::make_pair(ROCKSDB_READVALUE_ACTION_HISTOGRAM.toString(), currTime - readBeginTime));
std::make_pair(ROCKSDB_READVALUE_ACTION_HISTOGRAM.toString(), endTime - readBeginTime));
metricPromiseStream->send(
std::make_pair(ROCKSDB_READVALUE_LATENCY_HISTOGRAM.toString(), currTime - a.startTime));
std::make_pair(ROCKSDB_READVALUE_LATENCY_HISTOGRAM.toString(), endTime - a.startTime));
}
if (doPerfContextMetrics) {
perfContextMetrics->set(threadIndex);
}
sharedState->readLatency[threadIndex]->addMeasurement(endTime - readBeginTime);
}
struct ReadValuePrefixAction : TypedAction<Reader, ReadValuePrefixAction> {
@ -1546,7 +1566,8 @@ struct RocksDBKeyValueStore : IKeyValueStore {
if (doPerfContextMetrics) {
perfContextMetrics->reset();
}
double readBeginTime = timer_monotonic();
const double readBeginTime = timer_monotonic();
sharedState->readQueueLatency[threadIndex]->addMeasurement(readBeginTime - a.startTime);
if (a.getHistograms) {
metricPromiseStream->send(
std::make_pair(ROCKSDB_READPREFIX_QUEUEWAIT_HISTOGRAM.toString(), readBeginTime - a.startTime));
@ -1598,16 +1619,17 @@ struct RocksDBKeyValueStore : IKeyValueStore {
logRocksDBError(id, s, "ReadValuePrefix");
a.result.sendError(statusToError(s));
}
const double endTime = timer_monotonic();
if (a.getHistograms) {
double currTime = timer_monotonic();
metricPromiseStream->send(
std::make_pair(ROCKSDB_READPREFIX_ACTION_HISTOGRAM.toString(), currTime - readBeginTime));
std::make_pair(ROCKSDB_READPREFIX_ACTION_HISTOGRAM.toString(), endTime - readBeginTime));
metricPromiseStream->send(
std::make_pair(ROCKSDB_READPREFIX_LATENCY_HISTOGRAM.toString(), currTime - a.startTime));
std::make_pair(ROCKSDB_READPREFIX_LATENCY_HISTOGRAM.toString(), endTime - a.startTime));
}
if (doPerfContextMetrics) {
perfContextMetrics->set(threadIndex);
}
sharedState->readLatency[threadIndex]->addMeasurement(endTime - readBeginTime);
}
struct ReadRangeAction : TypedAction<Reader, ReadRangeAction>, FastAllocated<ReadRangeAction> {
@ -1628,7 +1650,8 @@ struct RocksDBKeyValueStore : IKeyValueStore {
if (doPerfContextMetrics) {
perfContextMetrics->reset();
}
double readBeginTime = timer_monotonic();
const double readBeginTime = timer_monotonic();
sharedState->readQueueLatency[threadIndex]->addMeasurement(readBeginTime - a.startTime);
if (a.getHistograms) {
metricPromiseStream->send(
std::make_pair(ROCKSDB_READRANGE_QUEUEWAIT_HISTOGRAM.toString(), readBeginTime - a.startTime));
@ -1722,16 +1745,17 @@ struct RocksDBKeyValueStore : IKeyValueStore {
result.readThrough = result[result.size() - 1].key;
}
a.result.send(result);
const double endTime = timer_monotonic();
if (a.getHistograms) {
double currTime = timer_monotonic();
metricPromiseStream->send(
std::make_pair(ROCKSDB_READRANGE_ACTION_HISTOGRAM.toString(), currTime - readBeginTime));
std::make_pair(ROCKSDB_READRANGE_ACTION_HISTOGRAM.toString(), endTime - readBeginTime));
metricPromiseStream->send(
std::make_pair(ROCKSDB_READRANGE_LATENCY_HISTOGRAM.toString(), currTime - a.startTime));
std::make_pair(ROCKSDB_READRANGE_LATENCY_HISTOGRAM.toString(), endTime - a.startTime));
}
if (doPerfContextMetrics) {
perfContextMetrics->set(threadIndex);
}
sharedState->scanLatency[threadIndex]->addMeasurement(endTime - readBeginTime);
}
};
@ -2595,15 +2619,23 @@ TEST_CASE("noSim/fdbserver/KeyValueStoreRocksDB/CheckpointRestoreKeyValues") {
std::string checkpointDir = cwd + "checkpoint";
CheckpointRequest request(
latestVersion, { allKeys }, RocksDB, deterministicRandom()->randomUniqueID(), checkpointDir);
latestVersion, { allKeys }, DataMoveRocksCF, deterministicRandom()->randomUniqueID(), checkpointDir);
CheckpointMetaData metaData = wait(kvStore->checkpoint(request));
state ICheckpointReader* cpReader = newCheckpointReader(metaData, deterministicRandom()->randomUniqueID());
wait(cpReader->init(BinaryWriter::toValue(KeyRangeRef("foo"_sr, "foobar"_sr), IncludeVersion())));
TraceEvent(SevDebug, "RocksDBCreatedCheckpoint");
state KeyRange testRange = KeyRangeRef("foo"_sr, "foobar"_sr);
state Standalone<StringRef> token = BinaryWriter::toValue(testRange, IncludeVersion());
state ICheckpointReader* cpReader =
newCheckpointReader(metaData, CheckpointAsKeyValues::True, deterministicRandom()->randomUniqueID());
TraceEvent(SevDebug, "RocksDBCheckpointReaderCreated");
ASSERT(cpReader != nullptr);
wait(cpReader->init(token));
TraceEvent(SevDebug, "RocksDBCheckpointReaderInited");
state std::unique_ptr<ICheckpointIterator> iter = cpReader->getIterator(testRange);
loop {
try {
state RangeResult res =
wait(cpReader->nextKeyValues(CLIENT_KNOBS->REPLY_BYTE_LIMIT, CLIENT_KNOBS->REPLY_BYTE_LIMIT));
wait(iter->nextBatch(CLIENT_KNOBS->REPLY_BYTE_LIMIT, CLIENT_KNOBS->REPLY_BYTE_LIMIT));
state int i = 0;
for (; i < res.size(); ++i) {
Optional<Value> val = wait(kvStore->readValue(res[i].key));
@ -2618,6 +2650,7 @@ TEST_CASE("noSim/fdbserver/KeyValueStoreRocksDB/CheckpointRestoreKeyValues") {
}
}
iter.reset();
std::vector<Future<Void>> closes;
closes.push_back(cpReader->close());
closes.push_back(kvStore->onClosed());

View File

@ -80,15 +80,6 @@ struct ShardedRocksDBKeyValueStore;
using rocksdb::BackgroundErrorReason;
struct RangeLessThan {
inline bool operator()(const KeyRange& l, const KeyRange& r) {
if (l.begin == r.begin) {
return l.end < r.end;
}
return l.begin < r.begin;
}
};
// Returns string representation of RocksDB background error reason.
// Error reason code:
// https://github.com/facebook/rocksdb/blob/12d798ac06bcce36be703b057d5f5f4dab3b270c/include/rocksdb/listener.h#L125
@ -374,7 +365,7 @@ rocksdb::Options getOptions() {
options.write_buffer_size = SERVER_KNOBS->ROCKSDB_CF_WRITE_BUFFER_SIZE;
options.statistics = rocksdb::CreateDBStatistics();
options.statistics->set_stats_level(rocksdb::kExceptHistogramOrTimers);
options.db_log_dir = SERVER_KNOBS->LOG_DIRECTORY;
options.db_log_dir = g_network->isSimulated() ? "" : SERVER_KNOBS->LOG_DIRECTORY;
return options;
}
@ -1085,7 +1076,7 @@ public:
TraceEvent(SevInfo, "ShardedRocksDB", logId)
.detail("PendingDeletionShardQueueSize", pendingDeletionShards.size());
while (!pendingDeletionShards.empty()) {
const auto& id = pendingDeletionShards.front();
const auto id = pendingDeletionShards.front();
auto it = physicalShards.find(id);
if (it == physicalShards.end() || it->second->dataShards.size() != 0) {
pendingDeletionShards.pop_front();
@ -2247,8 +2238,8 @@ struct ShardedRocksDBKeyValueStore : IKeyValueStore {
rocksdb::WriteBatch writeBatch;
if (format == DataMoveRocksCF) {
CheckpointMetaData& checkpoint = a.checkpoints.front();
std::sort(a.ranges.begin(), a.ranges.end(), RangeLessThan());
std::sort(checkpoint.ranges.begin(), checkpoint.ranges.end(), RangeLessThan());
std::sort(a.ranges.begin(), a.ranges.end(), KeyRangeRef::ArbitraryOrder());
std::sort(checkpoint.ranges.begin(), checkpoint.ranges.end(), KeyRangeRef::ArbitraryOrder());
if (a.ranges.empty() || checkpoint.ranges.empty() || a.ranges.size() > checkpoint.ranges.size() ||
a.ranges.front().begin != checkpoint.ranges.front().begin) {
TraceEvent(SevError, "ShardedRocksDBRestoreFailed", logId)
@ -2301,6 +2292,9 @@ struct ShardedRocksDBKeyValueStore : IKeyValueStore {
writeBatch.DeleteRange(ps->cf, toSlice(cRange.begin), toSlice(cRange.end));
}
}
} else if (format == RocksDBKeyValues) {
a.done.sendError(not_implemented());
return;
} else if (format == RocksDB) {
a.done.sendError(not_implemented());
return;
@ -3489,6 +3483,101 @@ TEST_CASE("noSim/ShardedRocksDB/Metadata") {
return Void();
}
TEST_CASE("noSim/ShardedRocksDB/CheckpointBasic") {
state std::string rocksDBTestDir = "sharded-rocks-checkpoint-restore";
state std::map<Key, Value> kvs({ { "a"_sr, "TestValueA"_sr },
{ "ab"_sr, "TestValueAB"_sr },
{ "ad"_sr, "TestValueAD"_sr },
{ "b"_sr, "TestValueB"_sr },
{ "ba"_sr, "TestValueBA"_sr },
{ "c"_sr, "TestValueC"_sr },
{ "d"_sr, "TestValueD"_sr },
{ "e"_sr, "TestValueE"_sr },
{ "h"_sr, "TestValueH"_sr },
{ "ha"_sr, "TestValueHA"_sr } });
platform::eraseDirectoryRecursive(rocksDBTestDir);
state IKeyValueStore* kvStore =
new ShardedRocksDBKeyValueStore(rocksDBTestDir, deterministicRandom()->randomUniqueID());
wait(kvStore->init());
// Add some ranges.
std::vector<Future<Void>> addRangeFutures;
addRangeFutures.push_back(kvStore->addRange(KeyRangeRef("a"_sr, "c"_sr), "shard-1"));
addRangeFutures.push_back(kvStore->addRange(KeyRangeRef("c"_sr, "f"_sr), "shard-2"));
addRangeFutures.push_back(kvStore->addRange(KeyRangeRef("h"_sr, "k"_sr), "shard-1"));
kvStore->persistRangeMapping(KeyRangeRef("a"_sr, "f"_sr), true);
wait(waitForAll(addRangeFutures) && kvStore->commit(false));
for (const auto& [k, v] : kvs) {
kvStore->set(KeyValueRef(k, v));
}
wait(kvStore->commit(false));
state std::string checkpointDir = "checkpoint";
platform::eraseDirectoryRecursive(checkpointDir);
// Checkpoint iterator returns only the desired keyrange, i.e., ["ab", "b"].
CheckpointRequest request(latestVersion,
{ KeyRangeRef("a"_sr, "c"_sr), KeyRangeRef("h"_sr, "k"_sr) },
DataMoveRocksCF,
deterministicRandom()->randomUniqueID(),
checkpointDir);
CheckpointMetaData metaData = wait(kvStore->checkpoint(request));
state Standalone<StringRef> token = BinaryWriter::toValue(KeyRangeRef("a"_sr, "k"_sr), IncludeVersion());
state ICheckpointReader* cpReader =
newCheckpointReader(metaData, CheckpointAsKeyValues::True, deterministicRandom()->randomUniqueID());
ASSERT(cpReader != nullptr);
wait(cpReader->init(token));
state KeyRange testRange(KeyRangeRef("ab"_sr, "b"_sr));
state std::unique_ptr<ICheckpointIterator> iter0 = cpReader->getIterator(testRange);
state int numKeys = 0;
try {
loop {
RangeResult res = wait(iter0->nextBatch(CLIENT_KNOBS->REPLY_BYTE_LIMIT, CLIENT_KNOBS->REPLY_BYTE_LIMIT));
for (const auto& kv : res) {
ASSERT(testRange.contains(kv.key));
ASSERT(kvs[kv.key] == kv.value);
++numKeys;
}
}
} catch (Error& e) {
ASSERT(e.code() == error_code_end_of_stream);
ASSERT(numKeys == 2);
}
testRange = KeyRangeRef("a"_sr, "k"_sr);
state std::unique_ptr<ICheckpointIterator> iter1 = cpReader->getIterator(testRange);
try {
numKeys = 0;
loop {
RangeResult res = wait(iter1->nextBatch(CLIENT_KNOBS->REPLY_BYTE_LIMIT, CLIENT_KNOBS->REPLY_BYTE_LIMIT));
for (const auto& kv : res) {
ASSERT(testRange.contains(kv.key));
ASSERT(kvs[kv.key] == kv.value);
++numKeys;
}
}
} catch (Error& e) {
ASSERT(e.code() == error_code_end_of_stream);
ASSERT(numKeys == 7);
}
iter0.reset();
iter1.reset();
ASSERT(!cpReader->inUse());
TraceEvent(SevDebug, "ShardedRocksCheckpointReaaderTested");
std::vector<Future<Void>> closes;
closes.push_back(cpReader->close());
closes.push_back(kvStore->onClosed());
kvStore->dispose();
wait(waitForAll(closes));
platform::eraseDirectoryRecursive(rocksDBTestDir);
platform::eraseDirectoryRecursive(checkpointDir);
return Void();
}
} // namespace
#endif // SSD_ROCKSDB_EXPERIMENTAL

View File

@ -208,7 +208,7 @@ void MockStorageServer::setShardStatus(const KeyRangeRef& range, MockShardStatus
twoWayShardSplitting(ranges.begin().range(), range.begin, ranges.begin().cvalue().shardSize, restrictSize);
}
if (ranges.end().begin() > range.end) {
CODE_PROBE(true, "Implicitly split end range to 2 pieces", probe::decoration::rare);
CODE_PROBE(true, "Implicitly split end range to 2 pieces");
auto lastRange = ranges.end();
--lastRange;
twoWayShardSplitting(lastRange.range(), range.end, ranges.end().cvalue().shardSize, restrictSize);
@ -228,7 +228,7 @@ void MockStorageServer::setShardStatus(const KeyRangeRef& range, MockShardStatus
it.value() = ShardInfo{ status, newSize };
} else if ((oldStatus == MockShardStatus::COMPLETED || oldStatus == MockShardStatus::FETCHED) &&
(status == MockShardStatus::INFLIGHT || status == MockShardStatus::FETCHED)) {
CODE_PROBE(true, "Shard already on server", probe::decoration::rare);
CODE_PROBE(true, "Shard already on server");
} else {
TraceEvent(SevError, "MockShardStatusTransitionError", id)
.detail("From", oldStatus)

View File

@ -1262,15 +1262,15 @@ ACTOR static Future<Void> startMoveShards(Database occ,
// TODO: make startMoveShards work with multiple ranges.
ASSERT(ranges.size() == 1);
state KeyRangeRef keys = ranges[0];
state bool cancelDataMove = false;
try {
state Key begin = keys.begin;
state KeyRange currentKeys = keys;
state int maxRetries = 0;
state bool complete = false;
loop {
state Key begin = keys.begin;
state KeyRange currentKeys = keys;
state int maxRetries = 0;
state bool complete = false;
state Transaction tr(occ);
complete = false;
try {
// Keep track of old dests that may need to have ranges removed from serverKeys
@ -1293,6 +1293,12 @@ ACTOR static Future<Void> startMoveShards(Database occ,
.detail("DataMoveID", dataMoveId)
.detail("DataMove", dataMove.toString());
ASSERT(!dataMove.ranges.empty() && dataMove.ranges.front().begin == keys.begin);
if (cancelDataMove) {
dataMove.setPhase(DataMoveMetaData::Deleting);
tr.set(dataMoveKeyFor(dataMoveId), dataMoveValue(dataMove));
wait(tr.commit());
throw data_move_cancelled();
}
if (dataMove.getPhase() == DataMoveMetaData::Deleting) {
TraceEvent(SevVerbose, "StartMoveShardsDataMove", relocationIntervalId)
.detail("DataMoveBeingDeleted", dataMoveId);
@ -1306,6 +1312,10 @@ ACTOR static Future<Void> startMoveShards(Database occ,
}
begin = dataMove.ranges.front().end;
} else {
if (cancelDataMove) {
throw data_move_cancelled();
}
dataMove = DataMoveMetaData();
dataMove.id = dataMoveId;
TraceEvent(SevVerbose, "StartMoveKeysNewDataMove", relocationIntervalId)
.detail("DataMoveRange", keys)
@ -1366,8 +1376,8 @@ ACTOR static Future<Void> startMoveShards(Database occ,
if (destId.isValid()) {
TraceEvent(SevWarn, "StartMoveShardsDestIDExist", relocationIntervalId)
.detail("Range", rangeIntersectKeys)
.detail("DataMoveID", dataMoveId)
.detail("DestID", destId)
.detail("DataMoveID", dataMoveId.toString())
.detail("DestID", destId.toString())
.log();
ASSERT(!dest.empty());
@ -1381,14 +1391,16 @@ ACTOR static Future<Void> startMoveShards(Database occ,
if (destId == anonymousShardId) {
wait(cleanUpSingleShardDataMove(
occ, rangeIntersectKeys, lock, startMoveKeysLock, dataMoveId, ddEnabledState));
throw retry();
} else {
if (cancelConflictingDataMoves) {
TraceEvent(
SevWarn, "StartMoveShardsCancelConflictingDataMove", relocationIntervalId)
.detail("Range", rangeIntersectKeys)
.detail("DataMoveID", dataMoveId)
.detail("ExistingDataMoveID", destId);
.detail("DataMoveID", dataMoveId.toString())
.detail("ExistingDataMoveID", destId.toString());
wait(cleanUpDataMove(occ, destId, lock, startMoveKeysLock, keys, ddEnabledState));
throw retry();
} else {
Optional<Value> val = wait(tr.get(dataMoveKeyFor(destId)));
ASSERT(val.present());
@ -1396,10 +1408,11 @@ ACTOR static Future<Void> startMoveShards(Database occ,
TraceEvent(
SevWarnAlways, "StartMoveShardsFoundConflictingDataMove", relocationIntervalId)
.detail("Range", rangeIntersectKeys)
.detail("DataMoveID", dataMoveId)
.detail("ExistingDataMoveID", destId)
.detail("DataMoveID", dataMoveId.toString())
.detail("ExistingDataMoveID", destId.toString())
.detail("ExistingDataMove", dmv.toString());
throw movekeys_conflict();
cancelDataMove = true;
throw retry();
}
}
}
@ -1485,16 +1498,16 @@ ACTOR static Future<Void> startMoveShards(Database occ,
break;
}
} catch (Error& e) {
TraceEvent(SevWarn, "StartMoveShardsError", dataMoveId)
.errorUnsuppressed(e)
.detail("DataMoveID", dataMoveId)
.detail("DataMoveRange", keys)
.detail("CurrentDataMoveMetaData", dataMove.toString());
state Error err = e;
if (err.code() == error_code_move_to_removed_server) {
throw;
if (e.code() == error_code_retry) {
wait(delay(1));
} else {
TraceEvent(SevWarn, "StartMoveShardsError", dataMoveId)
.errorUnsuppressed(e)
.detail("DataMoveID", dataMoveId)
.detail("DataMoveRange", keys)
.detail("CurrentDataMoveMetaData", dataMove.toString());
wait(tr.onError(e));
}
wait(tr.onError(e));
}
}
} catch (Error& e) {

View File

@ -23,6 +23,7 @@
#include <type_traits>
#include "fdbclient/FDBOptions.g.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/SystemData.h"
#include "flow/ActorCollection.h"
#include "fdbrpc/simulator.h"
@ -653,6 +654,63 @@ ACTOR Future<int64_t> getVersionOffset(Database cx,
}
}
// Returns DC lag for simulation runs
ACTOR Future<Version> getDatacenterLag(Database cx, Reference<AsyncVar<ServerDBInfo> const> dbInfo) {
loop {
if (!g_network->isSimulated() || g_simulator->usableRegions == 1) {
return 0;
}
state Optional<TLogInterface> primaryLog;
state Optional<TLogInterface> remoteLog;
if (dbInfo->get().recoveryState >= RecoveryState::ALL_LOGS_RECRUITED) {
for (const auto& logset : dbInfo->get().logSystemConfig.tLogs) {
if (logset.isLocal && logset.locality != tagLocalitySatellite) {
for (const auto& tlog : logset.tLogs) {
if (tlog.present()) {
primaryLog = tlog.interf();
break;
}
}
}
if (!logset.isLocal) {
for (const auto& tlog : logset.tLogs) {
if (tlog.present()) {
remoteLog = tlog.interf();
break;
}
}
}
}
}
if (!primaryLog.present() || !remoteLog.present()) {
wait(dbInfo->onChange());
continue;
}
ASSERT(primaryLog.present());
ASSERT(remoteLog.present());
state Future<Void> onChange = dbInfo->onChange();
loop {
state Future<TLogQueuingMetricsReply> primaryMetrics =
brokenPromiseToNever(primaryLog.get().getQueuingMetrics.getReply(TLogQueuingMetricsRequest()));
state Future<TLogQueuingMetricsReply> remoteMetrics =
brokenPromiseToNever(remoteLog.get().getQueuingMetrics.getReply(TLogQueuingMetricsRequest()));
wait((success(primaryMetrics) && success(remoteMetrics)) || onChange);
if (onChange.isReady()) {
break;
}
TraceEvent("DCLag").detail("Primary", primaryMetrics.get().v).detail("Remote", remoteMetrics.get().v);
ASSERT(primaryMetrics.get().v >= 0 && remoteMetrics.get().v >= 0);
return primaryMetrics.get().v - remoteMetrics.get().v;
}
}
}
ACTOR Future<Void> repairDeadDatacenter(Database cx,
Reference<AsyncVar<ServerDBInfo> const> dbInfo,
std::string context) {
@ -780,6 +838,8 @@ ACTOR Future<Void> waitForQuietDatabase(Database cx,
state Future<bool> dataDistributionActive;
state Future<bool> storageServersRecruiting;
state Future<int64_t> versionOffset;
state Future<Version> dcLag;
state Version maxDcLag = 30e6;
auto traceMessage = "QuietDatabase" + phase + "Begin";
TraceEvent(traceMessage.c_str()).log();
@ -817,10 +877,11 @@ ACTOR Future<Void> waitForQuietDatabase(Database cx,
dataDistributionActive = getDataDistributionActive(cx, distributorWorker);
storageServersRecruiting = getStorageServersRecruiting(cx, distributorWorker, distributorUID);
versionOffset = getVersionOffset(cx, distributorWorker, dbInfo);
dcLag = getDatacenterLag(cx, dbInfo);
wait(success(dataInFlight) && success(tLogQueueInfo) && success(dataDistributionQueueSize) &&
success(teamCollectionValid) && success(storageQueueSize) && success(dataDistributionActive) &&
success(storageServersRecruiting) && success(versionOffset));
success(storageServersRecruiting) && success(versionOffset) && success(dcLag));
maxVersionOffset += dbInfo->get().recoveryCount * SERVER_KNOBS->MAX_VERSIONS_IN_FLIGHT;
@ -836,7 +897,8 @@ ACTOR Future<Void> waitForQuietDatabase(Database cx,
.add(evt, "MaxStorageQueueSize", storageQueueSize.get(), maxStorageServerQueueGate)
.add(evt, "DataDistributionActive", dataDistributionActive.get(), true, std::equal_to<>())
.add(evt, "StorageServersRecruiting", storageServersRecruiting.get(), false, std::equal_to<>())
.add(evt, "VersionOffset", versionOffset.get(), maxVersionOffset);
.add(evt, "VersionOffset", versionOffset.get(), maxVersionOffset)
.add(evt, "DatacenterLag", dcLag.get(), maxDcLag);
evt.detail("RecoveryCount", dbInfo->get().recoveryCount).detail("NumSuccesses", numSuccesses);
evt.log();

View File

@ -42,6 +42,8 @@
#include "flow/actorcompiler.h" // has to be last include
FDB_DEFINE_BOOLEAN_PARAM(CheckpointAsKeyValues);
#ifdef SSD_ROCKSDB_EXPERIMENTAL
// Enforcing rocksdb version to be 7.7.3.
static_assert((ROCKSDB_MAJOR == 7 && ROCKSDB_MINOR == 7 && ROCKSDB_PATCH == 3),
@ -53,6 +55,47 @@ using DB = rocksdb::DB*;
using CF = rocksdb::ColumnFamilyHandle*;
const KeyRef persistVersion = "\xff\xffVersion"_sr;
const KeyRef readerInitialized = "\xff\xff/ReaderInitialized"_sr;
const std::string checkpointCf = "RocksDBCheckpoint";
const std::string checkpointReaderSubDir = "/reader";
const std::string rocksDefaultCf = "default";
rocksdb::ExportImportFilesMetaData getMetaData(const CheckpointMetaData& checkpoint) {
rocksdb::ExportImportFilesMetaData metaData;
if (checkpoint.getFormat() != DataMoveRocksCF) {
return metaData;
}
RocksDBColumnFamilyCheckpoint rocksCF = getRocksCF(checkpoint);
metaData.db_comparator_name = rocksCF.dbComparatorName;
for (const LiveFileMetaData& fileMetaData : rocksCF.sstFiles) {
rocksdb::LiveFileMetaData liveFileMetaData;
liveFileMetaData.size = fileMetaData.size;
liveFileMetaData.name = fileMetaData.name;
liveFileMetaData.file_number = fileMetaData.file_number;
liveFileMetaData.db_path = fileMetaData.db_path;
liveFileMetaData.smallest_seqno = fileMetaData.smallest_seqno;
liveFileMetaData.largest_seqno = fileMetaData.largest_seqno;
liveFileMetaData.smallestkey = fileMetaData.smallestkey;
liveFileMetaData.largestkey = fileMetaData.largestkey;
liveFileMetaData.num_reads_sampled = fileMetaData.num_reads_sampled;
liveFileMetaData.being_compacted = fileMetaData.being_compacted;
liveFileMetaData.num_entries = fileMetaData.num_entries;
liveFileMetaData.num_deletions = fileMetaData.num_deletions;
liveFileMetaData.temperature = static_cast<rocksdb::Temperature>(fileMetaData.temperature);
liveFileMetaData.oldest_blob_file_number = fileMetaData.oldest_blob_file_number;
liveFileMetaData.oldest_ancester_time = fileMetaData.oldest_ancester_time;
liveFileMetaData.file_creation_time = fileMetaData.file_creation_time;
liveFileMetaData.file_checksum = fileMetaData.file_checksum;
liveFileMetaData.file_checksum_func_name = fileMetaData.file_checksum_func_name;
liveFileMetaData.column_family_name = fileMetaData.column_family_name;
liveFileMetaData.level = fileMetaData.level;
metaData.files.push_back(liveFileMetaData);
}
return metaData;
}
rocksdb::Slice toSlice(StringRef s) {
return rocksdb::Slice(reinterpret_cast<const char*>(s.begin()), s.size());
@ -69,8 +112,8 @@ rocksdb::ColumnFamilyOptions getCFOptions() {
rocksdb::Options getOptions() {
rocksdb::Options options({}, getCFOptions());
options.create_if_missing = false;
options.db_log_dir = SERVER_KNOBS->LOG_DIRECTORY;
options.create_if_missing = true;
options.db_log_dir = g_network->isSimulated() ? "" : SERVER_KNOBS->LOG_DIRECTORY;
return options;
}
@ -103,27 +146,65 @@ Error statusToError(const rocksdb::Status& s) {
// RocksDBCheckpointReader reads a RocksDB checkpoint, and returns the key-value pairs via nextKeyValues.
class RocksDBCheckpointReader : public ICheckpointReader {
public:
class RocksDBCheckpointIterator : public ICheckpointIterator {
public:
RocksDBCheckpointIterator(RocksDBCheckpointReader* reader, const KeyRange& range)
: reader(reader), range(range) {
ASSERT(reader != nullptr);
ASSERT(reader->db != nullptr);
ASSERT(reader->cf != nullptr);
this->beginSlice = toSlice(this->range.begin);
this->endSlice = toSlice(this->range.end);
rocksdb::ReadOptions options = getReadOptions();
options.iterate_lower_bound = &beginSlice;
options.iterate_upper_bound = &endSlice;
options.fill_cache = false; // Optimized for bulk scan.
options.readahead_size = SERVER_KNOBS->ROCKSDB_CHECKPOINT_READ_AHEAD_SIZE;
const uint64_t deadlineMicros =
reader->db->GetEnv()->NowMicros() + SERVER_KNOBS->ROCKSDB_READ_CHECKPOINT_TIMEOUT * 1000000;
options.deadline = std::chrono::microseconds(deadlineMicros);
this->iterator = std::unique_ptr<rocksdb::Iterator>(reader->db->NewIterator(options, reader->cf));
iterator->Seek(this->beginSlice);
}
~RocksDBCheckpointIterator() { this->reader->numIter--; }
Future<RangeResult> nextBatch(const int rowLimit, const int ByteLimit) override;
rocksdb::Iterator* getIterator() { return iterator.get(); }
const rocksdb::Slice& end() const { return this->endSlice; }
private:
RocksDBCheckpointReader* const reader;
const KeyRange range;
rocksdb::Slice beginSlice;
rocksdb::Slice endSlice;
std::unique_ptr<rocksdb::Iterator> iterator;
};
RocksDBCheckpointReader(const CheckpointMetaData& checkpoint, UID logID);
Future<Void> init(StringRef token) override;
Future<RangeResult> nextKeyValues(const int rowLimit, const int byteLimit) override;
Future<RangeResult> nextKeyValues(const int rowLimit, const int byteLimit) override { throw not_implemented(); }
Future<Standalone<StringRef>> nextChunk(const int byteLimit) override { throw not_implemented(); }
Future<Void> close() override { return doClose(this); }
std::unique_ptr<ICheckpointIterator> getIterator(KeyRange range) override;
bool inUse() const override { return this->numIter > 0; }
private:
struct Reader : IThreadPoolReceiver {
struct OpenAction : TypedAction<Reader, OpenAction> {
OpenAction(std::string path, KeyRange range, Version version)
: path(std::move(path)), range(range), version(version) {}
OpenAction(CheckpointMetaData checkpoint) : checkpoint(std::move(checkpoint)) {}
double getTimeEstimate() const override { return SERVER_KNOBS->COMMIT_TIME_ESTIMATE; }
const std::string path;
const KeyRange range;
const Version version;
const CheckpointMetaData checkpoint;
ThreadReturnPromise<Void> done;
};
@ -137,17 +218,18 @@ private:
};
struct ReadRangeAction : TypedAction<Reader, ReadRangeAction>, FastAllocated<ReadRangeAction> {
ReadRangeAction(int rowLimit, int byteLimit)
: rowLimit(rowLimit), byteLimit(byteLimit), startTime(timer_monotonic()) {}
ReadRangeAction(int rowLimit, int byteLimit, RocksDBCheckpointIterator* iterator)
: rowLimit(rowLimit), byteLimit(byteLimit), iterator(iterator), startTime(timer_monotonic()) {}
double getTimeEstimate() const override { return SERVER_KNOBS->READ_RANGE_TIME_ESTIMATE; }
const int rowLimit, byteLimit;
RocksDBCheckpointIterator* const iterator;
const double startTime;
ThreadReturnPromise<RangeResult> result;
};
explicit Reader(DB& db);
explicit Reader(DB& db, CF& cf);
~Reader() override {}
void init() override {}
@ -158,35 +240,48 @@ private:
void action(ReadRangeAction& a);
rocksdb::Status tryOpenForRead(const std::string& path);
rocksdb::Status importCheckpoint(const std::string& path, const CheckpointMetaData& checkpoint);
rocksdb::Status closeInternal(const std::string& path, const bool deleteOnClose);
DB& db;
CF cf;
Key begin;
Key end;
CF& cf;
std::vector<rocksdb::ColumnFamilyHandle*> handles;
double readRangeTimeout;
std::unique_ptr<rocksdb::Iterator> cursor;
};
Future<RangeResult> nextBatch(const int rowLimit, const int byteLimit, RocksDBCheckpointIterator* iterator);
ACTOR static Future<Void> doClose(RocksDBCheckpointReader* self);
DB db = nullptr;
CF cf = nullptr;
std::string path;
const UID id;
Version version;
Reference<IThreadPool> readThreads;
CheckpointMetaData checkpoint;
Reference<IThreadPool> threads;
Future<Void> openFuture;
int numIter;
};
Future<RangeResult> RocksDBCheckpointReader::RocksDBCheckpointIterator::nextBatch(const int rowLimit,
const int ByteLimit) {
return this->reader->nextBatch(rowLimit, ByteLimit, this);
}
RocksDBCheckpointReader::RocksDBCheckpointReader(const CheckpointMetaData& checkpoint, UID logID)
: id(logID), version(checkpoint.version) {
RocksDBCheckpoint rocksCheckpoint = getRocksCheckpoint(checkpoint);
this->path = rocksCheckpoint.checkpointDir;
: id(logID), checkpoint(checkpoint), numIter(0) {
if (g_network->isSimulated()) {
readThreads = CoroThreadPool::createThreadPool();
threads = CoroThreadPool::createThreadPool();
} else {
readThreads = createGenericThreadPool();
threads = createGenericThreadPool();
}
for (int i = 0; i < SERVER_KNOBS->ROCKSDB_CHECKPOINT_READER_PARALLELISM; ++i) {
threads->addThread(new Reader(db, cf), "fdb-rocks-cr");
}
readThreads->addThread(new Reader(db), "fdb-rocks-rd");
}
Future<Void> RocksDBCheckpointReader::init(StringRef token) {
@ -194,143 +289,70 @@ Future<Void> RocksDBCheckpointReader::init(StringRef token) {
return openFuture;
}
KeyRange range = BinaryReader::fromStringRef<KeyRange>(token, IncludeVersion());
auto a = std::make_unique<Reader::OpenAction>(this->path, range, this->version);
auto a = std::make_unique<Reader::OpenAction>(this->checkpoint);
openFuture = a->done.getFuture();
readThreads->post(a.release());
threads->post(a.release());
return openFuture;
}
Future<RangeResult> RocksDBCheckpointReader::nextKeyValues(const int rowLimit, const int byteLimit) {
auto a = std::make_unique<Reader::ReadRangeAction>(rowLimit, byteLimit);
Future<RangeResult> RocksDBCheckpointReader::nextBatch(const int rowLimit,
const int byteLimit,
RocksDBCheckpointIterator* iterator) {
auto a = std::make_unique<Reader::ReadRangeAction>(rowLimit, byteLimit, iterator);
auto res = a->result.getFuture();
readThreads->post(a.release());
threads->post(a.release());
return res;
}
RocksDBCheckpointReader::Reader::Reader(DB& db) : db(db), cf(nullptr) {
if (g_network->isSimulated()) {
// In simulation, increasing the read operation timeouts to 5 minutes, as some of the tests have
// very high load and single read thread cannot process all the load within the timeouts.
readRangeTimeout = 5 * 60;
} else {
readRangeTimeout = SERVER_KNOBS->ROCKSDB_READ_RANGE_TIMEOUT;
}
std::unique_ptr<ICheckpointIterator> RocksDBCheckpointReader::getIterator(KeyRange range) {
++this->numIter;
return std::unique_ptr<ICheckpointIterator>(new RocksDBCheckpointIterator(this, range));
}
RocksDBCheckpointReader::Reader::Reader(DB& db, CF& cf) : db(db), cf(cf) {}
void RocksDBCheckpointReader::Reader::action(RocksDBCheckpointReader::Reader::OpenAction& a) {
TraceEvent(SevDebug, "RocksDBCheckpointReaderInitBegin").detail("Checkpoint", a.checkpoint.toString());
ASSERT(cf == nullptr);
std::vector<std::string> columnFamilies;
rocksdb::Options options = getOptions();
rocksdb::Status status = rocksdb::DB::ListColumnFamilies(options, a.path, &columnFamilies);
if (std::find(columnFamilies.begin(), columnFamilies.end(), "default") == columnFamilies.end()) {
columnFamilies.push_back("default");
}
rocksdb::ColumnFamilyOptions cfOptions = getCFOptions();
std::vector<rocksdb::ColumnFamilyDescriptor> descriptors;
for (const std::string& name : columnFamilies) {
descriptors.push_back(rocksdb::ColumnFamilyDescriptor{ name, cfOptions });
}
status = rocksdb::DB::OpenForReadOnly(options, a.path, descriptors, &handles, &db);
if (!status.ok()) {
logRocksDBError(status, "OpenForReadOnly");
a.done.sendError(statusToError(status));
const CheckpointMetaData& checkpoint = a.checkpoint;
const CheckpointFormat format = checkpoint.getFormat();
if (format != DataMoveRocksCF) {
TraceEvent(SevDebug, "RocksDBCheckpointReaderError").detail("InvalidFormat", checkpoint.toString());
a.done.sendError(not_implemented());
return;
}
for (rocksdb::ColumnFamilyHandle* handle : handles) {
if (handle->GetName() == SERVER_KNOBS->DEFAULT_FDB_ROCKSDB_COLUMN_FAMILY) {
cf = handle;
break;
RocksDBColumnFamilyCheckpoint rocksCF = getRocksCF(checkpoint);
ASSERT(!rocksCF.sstFiles.empty());
const std::string path = rocksCF.sstFiles.front().db_path + checkpointReaderSubDir;
rocksdb::Status status = tryOpenForRead(path);
if (!status.ok()) {
platform::eraseDirectoryRecursive(path);
status = importCheckpoint(path, checkpoint);
if (status.ok()) {
status = tryOpenForRead(path);
}
}
ASSERT(db != nullptr && cf != nullptr);
begin = a.range.begin;
end = a.range.end;
TraceEvent(SevInfo, "RocksDBCheckpointReaderInit")
.detail("Path", a.path)
.detail("Method", "OpenForReadOnly")
.detail("ColumnFamily", cf->GetName())
.detail("Begin", begin)
.detail("End", end);
rocksdb::PinnableSlice value;
rocksdb::ReadOptions readOptions = getReadOptions();
status = db->Get(readOptions, cf, toSlice(persistVersion), &value);
if (!status.ok() && !status.IsNotFound()) {
logRocksDBError(status, "Checkpoint");
if (!status.ok()) {
a.done.sendError(statusToError(status));
return;
}
const Version version =
status.IsNotFound() ? latestVersion : BinaryReader::fromStringRef<Version>(toStringRef(value), Unversioned());
ASSERT(version == a.version);
cursor = std::unique_ptr<rocksdb::Iterator>(db->NewIterator(readOptions, cf));
cursor->Seek(toSlice(begin));
a.done.send(Void());
TraceEvent(SevDebug, "RocksDBCheckpointReaderInitEnd").detail("Path", path).detail("ColumnFamily", cf->GetName());
}
void RocksDBCheckpointReader::Reader::action(RocksDBCheckpointReader::Reader::CloseAction& a) {
if (db == nullptr) {
a.done.send(Void());
return;
}
for (rocksdb::ColumnFamilyHandle* handle : handles) {
if (handle != nullptr) {
TraceEvent("RocksDBCheckpointReaderDestroyCF").detail("Path", a.path).detail("CF", handle->GetName());
db->DestroyColumnFamilyHandle(handle);
}
}
handles.clear();
rocksdb::Status s = db->Close();
if (!s.ok()) {
logRocksDBError(s, "Close");
}
if (a.deleteOnClose) {
std::set<std::string> columnFamilies{ "default" };
columnFamilies.insert(SERVER_KNOBS->DEFAULT_FDB_ROCKSDB_COLUMN_FAMILY);
std::vector<rocksdb::ColumnFamilyDescriptor> descriptors;
for (const std::string& name : columnFamilies) {
descriptors.push_back(rocksdb::ColumnFamilyDescriptor{ name, getCFOptions() });
}
s = rocksdb::DestroyDB(a.path, getOptions(), descriptors);
if (!s.ok()) {
logRocksDBError(s, "Destroy");
} else {
TraceEvent("RocksDBCheckpointReader").detail("Path", a.path).detail("Method", "Destroy");
}
}
TraceEvent("RocksDBCheckpointReader").detail("Path", a.path).detail("Method", "Close");
closeInternal(a.path, a.deleteOnClose);
a.done.send(Void());
}
void RocksDBCheckpointReader::Reader::action(RocksDBCheckpointReader::Reader::ReadRangeAction& a) {
const double readBeginTime = timer_monotonic();
if (readBeginTime - a.startTime > readRangeTimeout) {
TraceEvent(SevWarn, "RocksDBCheckpointReaderError")
.detail("Error", "Read range request timedout")
.detail("Method", "ReadRangeAction")
.detail("Timeout value", readRangeTimeout);
a.result.sendError(timed_out());
return;
}
TraceEvent(SevDebug, "RocksDBCheckpointReaderReadRangeBegin");
ASSERT(a.iterator != nullptr);
RangeResult result;
if (a.rowLimit == 0 || a.byteLimit == 0) {
@ -341,55 +363,176 @@ void RocksDBCheckpointReader::Reader::action(RocksDBCheckpointReader::Reader::Re
// For now, only forward scan is supported.
ASSERT(a.rowLimit > 0);
rocksdb::Iterator* iter = a.iterator->getIterator();
int accumulatedBytes = 0;
rocksdb::Status s;
while (cursor->Valid() && toStringRef(cursor->key()) < end) {
KeyValueRef kv(toStringRef(cursor->key()), toStringRef(cursor->value()));
while (iter->Valid() && iter->key().compare(a.iterator->end()) < 0) {
KeyValueRef kv(toStringRef(iter->key()), toStringRef(iter->value()));
accumulatedBytes += sizeof(KeyValueRef) + kv.expectedSize();
result.push_back_deep(result.arena(), kv);
cursor->Next();
iter->Next();
if (result.size() >= a.rowLimit || accumulatedBytes >= a.byteLimit) {
break;
}
if (timer_monotonic() - a.startTime > readRangeTimeout) {
TraceEvent(SevWarn, "RocksDBCheckpointReaderError")
.detail("Error", "Read range request timedout")
.detail("Method", "ReadRangeAction")
.detail("Timeout value", readRangeTimeout);
a.result.sendError(transaction_too_old());
delete (cursor.release());
return;
}
}
s = cursor->status();
s = iter->status();
if (!s.ok()) {
logRocksDBError(s, "ReadRange");
a.result.sendError(statusToError(s));
delete (cursor.release());
return;
}
if (result.empty()) {
delete (cursor.release());
a.result.sendError(end_of_stream());
} else {
a.result.send(result);
}
}
rocksdb::Status RocksDBCheckpointReader::Reader::tryOpenForRead(const std::string& path) {
std::vector<std::string> columnFamilies;
const rocksdb::Options options = getOptions();
rocksdb::Status status = rocksdb::DB::ListColumnFamilies(options, path, &columnFamilies);
if (std::find(columnFamilies.begin(), columnFamilies.end(), rocksDefaultCf) == columnFamilies.end() ||
std::find(columnFamilies.begin(), columnFamilies.end(), checkpointCf) == columnFamilies.end()) {
return rocksdb::Status::Aborted();
}
const rocksdb::ColumnFamilyOptions cfOptions = getCFOptions();
std::vector<rocksdb::ColumnFamilyDescriptor> descriptors;
for (const std::string& name : columnFamilies) {
descriptors.emplace_back(name, cfOptions);
}
status = rocksdb::DB::OpenForReadOnly(options, path, descriptors, &handles, &db);
if (!status.ok()) {
logRocksDBError(status, "OpenForReadOnly");
return status;
}
rocksdb::PinnableSlice value;
rocksdb::ReadOptions readOptions = getReadOptions();
status = db->Get(readOptions, db->DefaultColumnFamily(), toSlice(readerInitialized), &value);
if (!status.ok() && !status.IsNotFound()) {
logRocksDBError(status, "CheckpointCheckInitState");
return status;
}
if (status.IsNotFound()) {
status = closeInternal(path, /*deleteOnClose=*/true);
if (!status.ok()) {
return status;
} else {
delete db;
TraceEvent(SevDebug, "RocksDBCheckpointReaderTryOpenError").detail("Path", path);
return rocksdb::Status::Aborted();
}
}
ASSERT(handles.size() == 2);
for (rocksdb::ColumnFamilyHandle* handle : handles) {
if (handle->GetName() == checkpointCf) {
TraceEvent(SevDebug, "RocksDBCheckpointCF").detail("Path", path).detail("ColumnFamily", handle->GetName());
cf = handle;
break;
}
}
ASSERT(db != nullptr && cf != nullptr);
return rocksdb::Status::OK();
}
rocksdb::Status RocksDBCheckpointReader::Reader::importCheckpoint(const std::string& path,
const CheckpointMetaData& checkpoint) {
std::vector<std::string> columnFamilies;
const rocksdb::Options options = getOptions();
rocksdb::Status status = rocksdb::DB::ListColumnFamilies(options, path, &columnFamilies);
if (std::find(columnFamilies.begin(), columnFamilies.end(), rocksDefaultCf) == columnFamilies.end()) {
columnFamilies.push_back(rocksDefaultCf);
}
const rocksdb::ColumnFamilyOptions cfOptions = getCFOptions();
std::vector<rocksdb::ColumnFamilyDescriptor> descriptors;
for (const std::string& name : columnFamilies) {
descriptors.emplace_back(name, cfOptions);
}
status = rocksdb::DB::Open(options, path, descriptors, &handles, &db);
if (!status.ok()) {
logRocksDBError(status, "CheckpointReaderOpen");
return status;
}
rocksdb::ExportImportFilesMetaData metaData = getMetaData(checkpoint);
rocksdb::ImportColumnFamilyOptions importOptions;
importOptions.move_files = false;
status = db->CreateColumnFamilyWithImport(cfOptions, checkpointCf, importOptions, metaData, &cf);
if (!status.ok()) {
logRocksDBError(status, "CheckpointReaderImportCheckpoint");
return status;
}
handles.push_back(cf);
TraceEvent(SevDebug, "RocksDBCheckpointReaderImportedCF");
rocksdb::WriteOptions writeOptions;
writeOptions.sync = !SERVER_KNOBS->ROCKSDB_UNSAFE_AUTO_FSYNC;
status = db->Put(writeOptions, toSlice(readerInitialized), toSlice("1"_sr));
if (!status.ok()) {
logRocksDBError(status, "CheckpointReaderPersistInitKey");
return status;
}
ASSERT(db != nullptr && cf != nullptr);
return closeInternal(path, /*deleteOnClose=*/false);
}
rocksdb::Status RocksDBCheckpointReader::Reader::closeInternal(const std::string& path, const bool deleteOnClose) {
if (db == nullptr) {
return rocksdb::Status::OK();
}
for (rocksdb::ColumnFamilyHandle* handle : handles) {
if (handle != nullptr) {
TraceEvent("RocksDBCheckpointReaderDestroyCF").detail("Path", path).detail("CF", handle->GetName());
db->DestroyColumnFamilyHandle(handle);
}
}
handles.clear();
rocksdb::Status s = db->Close();
if (!s.ok()) {
logRocksDBError(s, "Close");
}
if (deleteOnClose) {
rocksdb::ColumnFamilyOptions options;
std::vector<rocksdb::ColumnFamilyDescriptor> descriptors;
descriptors.emplace_back(rocksDefaultCf, options);
descriptors.emplace_back(checkpointCf, options);
s = rocksdb::DestroyDB(path, getOptions(), descriptors);
if (!s.ok()) {
logRocksDBError(s, "Destroy");
} else {
TraceEvent(SevDebug, "RocksDBCheckpointReader").detail("Path", path).detail("Method", "Destroy");
}
}
TraceEvent(SevDebug, "RocksDBCheckpointReader").detail("Path", path).detail("Method", "Close");
return s;
}
ACTOR Future<Void> RocksDBCheckpointReader::doClose(RocksDBCheckpointReader* self) {
if (self == nullptr)
return Void();
auto a = new RocksDBCheckpointReader::Reader::CloseAction(self->path, false);
auto f = a->done.getFuture();
self->readThreads->post(a);
self->threads->post(a);
wait(f);
if (self != nullptr) {
wait(self->readThreads->stop());
wait(self->threads->stop());
}
if (self != nullptr) {
@ -602,13 +745,15 @@ ACTOR Future<Void> fetchCheckpointRange(Database cx,
std::shared_ptr<rocksdb::SstFileWriter> writer,
std::function<Future<Void>(const CheckpointMetaData&)> cFun,
int maxRetries = 3) {
state std::string localFile = dir + "/" + metaData->checkpointID.toString() + ".sst";
RocksDBCheckpoint rcp = getRocksCheckpoint(*metaData);
state std::string localFile =
dir + "/" + UID(metaData->checkpointID.first(), deterministicRandom()->randomUInt64()).toString() + ".sst";
RocksDBCheckpointKeyValues rkv = getRocksKeyValuesCheckpoint(*metaData);
TraceEvent("FetchCheckpointRange")
.detail("InitialState", metaData->toString())
.detail("RocksCheckpoint", rcp.toString());
.detail("RocksCheckpointKeyValues", rkv.toString())
.detail("FilePath", localFile);
for (const auto& file : rcp.fetchedFiles) {
for (const auto& file : rkv.fetchedFiles) {
ASSERT(!file.range.intersects(range));
}
@ -745,6 +890,40 @@ ACTOR Future<Void> fetchCheckpointRange(Database cx,
return Void();
}
ACTOR Future<Void> fetchCheckpointRanges(Database cx,
std::shared_ptr<CheckpointMetaData> metaData,
std::string dir,
std::function<Future<Void>(const CheckpointMetaData&)> cFun) {
RocksDBCheckpointKeyValues rkv = getRocksKeyValuesCheckpoint(*metaData);
TraceEvent("FetchCheckpointRanges")
.detail("InitialState", metaData->toString())
.detail("RocksCheckpointKeyValues", rkv.toString());
KeyRangeMap<CheckpointFile> fileMap;
for (const auto& file : rkv.fetchedFiles) {
fileMap.insert(file.range, file);
}
std::vector<Future<Void>> fs;
for (const auto& range : rkv.ranges) {
auto ranges = fileMap.intersectingRanges(range);
for (auto r = ranges.begin(); r != ranges.end(); ++r) {
CheckpointFile& file = r->value();
KeyRangeRef currentRange = range & r->range();
if (!file.isValid()) {
std::shared_ptr<rocksdb::SstFileWriter> writer =
std::make_shared<rocksdb::SstFileWriter>(rocksdb::EnvOptions(), rocksdb::Options());
fs.push_back(fetchCheckpointRange(cx, metaData, currentRange, dir, writer, cFun));
}
}
}
wait(waitForAll(fs));
if (cFun) {
wait(cFun(*metaData));
}
return Void();
}
} // namespace
ACTOR Future<CheckpointMetaData> fetchRocksDBCheckpoint(Database cx,
@ -759,7 +938,7 @@ ACTOR Future<CheckpointMetaData> fetchRocksDBCheckpoint(Database cx,
state std::shared_ptr<CheckpointMetaData> metaData = std::make_shared<CheckpointMetaData>(initialState);
if (metaData->format == DataMoveRocksCF) {
if (metaData->getFormat() == DataMoveRocksCF) {
state RocksDBColumnFamilyCheckpoint rocksCF = getRocksCF(initialState);
TraceEvent(SevDebug, "RocksDBCheckpointMetaData").detail("RocksCF", rocksCF.toString());
@ -772,10 +951,10 @@ ACTOR Future<CheckpointMetaData> fetchRocksDBCheckpoint(Database cx,
.detail("Server", describe(metaData->src));
}
wait(waitForAll(fs));
} else if (metaData->format == RocksDB) {
std::shared_ptr<rocksdb::SstFileWriter> writer =
std::make_shared<rocksdb::SstFileWriter>(rocksdb::EnvOptions(), rocksdb::Options());
wait(fetchCheckpointRange(cx, metaData, metaData->ranges.front(), dir, writer, cFun));
} else if (metaData->getFormat() == RocksDBKeyValues) {
wait(fetchCheckpointRanges(cx, metaData, dir, cFun));
} else if (metaData->getFormat() == RocksDB) {
throw not_implemented();
}
return *metaData;
@ -846,12 +1025,14 @@ int64_t getTotalFetchedBytes(const std::vector<CheckpointMetaData>& checkpoints)
return totalBytes;
}
ICheckpointReader* newRocksDBCheckpointReader(const CheckpointMetaData& checkpoint, UID logID) {
ICheckpointReader* newRocksDBCheckpointReader(const CheckpointMetaData& checkpoint,
const CheckpointAsKeyValues checkpointAsKeyValues,
UID logID) {
#ifdef SSD_ROCKSDB_EXPERIMENTAL
const CheckpointFormat format = checkpoint.getFormat();
if (format == DataMoveRocksCF) {
if (format == DataMoveRocksCF && !checkpointAsKeyValues) {
return new RocksDBCFCheckpointReader(checkpoint, logID);
} else if (format == RocksDB) {
} else {
return new RocksDBCheckpointReader(checkpoint, logID);
}
#endif // SSD_ROCKSDB_EXPERIMENTAL
@ -870,4 +1051,11 @@ RocksDBCheckpoint getRocksCheckpoint(const CheckpointMetaData& checkpoint) {
ObjectReader reader(checkpoint.serializedCheckpoint.begin(), IncludeVersion());
reader.deserialize(rocksCheckpoint);
return rocksCheckpoint;
}
RocksDBCheckpointKeyValues getRocksKeyValuesCheckpoint(const CheckpointMetaData& checkpoint) {
RocksDBCheckpointKeyValues rocksCheckpoint;
ObjectReader reader(checkpoint.serializedCheckpoint.begin(), IncludeVersion());
reader.deserialize(rocksCheckpoint);
return rocksCheckpoint;
}

View File

@ -23,10 +23,12 @@
#include "flow/actorcompiler.h" // has to be last include
ICheckpointReader* newCheckpointReader(const CheckpointMetaData& checkpoint, UID logID) {
ICheckpointReader* newCheckpointReader(const CheckpointMetaData& checkpoint,
const CheckpointAsKeyValues checkpointAsKeyValues,
UID logID) {
const CheckpointFormat format = checkpoint.getFormat();
if (format == DataMoveRocksCF || format == RocksDB) {
return newRocksDBCheckpointReader(checkpoint, logID);
return newRocksDBCheckpointReader(checkpoint, checkpointAsKeyValues, logID);
} else {
throw not_implemented();
}
@ -51,11 +53,12 @@ ACTOR Future<CheckpointMetaData> fetchCheckpoint(Database cx,
std::string dir,
std::function<Future<Void>(const CheckpointMetaData&)> cFun) {
TraceEvent("FetchCheckpointBegin", initialState.checkpointID).detail("CheckpointMetaData", initialState.toString());
state CheckpointMetaData result;
const CheckpointFormat format = initialState.getFormat();
ASSERT(format != RocksDBKeyValues);
if (format == DataMoveRocksCF || format == RocksDB) {
CheckpointMetaData _result = wait(fetchRocksDBCheckpoint(cx, initialState, dir, cFun));
result = _result;
wait(store(result, fetchRocksDBCheckpoint(cx, initialState, dir, cFun)));
} else {
throw not_implemented();
}
@ -64,15 +67,30 @@ ACTOR Future<CheckpointMetaData> fetchCheckpoint(Database cx,
return result;
}
ACTOR Future<std::vector<CheckpointMetaData>> fetchCheckpoints(
Database cx,
std::vector<CheckpointMetaData> initialStates,
std::string dir,
std::function<Future<Void>(const CheckpointMetaData&)> cFun) {
std::vector<Future<CheckpointMetaData>> actors;
for (const auto& checkpoint : initialStates) {
actors.push_back(fetchCheckpoint(cx, checkpoint, dir, cFun));
ACTOR Future<CheckpointMetaData> fetchCheckpointRanges(Database cx,
CheckpointMetaData initialState,
std::string dir,
std::vector<KeyRange> ranges,
std::function<Future<Void>(const CheckpointMetaData&)> cFun) {
TraceEvent(SevDebug, "FetchCheckpointRangesBegin", initialState.checkpointID)
.detail("CheckpointMetaData", initialState.toString())
.detail("Ranges", describe(ranges));
ASSERT(!ranges.empty());
state CheckpointMetaData result;
const CheckpointFormat format = initialState.getFormat();
if (format != RocksDBKeyValues) {
if (format != DataMoveRocksCF) {
throw not_implemented();
}
initialState.setFormat(RocksDBKeyValues);
initialState.serializedCheckpoint = ObjectWriter::toValue(RocksDBCheckpointKeyValues(ranges), IncludeVersion());
}
std::vector<CheckpointMetaData> res = wait(getAll(actors));
return res;
wait(store(result, fetchRocksDBCheckpoint(cx, initialState, dir, cFun)));
TraceEvent(SevDebug, "FetchCheckpointRangesEnd", initialState.checkpointID)
.detail("CheckpointMetaData", result.toString())
.detail("Ranges", describe(ranges));
return result;
}

View File

@ -2466,22 +2466,28 @@ ACTOR static Future<JsonBuilderObject> blobRestoreStatusFetcher(Database db, std
case BlobRestorePhase::INIT:
statusObj["blob_full_restore_phase"] = "Initializing";
break;
case BlobRestorePhase::LOAD_MANIFEST:
case BlobRestorePhase::STARTING_MIGRATOR:
statusObj["blob_full_restore_phase"] = "Starting migrator";
break;
case BlobRestorePhase::LOADING_MANIFEST:
statusObj["blob_full_restore_phase"] = "Loading manifest";
break;
case BlobRestorePhase::MANIFEST_DONE:
statusObj["blob_full_restore_phase"] = "Manifest loaded";
case BlobRestorePhase::LOADED_MANIFEST:
statusObj["blob_full_restore_phase"] = "Manifest is loaded";
break;
case BlobRestorePhase::MIGRATE:
case BlobRestorePhase::COPYING_DATA:
statusObj["blob_full_restore_phase"] = "Copying data";
statusObj["blob_full_restore_progress"] = status.get().progress;
statusObj["blob_full_restore_progress"] = status.get().status;
break;
case BlobRestorePhase::APPLY_MLOGS:
case BlobRestorePhase::APPLYING_MLOGS:
statusObj["blob_full_restore_phase"] = "Applying mutation logs";
statusObj["blob_full_restore_progress"] = status.get().progress;
statusObj["blob_full_restore_progress"] = status.get().status;
break;
case BlobRestorePhase::DONE:
statusObj["blob_full_restore_phase"] = "Completed";
statusObj["blob_full_restore_phase"] = "Completed successfully";
break;
case BlobRestorePhase::ERROR:
statusObj["blob_full_restore_phase"] = "Completed with fatal error";
break;
default:
statusObj["blob_full_restore_phase"] = "Unexpected phase";
@ -3124,6 +3130,8 @@ ACTOR Future<StatusReply> clusterGetStatus(
state JsonBuilderObject recoveryStateStatus = wait(
recoveryStateStatusFetcher(cx, ccWorker, mWorker, workers.size(), &status_incomplete_reasons, &statusCode));
state JsonBuilderObject idmpKeyStatus = wait(getIdmpKeyStatus(cx));
// machine metrics
state WorkerEvents mMetrics = workerEventsVec[0].present() ? workerEventsVec[0].get().first : WorkerEvents();
// process metrics
@ -3505,6 +3513,8 @@ ACTOR Future<StatusReply> clusterGetStatus(
if (!recoveryStateStatus.empty())
statusObj["recovery_state"] = recoveryStateStatus;
statusObj["idempotency_ids"] = idmpKeyStatus;
// cluster messages subsection;
JsonBuilderArray clientIssuesArr = getClientIssuesAsMessages(clientStatus);
if (clientIssuesArr.size() > 0) {

View File

@ -1622,6 +1622,7 @@ void peekMessagesFromMemory(Reference<LogData> self,
Version& endVersion) {
ASSERT(!messages.getLength());
int versionCount = 0;
auto& deque = getVersionMessages(self, tag);
//TraceEvent("TLogPeekMem", self->dbgid).detail("Tag", req.tag1).detail("PDS", self->persistentDataSequence).detail("PDDS", self->persistentDataDurableSequence).detail("Oldest", map1.empty() ? 0 : map1.begin()->key ).detail("OldestMsgCount", map1.empty() ? 0 : map1.begin()->value.size());
@ -1652,6 +1653,23 @@ void peekMessagesFromMemory(Reference<LogData> self,
DEBUG_TAGS_AND_MESSAGE(
"TLogPeek", currentVersion, StringRef((uint8_t*)data + offset, messages.getLength() - offset), self->logId)
.detail("PeekTag", tag);
versionCount++;
}
if (versionCount == 0) {
++self->emptyPeeks;
} else {
++self->nonEmptyPeeks;
// TODO (version vector) check if this should be included in "status details" json
if (self->peekVersionCounts.find(tag) == self->peekVersionCounts.end()) {
UID ssID = deterministicRandom()->randomUniqueID();
std::string s = "PeekVersionCounts " + tag.toString();
self->peekVersionCounts.try_emplace(
tag, s, ssID, SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL, SERVER_KNOBS->LATENCY_SKETCH_ACCURACY);
}
LatencySample& sample = self->peekVersionCounts.at(tag);
sample.addMeasurement(versionCount);
}
}

View File

@ -92,6 +92,8 @@ static FILE* g_debugStream = stdout;
#define TRACE \
debug_printf_always("%s: %s line %d %s\n", __FUNCTION__, __FILE__, __LINE__, platform::get_backtrace().c_str());
static bool g_allowXOREncryptionInSimulation = true;
using namespace std::string_view_literals;
// Returns a string where every line in lines is prefixed with prefix
@ -2588,10 +2590,11 @@ public:
Standalone<VectorRef<PhysicalPageID>> pageIDs,
Reference<ArenaPage> page,
bool header = false) {
debug_printf("DWALPager(%s) op=%s %s ptr=%p\n",
debug_printf("DWALPager(%s) op=%s %s %d ptr=%p\n",
filename.c_str(),
(header ? "writePhysicalHeader" : "writePhysicalPage"),
toString(pageIDs).c_str(),
page->getEncodingType(),
page->rawData());
// Set metadata before prewrite so it's in the pre-encrypted page in cache if the page is encrypted
@ -2657,16 +2660,17 @@ public:
} else if (cacheEntry.reading()) {
// This is very unlikely, maybe impossible in the current pager use cases
// Wait for the outstanding read to finish, then start the write
cacheEntry.writeFuture = mapAsync<Void, std::function<Future<Void>(Void)>, Void>(
success(cacheEntry.readFuture), [=](Void) { return writePhysicalPage(reason, level, pageIDs, data); });
cacheEntry.writeFuture = mapAsync(success(cacheEntry.readFuture),
[=](Void) { return writePhysicalPage(reason, level, pageIDs, data); });
}
// If the page is being written, wait for this write before issuing the new write to ensure the
// writes happen in the correct order
else if (cacheEntry.writing()) {
// This is very unlikely, maybe impossible in the current pager use cases
// Wait for the previous write to finish, then start new write
cacheEntry.writeFuture = mapAsync<Void, std::function<Future<Void>(Void)>, Void>(
cacheEntry.writeFuture, [=](Void) { return writePhysicalPage(reason, level, pageIDs, data); });
cacheEntry.writeFuture =
mapAsync(cacheEntry.writeFuture, [=](Void) { return writePhysicalPage(reason, level, pageIDs, data); });
} else {
cacheEntry.writeFuture = detach(writePhysicalPage(reason, level, pageIDs, data));
}
@ -7977,10 +7981,11 @@ public:
// TODO(yiwu): When the cluster encryption config is available later, fail if the cluster is configured to
// enable encryption, but the Redwood instance is unencrypted.
if (encryptionKeyProvider && encryptionKeyProvider->enableEncryption()) {
ASSERT(encryptionKeyProvider->expectedEncodingType() == EncodingType::AESEncryptionV1);
encodingType = EncodingType::AESEncryptionV1;
encodingType = FLOW_KNOBS->ENCRYPT_HEADER_AUTH_TOKEN_ENABLED ? EncodingType::AESEncryptionWithAuth
: EncodingType::AESEncryption;
ASSERT_EQ(encodingType, encryptionKeyProvider->expectedEncodingType());
m_keyProvider = encryptionKeyProvider;
} else if (g_network->isSimulated() && logID.hash() % 2 == 0) {
} else if (g_allowXOREncryptionInSimulation && g_network->isSimulated() && logID.hash() % 2 == 0) {
// Simulation only. Deterministically enable encryption based on uid
encodingType = EncodingType::XOREncryption_TestOnly;
m_keyProvider = makeReference<XOREncryptionKeyProvider_TestOnly>(filename);
@ -10043,7 +10048,8 @@ TEST_CASE("Lredwood/correctness/btree") {
params.getInt("encodingType").orDefault(deterministicRandom()->randomInt(0, EncodingType::MAX_ENCODING_TYPE));
state unsigned int encryptionDomainMode =
params.getInt("domainMode")
.orDefault(deterministicRandom()->randomInt(0, RandomEncryptionKeyProvider::EncryptionDomainMode::MAX));
.orDefault(deterministicRandom()->randomInt(
0, RandomEncryptionKeyProvider<AESEncryption>::EncryptionDomainMode::MAX));
state int pageSize =
shortTest ? 250 : (deterministicRandom()->coinflip() ? 4096 : deterministicRandom()->randomInt(250, 400));
state int extentSize =
@ -10096,9 +10102,12 @@ TEST_CASE("Lredwood/correctness/btree") {
state EncodingType encodingType = static_cast<EncodingType>(encoding);
state Reference<IPageEncryptionKeyProvider> keyProvider;
if (encodingType == EncodingType::AESEncryptionV1) {
keyProvider = makeReference<RandomEncryptionKeyProvider>(
RandomEncryptionKeyProvider::EncryptionDomainMode(encryptionDomainMode));
if (encodingType == EncodingType::AESEncryption) {
keyProvider = makeReference<RandomEncryptionKeyProvider<AESEncryption>>(
RandomEncryptionKeyProvider<AESEncryption>::EncryptionDomainMode(encryptionDomainMode));
} else if (encodingType == EncodingType::AESEncryptionWithAuth) {
keyProvider = makeReference<RandomEncryptionKeyProvider<AESEncryptionWithAuth>>(
RandomEncryptionKeyProvider<AESEncryptionWithAuth>::EncryptionDomainMode(encryptionDomainMode));
} else if (encodingType == EncodingType::XOREncryption_TestOnly) {
keyProvider = makeReference<XOREncryptionKeyProvider_TestOnly>(file);
}
@ -11245,9 +11254,13 @@ ACTOR Future<Void> sequentialInsert(IKeyValueStore* kvs, int prefixLen, int valu
return Void();
}
Future<Void> closeKVS(IKeyValueStore* kvs) {
Future<Void> closeKVS(IKeyValueStore* kvs, bool dispose = false) {
Future<Void> closed = kvs->onClosed();
kvs->close();
if (dispose) {
kvs->dispose();
} else {
kvs->close();
}
return closed;
}
@ -11424,3 +11437,69 @@ TEST_CASE(":/redwood/performance/histograms") {
return Void();
}
namespace {
void setAuthMode(EncodingType encodingType) {
auto& g_knobs = IKnobCollection::getMutableGlobalKnobCollection();
if (encodingType == AESEncryption) {
g_knobs.setKnob("encrypt_header_auth_token_enabled", KnobValueRef::create(bool{ false }));
} else {
g_knobs.setKnob("encrypt_header_auth_token_enabled", KnobValueRef::create(bool{ true }));
g_knobs.setKnob("encrypt_header_auth_token_algo", KnobValueRef::create(int{ 1 }));
}
}
} // anonymous namespace
TEST_CASE("/redwood/correctness/EnforceEncodingType") {
state const std::vector<std::pair<EncodingType, EncodingType>> testCases = {
{ XXHash64, AESEncryption }, { AESEncryption, AESEncryptionWithAuth }
};
state const std::map<EncodingType, Reference<IPageEncryptionKeyProvider>> encryptionKeyProviders = {
{ XXHash64, makeReference<NullKeyProvider>() },
{ AESEncryption, makeReference<RandomEncryptionKeyProvider<AESEncryption>>() },
{ AESEncryptionWithAuth, makeReference<RandomEncryptionKeyProvider<AESEncryptionWithAuth>>() }
};
state IKeyValueStore* kvs = nullptr;
g_allowXOREncryptionInSimulation = false;
for (const auto& testCase : testCases) {
state EncodingType initialEncodingType = testCase.first;
state EncodingType reopenEncodingType = testCase.second;
ASSERT_NE(initialEncodingType, reopenEncodingType);
ASSERT(ArenaPage::isEncodingTypeEncrypted(reopenEncodingType));
deleteFile("test.redwood-v1");
printf("Create KV store with encoding type %d\n", initialEncodingType);
setAuthMode(initialEncodingType);
kvs = openKVStore(KeyValueStoreType::SSD_REDWOOD_V1,
"test.redwood-v1",
UID(),
0,
false,
false,
false,
encryptionKeyProviders.at(initialEncodingType));
wait(kvs->init());
kvs->set(KeyValueRef("foo"_sr, "bar"_sr));
wait(kvs->commit());
wait(closeKVS(kvs));
// Reopen
printf("Reopen KV store with encoding type %d\n", reopenEncodingType);
setAuthMode(reopenEncodingType);
kvs = openKVStore(KeyValueStoreType::SSD_REDWOOD_V1,
"test.redwood-v1",
UID(),
0,
false,
false,
false,
encryptionKeyProviders.at(reopenEncodingType));
wait(kvs->init());
try {
Optional<Value> v = wait(kvs->readValue("foo"_sr));
UNREACHABLE();
} catch (Error& e) {
ASSERT_EQ(e.code(), error_code_unexpected_encoding_type);
}
wait(closeKVS(kvs, true /*dispose*/));
}
return Void();
}

View File

@ -163,8 +163,11 @@ ACTOR Future<Void> printRestoreSummary(Database db, Reference<BlobConnectionProv
ACTOR Future<BlobGranuleRestoreVersionVector> listBlobGranules(Database db, Reference<BlobConnectionProvider> blobConn);
ACTOR Future<int64_t> lastBlobEpoc(Database db, Reference<BlobConnectionProvider> blobConn);
ACTOR Future<bool> isFullRestoreMode(Database db, KeyRangeRef range);
ACTOR Future<Void> updateRestoreStatus(Database db, KeyRangeRef range, BlobRestoreStatus status);
ACTOR Future<KeyRange> getRestoringRange(Database db, KeyRangeRef keys);
ACTOR Future<Void> updateRestoreStatus(Database db,
KeyRangeRef range,
BlobRestoreStatus status,
Optional<BlobRestorePhase> expectedPhase);
ACTOR Future<std::pair<KeyRange, BlobRestoreStatus>> getRestoreRangeStatus(Database db, KeyRangeRef keys);
ACTOR Future<Optional<BlobRestoreStatus>> getRestoreStatus(Database db, KeyRangeRef range);
#include "flow/unactorcompiler.h"

View File

@ -318,13 +318,13 @@ public:
}
};
bool workerAvailable(WorkerInfo const& worker, bool checkStable) {
bool workerAvailable(WorkerInfo const& worker, bool checkStable) const {
return (now() - startTime < 2 * FLOW_KNOBS->SERVER_REQUEST_INTERVAL) ||
(IFailureMonitor::failureMonitor().getState(worker.details.interf.storage.getEndpoint()).isAvailable() &&
(!checkStable || worker.reboots < 2));
}
bool isLongLivedStateless(Optional<Key> const& processId) {
bool isLongLivedStateless(Optional<Key> const& processId) const {
return (db.serverInfo->get().distributor.present() &&
db.serverInfo->get().distributor.get().locality.processId() == processId) ||
(db.serverInfo->get().ratekeeper.present() &&
@ -2181,27 +2181,29 @@ public:
void compareWorkers(const DatabaseConfiguration& conf,
const std::vector<WorkerInterface>& first,
std::map<Optional<Standalone<StringRef>>, int>& firstUsed,
const std::map<Optional<Standalone<StringRef>>, int>& firstUsed,
const std::vector<WorkerInterface>& second,
std::map<Optional<Standalone<StringRef>>, int>& secondUsed,
const std::map<Optional<Standalone<StringRef>>, int>& secondUsed,
ProcessClass::ClusterRole role,
std::string description) {
std::vector<WorkerDetails> firstDetails;
for (auto& it : first) {
auto w = id_worker.find(it.locality.processId());
for (auto& worker : first) {
auto w = id_worker.find(worker.locality.processId());
ASSERT(w != id_worker.end());
ASSERT(!conf.isExcludedServer(w->second.details.interf.addresses()));
firstDetails.push_back(w->second.details);
auto const& [_, workerInfo] = *w;
ASSERT(!conf.isExcludedServer(workerInfo.details.interf.addresses()));
firstDetails.push_back(workerInfo.details);
//TraceEvent("CompareAddressesFirst").detail(description.c_str(), w->second.details.interf.address());
}
RoleFitness firstFitness(firstDetails, role, firstUsed);
std::vector<WorkerDetails> secondDetails;
for (auto& it : second) {
auto w = id_worker.find(it.locality.processId());
for (auto& worker : second) {
auto w = id_worker.find(worker.locality.processId());
ASSERT(w != id_worker.end());
ASSERT(!conf.isExcludedServer(w->second.details.interf.addresses()));
secondDetails.push_back(w->second.details);
auto const& [_, workerInfo] = *w;
ASSERT(!conf.isExcludedServer(workerInfo.details.interf.addresses()));
secondDetails.push_back(workerInfo.details);
//TraceEvent("CompareAddressesSecond").detail(description.c_str(), w->second.details.interf.address());
}
RoleFitness secondFitness(secondDetails, role, secondUsed);
@ -3323,7 +3325,7 @@ public:
return recentHealthTriggeredRecoveryTime.size();
}
bool isExcludedDegradedServer(const NetworkAddressList& a) {
bool isExcludedDegradedServer(const NetworkAddressList& a) const {
for (const auto& server : excludedDegradedServers) {
if (a.contains(server))
return true;

View File

@ -198,6 +198,7 @@ struct DDTeamCollectionInitParams {
PromiseStream<GetMetricsRequest> getShardMetrics;
Promise<UID> removeFailedServer;
PromiseStream<Promise<int>> getUnhealthyRelocationCount;
PromiseStream<Promise<int64_t>> getAverageShardBytes;
};
class DDTeamCollection : public ReferenceCounted<DDTeamCollection> {
@ -235,6 +236,7 @@ protected:
Reference<AsyncVar<bool>> pauseWiggle;
Reference<AsyncVar<bool>> processingWiggle; // track whether wiggling relocation is being processed
PromiseStream<StorageWiggleValue> nextWiggleInfo;
PromiseStream<Promise<int64_t>> getAverageShardBytes;
std::vector<Reference<TCTeamInfo>> badTeams;
Reference<ShardsAffectedByTeamFailure> shardsAffectedByTeamFailure;
@ -463,6 +465,10 @@ protected:
Future<Void> waitForAllDataRemoved(UID serverID, Version addedVersion) const;
// calculate minLoadBytes / avgLoadBytes among servers. An unhealthy server's load is considered as 0. If the
// average load of each storage server is less than smallLoadThreshold, return 1 always.
double loadBytesBalanceRatio(int64_t smallLoadThreshold) const;
// Create a transaction updating `perpetualStorageWiggleIDPrefix` to the next serverID according to a sorted
// wiggle_pq maintained by the wiggler.
Future<Void> updateNextWigglingStorageID();

View File

@ -279,11 +279,13 @@ public:
struct PhysicalShard {
PhysicalShard() : id(UID().first()) {}
PhysicalShard(uint64_t id,
PhysicalShard(Reference<IDDTxnProcessor> txnProcessor,
uint64_t id,
StorageMetrics const& metrics,
std::vector<ShardsAffectedByTeamFailure::Team> teams,
PhysicalShardCreationTime whenCreated)
: id(id), metrics(metrics), teams(teams), whenCreated(whenCreated) {}
: txnProcessor(txnProcessor), id(id), metrics(metrics),
stats(makeReference<AsyncVar<Optional<StorageMetrics>>>()), teams(teams), whenCreated(whenCreated) {}
// Adds `newRange` to this physical shard and starts monitoring the shard.
void addRange(const KeyRange& newRange);
@ -293,17 +295,23 @@ public:
std::string toString() const { return fmt::format("{}", std::to_string(id)); }
Reference<IDDTxnProcessor> txnProcessor;
uint64_t id; // physical shard id (never changed)
StorageMetrics metrics; // current metrics, updated by shardTracker
// todo(zhewu): combine above metrics with stats. They are redundant.
Reference<AsyncVar<Optional<StorageMetrics>>> stats; // Stats of this physical shard.
std::vector<ShardsAffectedByTeamFailure::Team> teams; // which team owns this physical shard (never changed)
PhysicalShardCreationTime whenCreated; // when this physical shard is created (never changed)
struct RangeData {
Future<Void> trackMetrics; // TODO(zhewu): add shard tracking actor.
Reference<AsyncVar<Optional<ShardMetrics>>>
stats; // TODO(zhewu): aggregate all metrics to a single physical shard metrics.
Future<Void> trackMetrics;
Reference<AsyncVar<Optional<ShardMetrics>>> stats;
};
std::unordered_map<KeyRange, RangeData> rangeData;
private:
// Inserts a new key range into this physical shard. `newRange` must not exist in this shard already.
void insertNewRangeData(const KeyRange& newRange);
};
// Generate a random physical shard ID, which is not UID().first() nor anonymousShardId.first()

View File

@ -20,7 +20,7 @@
#ifndef FDBSERVER_ICLOSABLE_H
#define FDBSERVER_ICLOSABLE_H
#pragma once
#include "flow/flow.h"
class IClosable {
public:
// IClosable is a base interface for any disk-backed data structure that needs to support asynchronous errors,

View File

@ -162,6 +162,9 @@ public:
// Key provider to provider cipher keys randomly from a pre-generated pool. It does not maintain encryption domains.
// Use for testing.
template <EncodingType encodingType,
typename std::enable_if<encodingType == AESEncryption || encodingType == AESEncryptionWithAuth, bool>::type =
true>
class RandomEncryptionKeyProvider : public IPageEncryptionKeyProvider {
public:
enum EncryptionDomainMode : unsigned int {
@ -171,7 +174,7 @@ public:
MAX,
};
explicit RandomEncryptionKeyProvider(EncryptionDomainMode mode) : mode(mode) {
explicit RandomEncryptionKeyProvider(EncryptionDomainMode mode = DISABLED) : mode(mode) {
ASSERT(mode < EncryptionDomainMode::MAX);
for (unsigned i = 0; i < NUM_CIPHER; i++) {
BlobCipherDetails cipherDetails;
@ -183,19 +186,20 @@ public:
}
virtual ~RandomEncryptionKeyProvider() = default;
EncodingType expectedEncodingType() const override { return EncodingType::AESEncryptionV1; }
EncodingType expectedEncodingType() const override { return encodingType; }
bool enableEncryption() const override { return true; }
bool enableEncryptionDomain() const override { return mode > 1; }
Future<EncryptionKey> getEncryptionKey(const void* encodingHeader) override {
using Header = ArenaPage::AESEncryptionV1Encoder::Header;
using Header = typename ArenaPage::AESEncryptionEncoder<encodingType>::Header;
const Header* h = reinterpret_cast<const Header*>(encodingHeader);
EncryptionKey s;
s.aesKey.cipherTextKey = getCipherKey(h->cipherTextDetails.encryptDomainId, h->cipherTextDetails.baseCipherId);
s.aesKey.cipherHeaderKey =
getCipherKey(h->cipherHeaderDetails.encryptDomainId, h->cipherHeaderDetails.baseCipherId);
s.aesKey.cipherTextKey =
getCipherKey(h->encryption.cipherTextDetails.encryptDomainId, h->encryption.cipherTextDetails.baseCipherId);
s.aesKey.cipherHeaderKey = getCipherKey(h->encryption.cipherHeaderDetails.encryptDomainId,
h->encryption.cipherHeaderDetails.baseCipherId);
return s;
}
@ -223,9 +227,9 @@ public:
int64_t getEncryptionDomainIdFromHeader(const void* encodingHeader) override {
ASSERT(encodingHeader != nullptr);
using Header = ArenaPage::AESEncryptionV1Encoder::Header;
using Header = typename ArenaPage::AESEncryptionEncoder<encodingType>::Header;
const Header* h = reinterpret_cast<const Header*>(encodingHeader);
return h->cipherTextDetails.encryptDomainId;
return h->encryption.cipherTextDetails.encryptDomainId;
}
private:
@ -277,9 +281,12 @@ private:
// Key provider which extract tenant id from range key prefixes, and fetch tenant specific encryption keys from
// EncryptKeyProxy.
template <EncodingType encodingType,
typename std::enable_if<encodingType == AESEncryption || encodingType == AESEncryptionWithAuth, bool>::type =
true>
class TenantAwareEncryptionKeyProvider : public IPageEncryptionKeyProvider {
public:
using EncodingHeader = ArenaPage::AESEncryptionV1Encoder::Header;
using EncodingHeader = typename ArenaPage::AESEncryptionEncoder<encodingType>::Header;
const StringRef systemKeysPrefix = systemKeys.begin;
@ -287,7 +294,7 @@ public:
virtual ~TenantAwareEncryptionKeyProvider() = default;
EncodingType expectedEncodingType() const override { return EncodingType::AESEncryptionV1; }
EncodingType expectedEncodingType() const override { return encodingType; }
bool enableEncryption() const override {
return isEncryptionOpSupported(EncryptOperationType::STORAGE_SERVER_ENCRYPTION);
@ -297,9 +304,9 @@ public:
ACTOR static Future<EncryptionKey> getEncryptionKey(TenantAwareEncryptionKeyProvider* self,
const void* encodingHeader) {
const BlobCipherEncryptHeader* header = reinterpret_cast<const EncodingHeader*>(encodingHeader);
const BlobCipherEncryptHeader& header = reinterpret_cast<const EncodingHeader*>(encodingHeader)->encryption;
TextAndHeaderCipherKeys cipherKeys =
wait(getEncryptCipherKeys(self->db, *header, BlobCipherMetrics::KV_REDWOOD));
wait(getEncryptCipherKeys(self->db, header, BlobCipherMetrics::KV_REDWOOD));
EncryptionKey encryptionKey;
encryptionKey.aesKey = cipherKeys;
return encryptionKey;
@ -349,8 +356,8 @@ public:
int64_t getEncryptionDomainIdFromHeader(const void* encodingHeader) override {
ASSERT(encodingHeader != nullptr);
const BlobCipherEncryptHeader* header = reinterpret_cast<const EncodingHeader*>(encodingHeader);
return header->cipherTextDetails.encryptDomainId;
const BlobCipherEncryptHeader& header = reinterpret_cast<const EncodingHeader*>(encodingHeader)->encryption;
return header.cipherTextDetails.encryptDomainId;
}
private:

View File

@ -94,7 +94,13 @@ static const std::vector<std::pair<PagerEvents, PagerEventReasons>> L0PossibleEv
{ PagerEvents::PageWrite, PagerEventReasons::MetaData },
};
enum EncodingType : uint8_t { XXHash64 = 0, XOREncryption_TestOnly = 1, AESEncryptionV1 = 2, MAX_ENCODING_TYPE = 3 };
enum EncodingType : uint8_t {
XXHash64 = 0,
XOREncryption_TestOnly = 1,
AESEncryption = 2,
AESEncryptionWithAuth = 3,
MAX_ENCODING_TYPE = 4
};
enum PageType : uint8_t {
HeaderPage = 0,
@ -224,13 +230,12 @@ public:
int rawSize() const { return bufferSize; }
// Encryption key used to encrypt a page. Different encoding types may use different structs to represent
// an encryption key, and EncryptionKeyRef is a union of these structs.
struct EncryptionKeyRef {
TextAndHeaderCipherKeys aesKey; // For AESEncryptionV1
// an encryption key, and EncryptionKey is a union of these structs.
struct EncryptionKey {
TextAndHeaderCipherKeys aesKey; // For AESEncryption and AESEncryptionWithAuth
uint8_t xorKey; // For XOREncryption_TestOnly
uint8_t xorWith; // For XOREncryption_TestOnly
};
using EncryptionKey = Standalone<EncryptionKeyRef>;
#pragma pack(push, 1)
@ -368,31 +373,65 @@ public:
}
};
struct AESEncryptionV1Encoder {
using Header = BlobCipherEncryptHeader;
// By default, xxhash is used to checksum the page. But ff authentication is enabled (such as when we are using
// aes256-ctr-hmac-sha256 encryption scheme), the auth tag plays the role of a checksum while assuring authenticity
// of the data. xxhash checksum is not needed in this case.
template <EncodingType encodingType,
typename std::enable_if<encodingType == AESEncryption || encodingType == AESEncryptionWithAuth,
bool>::type = true>
struct AESEncryptionEncoder {
struct AESEncryptionEncodingHeader {
BlobCipherEncryptHeader encryption;
XXH64_hash_t checksum;
};
static void encode(void* header, const TextAndHeaderCipherKeys& cipherKeys, uint8_t* payload, int len) {
struct AESEncryptionWithAuthEncodingHeader {
BlobCipherEncryptHeader encryption;
};
using Header = typename std::conditional<encodingType == AESEncryption,
AESEncryptionEncodingHeader,
AESEncryptionWithAuthEncodingHeader>::type;
static void encode(void* header,
const TextAndHeaderCipherKeys& cipherKeys,
uint8_t* payload,
int len,
PhysicalPageID seed) {
Header* h = reinterpret_cast<Header*>(header);
EncryptBlobCipherAes265Ctr cipher(cipherKeys.cipherTextKey,
cipherKeys.cipherHeaderKey,
getEncryptAuthTokenMode(ENCRYPT_HEADER_AUTH_TOKEN_MODE_SINGLE),
BlobCipherMetrics::KV_REDWOOD);
Arena arena;
StringRef ciphertext = cipher.encrypt(payload, len, h, arena)->toStringRef();
StringRef ciphertext = cipher.encrypt(payload, len, &h->encryption, arena)->toStringRef();
ASSERT_EQ(len, ciphertext.size());
memcpy(payload, ciphertext.begin(), len);
if constexpr (encodingType == AESEncryption) {
h->checksum = XXH3_64bits_withSeed(payload, len, seed);
}
}
static void decode(void* header, const TextAndHeaderCipherKeys& cipherKeys, uint8_t* payload, int len) {
static void decode(void* header,
const TextAndHeaderCipherKeys& cipherKeys,
uint8_t* payload,
int len,
PhysicalPageID seed) {
Header* h = reinterpret_cast<Header*>(header);
if constexpr (encodingType == AESEncryption) {
if (h->checksum != XXH3_64bits_withSeed(payload, len, seed)) {
throw page_decoding_failed();
}
}
DecryptBlobCipherAes256Ctr cipher(
cipherKeys.cipherTextKey, cipherKeys.cipherHeaderKey, h->iv, BlobCipherMetrics::KV_REDWOOD);
cipherKeys.cipherTextKey, cipherKeys.cipherHeaderKey, h->encryption.iv, BlobCipherMetrics::KV_REDWOOD);
Arena arena;
StringRef plaintext = cipher.decrypt(payload, len, *h, arena)->toStringRef();
StringRef plaintext = cipher.decrypt(payload, len, h->encryption, arena)->toStringRef();
ASSERT_EQ(len, plaintext.size());
memcpy(payload, plaintext.begin(), len);
}
};
#pragma pack(pop)
// Get the size of the encoding header based on type
@ -403,8 +442,10 @@ public:
return sizeof(XXHashEncoder::Header);
} else if (t == EncodingType::XOREncryption_TestOnly) {
return sizeof(XOREncryptionEncoder::Header);
} else if (t == EncodingType::AESEncryptionV1) {
return sizeof(AESEncryptionV1Encoder::Header);
} else if (t == EncodingType::AESEncryption) {
return sizeof(AESEncryptionEncoder<AESEncryption>::Header);
} else if (t == EncodingType::AESEncryptionWithAuth) {
return sizeof(AESEncryptionEncoder<AESEncryptionWithAuth>::Header);
} else {
throw page_encoding_not_supported();
}
@ -511,8 +552,12 @@ public:
XXHashEncoder::encode(page->getEncodingHeader(), pPayload, payloadSize, pageID);
} else if (page->encodingType == EncodingType::XOREncryption_TestOnly) {
XOREncryptionEncoder::encode(page->getEncodingHeader(), encryptionKey, pPayload, payloadSize, pageID);
} else if (page->encodingType == EncodingType::AESEncryptionV1) {
AESEncryptionV1Encoder::encode(page->getEncodingHeader(), encryptionKey.aesKey, pPayload, payloadSize);
} else if (page->encodingType == EncodingType::AESEncryption) {
AESEncryptionEncoder<AESEncryption>::encode(
page->getEncodingHeader(), encryptionKey.aesKey, pPayload, payloadSize, pageID);
} else if (page->encodingType == EncodingType::AESEncryptionWithAuth) {
AESEncryptionEncoder<AESEncryptionWithAuth>::encode(
page->getEncodingHeader(), encryptionKey.aesKey, pPayload, payloadSize, pageID);
} else {
throw page_encoding_not_supported();
}
@ -557,8 +602,12 @@ public:
XXHashEncoder::decode(page->getEncodingHeader(), pPayload, payloadSize, pageID);
} else if (page->encodingType == EncodingType::XOREncryption_TestOnly) {
XOREncryptionEncoder::decode(page->getEncodingHeader(), encryptionKey, pPayload, payloadSize, pageID);
} else if (page->encodingType == EncodingType::AESEncryptionV1) {
AESEncryptionV1Encoder::decode(page->getEncodingHeader(), encryptionKey.aesKey, pPayload, payloadSize);
} else if (page->encodingType == EncodingType::AESEncryption) {
AESEncryptionEncoder<AESEncryption>::decode(
page->getEncodingHeader(), encryptionKey.aesKey, pPayload, payloadSize, pageID);
} else if (page->encodingType == EncodingType::AESEncryptionWithAuth) {
AESEncryptionEncoder<AESEncryptionWithAuth>::decode(
page->getEncodingHeader(), encryptionKey.aesKey, pPayload, payloadSize, pageID);
} else {
throw page_encoding_not_supported();
}
@ -567,7 +616,8 @@ public:
const Arena& getArena() const { return arena; }
static bool isEncodingTypeEncrypted(EncodingType t) {
return t == EncodingType::AESEncryptionV1 || t == EncodingType::XOREncryption_TestOnly;
return t == EncodingType::AESEncryption || t == EncodingType::AESEncryptionWithAuth ||
t == EncodingType::XOREncryption_TestOnly;
}
// Returns true if the page's encoding type employs encryption
@ -576,8 +626,9 @@ public:
// Return encryption domain id used. This method only use information from the encryptionKey.
// Caller should make sure encryption domain is in use.
int64_t getEncryptionDomainId() const {
// encryption domain is only supported by AESEncryptionV1.
ASSERT(getEncodingType() == EncodingType::AESEncryptionV1);
// encryption domain is only supported by AESEncryption and AESEncryptionWithAuth.
ASSERT(getEncodingType() == EncodingType::AESEncryption ||
getEncodingType() == EncodingType::AESEncryptionWithAuth);
const Reference<BlobCipherKey>& cipherKey = encryptionKey.aesKey.cipherTextKey;
ASSERT(cipherKey.isValid());
return cipherKey->getDomainId();

View File

@ -40,6 +40,8 @@ struct CheckpointFile {
CheckpointFile() = default;
CheckpointFile(std::string path, KeyRange range, int64_t size) : path(path), range(range), size(size) {}
bool isValid() const { return !path.empty(); }
std::string toString() const {
return "CheckpointFile:\nFile Name: " + this->path + "\nRange: " + range.toString() +
"\nSize: " + std::to_string(size) + "\n";
@ -241,6 +243,30 @@ struct RocksDBCheckpoint {
}
};
struct RocksDBCheckpointKeyValues {
constexpr static FileIdentifier file_identifier = 13804349;
std::vector<CheckpointFile> fetchedFiles; // Used for fetchCheckpoint, to record the progress.
std::vector<KeyRange> ranges; // The ranges we want to fetch.
RocksDBCheckpointKeyValues(std::vector<KeyRange> ranges) : ranges(ranges) {}
RocksDBCheckpointKeyValues() = default;
CheckpointFormat format() const { return RocksDBKeyValues; }
std::string toString() const {
std::string res = "RocksDBKeyValuesCheckpoint: [Target Ranges]: " + describe(ranges) + " [Fetched Files]: ";
for (const auto& file : fetchedFiles) {
res += file.toString();
}
return res;
}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, fetchedFiles, ranges);
}
};
// Fetch the checkpoint file(s) to local dir, the checkpoint is specified by initialState.
// If cFun is provided, the fetch progress can be checkpointed, so that next time, the fetch process
// can be continued, in case of crash.
@ -255,12 +281,16 @@ int64_t getTotalFetchedBytes(const std::vector<CheckpointMetaData>& checkpoints)
// Clean up on-disk files associated with checkpoint.
ACTOR Future<Void> deleteRocksCheckpoint(CheckpointMetaData checkpoint);
ICheckpointReader* newRocksDBCheckpointReader(const CheckpointMetaData& checkpoint, UID logID);
ICheckpointReader* newRocksDBCheckpointReader(const CheckpointMetaData& checkpoint,
const CheckpointAsKeyValues checkpointAsKeyValues,
UID logID);
RocksDBColumnFamilyCheckpoint getRocksCF(const CheckpointMetaData& checkpoint);
RocksDBCheckpoint getRocksCheckpoint(const CheckpointMetaData& checkpoint);
RocksDBCheckpointKeyValues getRocksKeyValuesCheckpoint(const CheckpointMetaData& checkpoint);
#include "flow/unactorcompiler.h"
#endif

View File

@ -31,6 +31,15 @@
#include "flow/actorcompiler.h" // has to be last include
FDB_DECLARE_BOOLEAN_PARAM(CheckpointAsKeyValues);
class ICheckpointIterator {
public:
virtual Future<RangeResult> nextBatch(const int rowLimit, const int ByteLimit) = 0;
virtual ~ICheckpointIterator() {}
};
// An ICheckpointReader can read the contents of a checkpoint created from a KV store,
// i.e., by IKeyValueStore::checkpoint().
class ICheckpointReader {
@ -47,11 +56,17 @@ public:
virtual Future<Void> close() = 0;
virtual std::unique_ptr<ICheckpointIterator> getIterator(KeyRange range) { throw not_implemented(); }
virtual bool inUse() const { return false; }
protected:
virtual ~ICheckpointReader() {}
};
ICheckpointReader* newCheckpointReader(const CheckpointMetaData& checkpoint, UID logID);
ICheckpointReader* newCheckpointReader(const CheckpointMetaData& checkpoint,
const CheckpointAsKeyValues checkpointAsKeyValues,
UID logID);
// Delete a checkpoint.
ACTOR Future<Void> deleteCheckpoint(CheckpointMetaData checkpoint);
@ -64,10 +79,12 @@ ACTOR Future<CheckpointMetaData> fetchCheckpoint(Database cx,
std::string dir,
std::function<Future<Void>(const CheckpointMetaData&)> cFun = nullptr);
ACTOR Future<std::vector<CheckpointMetaData>> fetchCheckpoints(
// Same as above, except that the checkpoint is fetched as key-value pairs.
ACTOR Future<CheckpointMetaData> fetchCheckpointRanges(
Database cx,
std::vector<CheckpointMetaData> initialStates,
CheckpointMetaData initialState,
std::string dir,
std::vector<KeyRange> ranges,
std::function<Future<Void>(const CheckpointMetaData&)> cFun = nullptr);
#include "flow/unactorcompiler.h"

View File

@ -0,0 +1,215 @@
/*
* SingletonRoles.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
#include "fdbserver/RatekeeperInterface.h"
#include "fdbserver/DataDistributorInterface.h"
// This is used to artificially amplify the used count for processes
// occupied by non-singletons. This ultimately makes it less desirable
// for singletons to use those processes as well. This constant should
// be increased if we ever have more than 100 singletons (unlikely).
static constexpr int PID_USED_AMP_FOR_NON_SINGLETON = 100;
// Wrapper for singleton interfaces
template <class Interface>
class Singleton {
protected:
const Optional<Interface>& interface;
explicit Singleton(const Optional<Interface>& interface) : interface(interface) {}
public:
Interface const& getInterface() const { return interface.get(); }
bool isPresent() const { return interface.present(); }
};
struct RatekeeperSingleton : Singleton<RatekeeperInterface> {
RatekeeperSingleton(const Optional<RatekeeperInterface>& interface) : Singleton(interface) {}
Role getRole() const { return Role::RATEKEEPER; }
ProcessClass::ClusterRole getClusterRole() const { return ProcessClass::Ratekeeper; }
void setInterfaceToDbInfo(ClusterControllerData& cc) const {
if (interface.present()) {
TraceEvent("CCRK_SetInf", cc.id).detail("Id", interface.get().id());
cc.db.setRatekeeper(interface.get());
}
}
void halt(ClusterControllerData& cc, Optional<Standalone<StringRef>> pid) const {
if (interface.present() && cc.id_worker.count(pid)) {
cc.id_worker[pid].haltRatekeeper =
brokenPromiseToNever(interface.get().haltRatekeeper.getReply(HaltRatekeeperRequest(cc.id)));
}
}
void recruit(ClusterControllerData& cc) const {
cc.lastRecruitTime = now();
cc.recruitRatekeeper.set(true);
}
};
struct DataDistributorSingleton : Singleton<DataDistributorInterface> {
DataDistributorSingleton(const Optional<DataDistributorInterface>& interface) : Singleton(interface) {}
Role getRole() const { return Role::DATA_DISTRIBUTOR; }
ProcessClass::ClusterRole getClusterRole() const { return ProcessClass::DataDistributor; }
void setInterfaceToDbInfo(ClusterControllerData& cc) const {
if (interface.present()) {
TraceEvent("CCDD_SetInf", cc.id).detail("Id", interface.get().id());
cc.db.setDistributor(interface.get());
}
}
void halt(ClusterControllerData& cc, Optional<Standalone<StringRef>> pid) const {
if (interface.present() && cc.id_worker.count(pid)) {
cc.id_worker[pid].haltDistributor =
brokenPromiseToNever(interface.get().haltDataDistributor.getReply(HaltDataDistributorRequest(cc.id)));
}
}
void recruit(ClusterControllerData& cc) const {
cc.lastRecruitTime = now();
cc.recruitDistributor.set(true);
}
};
struct ConsistencyScanSingleton : Singleton<ConsistencyScanInterface> {
ConsistencyScanSingleton(const Optional<ConsistencyScanInterface>& interface) : Singleton(interface) {}
Role getRole() const { return Role::CONSISTENCYSCAN; }
ProcessClass::ClusterRole getClusterRole() const { return ProcessClass::ConsistencyScan; }
void setInterfaceToDbInfo(ClusterControllerData& cc) const {
if (interface.present()) {
TraceEvent("CCCK_SetInf", cc.id).detail("Id", interface.get().id());
cc.db.setConsistencyScan(interface.get());
}
}
void halt(ClusterControllerData& cc, Optional<Standalone<StringRef>> pid) const {
if (interface.present()) {
cc.id_worker[pid].haltConsistencyScan =
brokenPromiseToNever(interface.get().haltConsistencyScan.getReply(HaltConsistencyScanRequest(cc.id)));
}
}
void recruit(ClusterControllerData& cc) const {
cc.lastRecruitTime = now();
cc.recruitConsistencyScan.set(true);
}
};
struct BlobManagerSingleton : Singleton<BlobManagerInterface> {
BlobManagerSingleton(const Optional<BlobManagerInterface>& interface) : Singleton(interface) {}
Role getRole() const { return Role::BLOB_MANAGER; }
ProcessClass::ClusterRole getClusterRole() const { return ProcessClass::BlobManager; }
void setInterfaceToDbInfo(ClusterControllerData& cc) const {
if (interface.present()) {
TraceEvent("CCBM_SetInf", cc.id).detail("Id", interface.get().id());
cc.db.setBlobManager(interface.get());
}
}
void halt(ClusterControllerData& cc, Optional<Standalone<StringRef>> pid) const {
if (interface.present() && cc.id_worker.count(pid)) {
cc.id_worker[pid].haltBlobManager =
brokenPromiseToNever(interface.get().haltBlobManager.getReply(HaltBlobManagerRequest(cc.id)));
}
}
void recruit(ClusterControllerData& cc) const {
cc.lastRecruitTime = now();
cc.recruitBlobManager.set(true);
}
void haltBlobGranules(ClusterControllerData& cc, Optional<Standalone<StringRef>> pid) const {
if (interface.present()) {
cc.id_worker[pid].haltBlobManager =
brokenPromiseToNever(interface.get().haltBlobGranules.getReply(HaltBlobGranulesRequest(cc.id)));
}
}
};
struct BlobMigratorSingleton : Singleton<BlobMigratorInterface> {
BlobMigratorSingleton(const Optional<BlobMigratorInterface>& interface) : Singleton(interface) {}
Role getRole() const { return Role::BLOB_MIGRATOR; }
ProcessClass::ClusterRole getClusterRole() const { return ProcessClass::BlobMigrator; }
void setInterfaceToDbInfo(ClusterControllerData& cc) const {
if (interface.present()) {
TraceEvent("CCMG_SetInf", cc.id).detail("Id", interface.get().id());
cc.db.setBlobMigrator(interface.get());
}
}
void halt(ClusterControllerData& cc, Optional<Standalone<StringRef>> pid) const {
if (interface.present()) {
TraceEvent("CCMG_Halt", cc.id).detail("Id", interface.get().id());
cc.id_worker[pid].haltBlobMigrator =
brokenPromiseToNever(interface.get().haltBlobMigrator.getReply(HaltBlobMigratorRequest(cc.id)));
}
}
void recruit(ClusterControllerData& cc) const {
cc.lastRecruitTime = now();
cc.recruitBlobMigrator.set(true);
}
};
struct EncryptKeyProxySingleton : Singleton<EncryptKeyProxyInterface> {
EncryptKeyProxySingleton(const Optional<EncryptKeyProxyInterface>& interface) : Singleton(interface) {}
Role getRole() const { return Role::ENCRYPT_KEY_PROXY; }
ProcessClass::ClusterRole getClusterRole() const { return ProcessClass::EncryptKeyProxy; }
void setInterfaceToDbInfo(ClusterControllerData& cc) const {
if (interface.present()) {
TraceEvent("CCEKP_SetInf", cc.id).detail("Id", interface.get().id());
cc.db.setEncryptKeyProxy(interface.get());
}
}
void halt(ClusterControllerData& cc, Optional<Standalone<StringRef>> pid) const {
if (interface.present() && cc.id_worker.count(pid)) {
cc.id_worker[pid].haltEncryptKeyProxy =
brokenPromiseToNever(interface.get().haltEncryptKeyProxy.getReply(HaltEncryptKeyProxyRequest(cc.id)));
}
}
void recruit(ClusterControllerData& cc) const {
cc.lastRecruitTime = now();
cc.recruitEncryptKeyProxy.set(true);
}
};
struct SingletonRecruitThrottler {
double lastRecruitStart;
SingletonRecruitThrottler() : lastRecruitStart(-1) {}
double newRecruitment() {
double n = now();
double waitTime =
std::max(0.0, (lastRecruitStart + SERVER_KNOBS->CC_THROTTLE_SINGLETON_RERECRUIT_INTERVAL - n));
lastRecruitStart = n;
return waitTime;
}
};

View File

@ -192,7 +192,7 @@ Future<Void> serveStorageMetricsRequests(ServiceType* self, StorageServerInterfa
choose {
when(state WaitMetricsRequest req = waitNext(ssi.waitMetrics.getFuture())) {
if (!req.tenantInfo.hasTenant() && !self->isReadable(req.keys)) {
CODE_PROBE(true, "waitMetrics immediate wrong_shard_server()", probe::decoration::rare);
CODE_PROBE(true, "waitMetrics immediate wrong_shard_server()");
self->sendErrorWithPenalty(req.reply, wrong_shard_server(), self->getPenalty());
} else {
self->addActor(self->waitMetricsTenantAware(req));

View File

@ -2425,7 +2425,7 @@ ACTOR Future<Void> fetchCheckpointQ(StorageServer* self, FetchCheckpointRequest
}
try {
reader = newCheckpointReader(it->second, deterministicRandom()->randomUniqueID());
reader = newCheckpointReader(it->second, CheckpointAsKeyValues::False, deterministicRandom()->randomUniqueID());
wait(reader->init(req.token));
loop {
@ -2479,13 +2479,15 @@ ACTOR Future<Void> fetchCheckpointKeyValuesQ(StorageServer* self, FetchCheckpoin
}
state ICheckpointReader* reader = nullptr;
state std::unique_ptr<ICheckpointIterator> iter;
try {
reader = newCheckpointReader(it->second, self->thisServerID);
reader = newCheckpointReader(it->second, CheckpointAsKeyValues::True, self->thisServerID);
wait(reader->init(BinaryWriter::toValue(req.range, IncludeVersion())));
iter = reader->getIterator(req.range);
loop {
state RangeResult res =
wait(reader->nextKeyValues(CLIENT_KNOBS->REPLY_BYTE_LIMIT, CLIENT_KNOBS->REPLY_BYTE_LIMIT));
wait(iter->nextBatch(CLIENT_KNOBS->REPLY_BYTE_LIMIT, CLIENT_KNOBS->REPLY_BYTE_LIMIT));
if (!res.empty()) {
TraceEvent(SevDebug, "FetchCheckpontKeyValuesReadRange", self->thisServerID)
.detail("CheckpointID", req.checkpointID)
@ -2524,7 +2526,10 @@ ACTOR Future<Void> fetchCheckpointKeyValuesQ(StorageServer* self, FetchCheckpoin
}
}
wait(reader->close());
iter.reset();
if (!reader->inUse()) {
wait(reader->close());
}
return Void();
}
@ -6279,6 +6284,7 @@ ACTOR Future<Standalone<VectorRef<BlobGranuleChunkRef>>> tryReadBlobGranules(Tra
// Read keys from blob storage if they exist. Fail back to tryGetRange, which reads keys
// from storage servers with locally attached disks
ACTOR Future<Void> tryGetRangeFromBlob(PromiseStream<RangeResult> results,
Transaction* tr,
KeyRange keys,
@ -6315,7 +6321,7 @@ ACTOR Future<Void> tryGetRangeFromBlob(PromiseStream<RangeResult> results,
.detail("Error", e.what());
tr->reset();
tr->setVersion(fetchVersion);
throw;
results.sendError(e);
}
return Void();
}
@ -7223,8 +7229,15 @@ ACTOR Future<Void> fetchKeys(StorageServer* data, AddingShard* shard) {
state PromiseStream<RangeResult> results;
state Future<Void> hold;
if (isFullRestore) {
KeyRange range = wait(getRestoringRange(data->cx, keys));
hold = tryGetRangeFromBlob(results, &tr, range, fetchVersion, data->blobConn);
std::pair<KeyRange, BlobRestoreStatus> rangeStatus = wait(getRestoreRangeStatus(data->cx, keys));
// Read from blob only when it's copying data for full restore. Otherwise it may cause data corruptions
// e.g we don't want to copy from blob any more when it's applying mutation logs(APPLYING_MLOGS)
if (rangeStatus.second.phase == BlobRestorePhase::COPYING_DATA ||
rangeStatus.second.phase == BlobRestorePhase::ERROR) {
hold = tryGetRangeFromBlob(results, &tr, rangeStatus.first, fetchVersion, data->blobConn);
} else {
hold = tryGetRange(results, &tr, keys);
}
} else {
hold = tryGetRange(results, &tr, keys);
}
@ -7289,7 +7302,9 @@ ACTOR Future<Void> fetchKeys(StorageServer* data, AddingShard* shard) {
} catch (Error& e) {
if (e.code() != error_code_end_of_stream && e.code() != error_code_connection_failed &&
e.code() != error_code_transaction_too_old && e.code() != error_code_future_version &&
e.code() != error_code_process_behind && e.code() != error_code_server_overloaded) {
e.code() != error_code_process_behind && e.code() != error_code_server_overloaded &&
e.code() != error_code_blob_granule_request_failed &&
e.code() != error_code_blob_granule_transaction_too_old) {
throw;
}
lastError = e;

View File

@ -35,6 +35,7 @@
#include "flow/ActorCollection.h"
#include "flow/Error.h"
#include "flow/FileIdentifier.h"
#include "flow/Knobs.h"
#include "flow/ObjectSerializer.h"
#include "flow/Platform.h"
#include "flow/ProtocolVersion.h"
@ -1045,6 +1046,9 @@ ACTOR Future<Void> healthMonitor(Reference<AsyncVar<Optional<ClusterControllerFu
}
bool degradedPeer = false;
bool disconnectedPeer = false;
// If peer->lastLoggedTime == 0, we just started monitor this peer and haven't logged it once yet.
double lastLoggedTime = peer->lastLoggedTime <= 0.0 ? peer->lastConnectTime : peer->lastLoggedTime;
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
@ -1062,7 +1066,7 @@ ACTOR Future<Void> healthMonitor(Reference<AsyncVar<Optional<ClusterControllerFu
if (disconnectedPeer || degradedPeer) {
TraceEvent("HealthMonitorDetectDegradedPeer")
.detail("Peer", address)
.detail("Elapsed", now() - peer->lastLoggedTime)
.detail("Elapsed", now() - lastLoggedTime)
.detail("Disconnected", disconnectedPeer)
.detail("MinLatency", peer->pingLatencies.min())
.detail("MaxLatency", peer->pingLatencies.max())
@ -1093,7 +1097,7 @@ ACTOR Future<Void> healthMonitor(Reference<AsyncVar<Optional<ClusterControllerFu
TraceEvent("HealthMonitorDetectDegradedPeer")
.detail("Peer", address)
.detail("Satellite", true)
.detail("Elapsed", now() - peer->lastLoggedTime)
.detail("Elapsed", now() - lastLoggedTime)
.detail("Disconnected", disconnectedPeer)
.detail("MinLatency", peer->pingLatencies.min())
.detail("MaxLatency", peer->pingLatencies.max())
@ -1866,8 +1870,15 @@ ACTOR Future<Void> workerServer(Reference<IClusterConnectionRecord> connRecord,
if (s.storedComponent == DiskStore::Storage) {
LocalLineage _;
getCurrentLineage()->modify(&RoleLineage::role) = ProcessClass::ClusterRole::Storage;
Reference<IPageEncryptionKeyProvider> encryptionKeyProvider =
makeReference<TenantAwareEncryptionKeyProvider>(dbInfo);
Reference<IPageEncryptionKeyProvider> encryptionKeyProvider;
if (FLOW_KNOBS->ENCRYPT_HEADER_AUTH_TOKEN_ENABLED) {
encryptionKeyProvider =
makeReference<TenantAwareEncryptionKeyProvider<EncodingType::AESEncryptionWithAuth>>(dbInfo);
} else {
encryptionKeyProvider =
makeReference<TenantAwareEncryptionKeyProvider<EncodingType::AESEncryption>>(dbInfo);
}
IKeyValueStore* kv = openKVStore(
s.storeType,
s.filename,
@ -2410,7 +2421,7 @@ ACTOR Future<Void> workerServer(Reference<IClusterConnectionRecord> connRecord,
ReplyPromise<InitializeBackupReply> backupReady = req.reply;
backupWorkerCache.set(req.reqId, backupReady.getFuture());
Future<Void> backupProcess = backupWorker(recruited, req, dbInfo);
backupProcess = storageCache.removeOnReady(req.reqId, backupProcess);
backupProcess = backupWorkerCache.removeOnReady(req.reqId, backupProcess);
errorForwarders.add(forwardError(errors, Role::BACKUP, recruited.id(), backupProcess));
TraceEvent("BackupInitRequest", req.reqId).detail("BackupId", recruited.id());
InitializeBackupReply reply(recruited, req.backupEpoch);
@ -2561,8 +2572,15 @@ ACTOR Future<Void> workerServer(Reference<IClusterConnectionRecord> connRecord,
folder,
isTss ? testingStoragePrefix.toString() : fileStoragePrefix.toString(),
recruited.id());
Reference<IPageEncryptionKeyProvider> encryptionKeyProvider =
makeReference<TenantAwareEncryptionKeyProvider>(dbInfo);
Reference<IPageEncryptionKeyProvider> encryptionKeyProvider;
if (FLOW_KNOBS->ENCRYPT_HEADER_AUTH_TOKEN_ENABLED) {
encryptionKeyProvider =
makeReference<TenantAwareEncryptionKeyProvider<EncodingType::AESEncryptionWithAuth>>(
dbInfo);
} else {
encryptionKeyProvider =
makeReference<TenantAwareEncryptionKeyProvider<EncodingType::AESEncryption>>(dbInfo);
}
IKeyValueStore* data = openKVStore(
req.storeType,
filename,

View File

@ -0,0 +1,157 @@
/*
* BlobRestoreWorkload.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "fdbclient/BlobGranuleCommon.h"
#include "fdbclient/ClientBooleanParams.h"
#include "fdbclient/ClientKnobs.h"
#include "fdbclient/BackupAgent.actor.h"
#include "fdbclient/BackupContainer.h"
#include "fdbclient/BackupContainerFileSystem.h"
#include "fdbclient/Knobs.h"
#include "fdbclient/SystemData.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "fdbserver/BlobGranuleServerCommon.actor.h"
#include "flow/Error.h"
#include "flow/actorcompiler.h" // This must be the last #include.
// This worload provides building blocks to test blob restore. The following 2 functions are offered:
// 1) SetupBlob - blobbify key ranges so that we could backup fdb to a blob storage
// 2) PerformRestore - Start blob restore to the extra db instance and wait until it finishes
//
// A general flow to test blob restore:
// 1) start two db instances and blobbify normalKeys for the default db
// 2) submit mutation log only backup to the default db with IncrementalBackup
// 3) start cycle workload to write data to the default db
// 4) perform blob restore to the extra db
// 5) verify data in the extra db
//
// Please refer to BlobRestoreBasic.toml to see how to run a blob restore test with the help from IncrementalBackup
// and Cycle.
//
struct BlobRestoreWorkload : TestWorkload {
static constexpr auto NAME = "BlobRestoreWorkload";
BlobRestoreWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) {
ASSERT(g_simulator->extraDatabases.size() == 1); // extra db must be enabled
extraDb_ = Database::createSimulatedExtraDatabase(g_simulator->extraDatabases[0]);
setupBlob_ = getOption(options, "setupBlob"_sr, false);
performRestore_ = getOption(options, "performRestore"_sr, false);
}
Future<Void> setup(Database const& cx) override { return Void(); }
Future<Void> start(Database const& cx) override {
if (clientId != 0)
return Void();
return _start(cx, this);
}
ACTOR static Future<Void> _start(Database cx, BlobRestoreWorkload* self) {
state bool result = false;
if (self->setupBlob_) {
fmt::print("Blobbify normal range\n");
wait(store(result, cx->blobbifyRange(normalKeys)));
}
if (self->performRestore_) {
fmt::print("Perform blob restore\n");
wait(store(result, self->extraDb_->blobRestore(normalKeys)));
state std::vector<Future<Void>> futures;
futures.push_back(self->runBackupAgent(self));
futures.push_back(self->monitorProgress(cx, self));
wait(waitForAny(futures));
}
return Void();
}
// Start backup agent on the extra db
ACTOR Future<Void> runBackupAgent(BlobRestoreWorkload* self) {
state FileBackupAgent backupAgent;
state Future<Void> future = backupAgent.run(
self->extraDb_, 1.0 / CLIENT_KNOBS->BACKUP_AGGREGATE_POLL_RATE, CLIENT_KNOBS->SIM_BACKUP_TASKS_PER_AGENT);
wait(Future<Void>(Never()));
throw internal_error();
}
// Monitor restore progress and copy data back to original db after successful restore
ACTOR Future<Void> monitorProgress(Database cx, BlobRestoreWorkload* self) {
loop {
Optional<BlobRestoreStatus> status = wait(getRestoreStatus(self->extraDb_, normalKeys));
if (status.present()) {
state BlobRestoreStatus s = status.get();
if (s.phase == BlobRestorePhase::DONE) {
wait(copyToOriginalDb(cx, self));
return Void();
}
// TODO need to define more specific error handling
if (s.phase == BlobRestorePhase::ERROR) {
fmt::print("Unexpected restore error code = {}\n", s.status);
return Void();
}
}
wait(delay(1));
}
}
//
ACTOR static Future<Void> copyToOriginalDb(Database cx, BlobRestoreWorkload* self) {
state RangeResult data;
// Read data from restored db
state Transaction tr1(self->extraDb_->clone());
loop {
try {
RangeResult result = wait(tr1.getRange(normalKeys, CLIENT_KNOBS->TOO_MANY));
data = result;
break;
} catch (Error& e) {
wait(tr1.onError(e));
}
}
// Write back to original db for Cycle worker load to verify
state Transaction tr2(cx);
loop {
try {
tr2.clear(normalKeys);
for (auto kv : data) {
tr2.set(kv.key, kv.value);
}
wait(tr2.commit());
fmt::print("Copied {} rows to origin db\n", data.size());
return Void();
} catch (Error& e) {
wait(tr2.onError(e));
}
}
}
Future<bool> check(Database const& cx) override { return true; }
void getMetrics(std::vector<PerfMetric>& m) override {}
void disableFailureInjectionWorkloads(std::set<std::string>& out) const override { out.emplace("Attrition"); }
private:
Database extraDb_;
bool setupBlob_;
bool performRestore_;
};
WorkloadFactory<BlobRestoreWorkload> BlobRestoreWorkloadFactory;

View File

@ -121,12 +121,12 @@ struct IncrementalBackupWorkload : TestWorkload {
.detail("ContiguousLogEndVersion",
desc.contiguousLogEnd.present() ? desc.contiguousLogEnd.get() : invalidVersion)
.detail("TargetVersion", v);
if (self->waitRetries != -1 && tries > self->waitRetries)
break;
if (!desc.contiguousLogEnd.present())
continue;
if (desc.contiguousLogEnd.get() >= v)
break;
if (self->waitRetries != -1 && tries > self->waitRetries)
break;
// Avoid spamming requests with a delay
wait(delay(5.0));
}

View File

@ -208,7 +208,8 @@ struct PerpetualWiggleStatsWorkload : public TestWorkload {
makeReference<AsyncVar<bool>>(false),
PromiseStream<GetMetricsRequest>(),
Promise<UID>(),
PromiseStream<Promise<int>>() });
PromiseStream<Promise<int>>(),
PromiseStream<Promise<int64_t>>() });
tester.configuration.storageTeamSize = 3;
tester.configuration.perpetualStorageWiggleSpeed = 1;

View File

@ -45,6 +45,7 @@ void forceLinkCompressionUtilsTest();
void forceLinkAtomicTests();
void forceLinkIdempotencyIdTests();
void forceLinkBlobConnectionProviderTests();
void forceLinkArenaStringTests();
void forceLinkActorCollectionTests();
void forceLinkDDSketchTests();
@ -108,6 +109,7 @@ struct UnitTestWorkload : TestWorkload {
forceLinkAtomicTests();
forceLinkIdempotencyIdTests();
forceLinkBlobConnectionProviderTests();
forceLinkArenaStringTests();
forceLinkActorCollectionTests();
forceLinkDDSketchTests();
}

49
flow/ArenaString.cpp Normal file
View File

@ -0,0 +1,49 @@
#include "flow/UnitTest.h"
#include "flow/ArenaAllocator.h"
#include "flow/ArenaString.h"
TEST_CASE("/flow/ArenaString") {
Arena arena;
ArenaAllocator<char> alloc(arena);
{
ArenaString s("1", alloc);
auto shortStrBuf = s.data();
s.assign(100, '1');
auto longStrBuf = s.data();
ASSERT_NE(shortStrBuf, longStrBuf);
ArenaString t = s;
auto copiedStrBuf = t.data();
ASSERT_NE(copiedStrBuf, longStrBuf);
}
{
ArenaString s(alloc);
s.assign(100, 'a');
ArenaString t(100, 'a', alloc);
ASSERT(s == t);
}
{
// Default construction of string does not specify an allocator, and Arena by extension.
// Any modification that requires allocation will throw bad_allocator() when assigning beyond
// short-string-optimized length.
ArenaString s;
bool hit = false;
try {
s.assign(100, 'a');
} catch (Error& e) {
hit = true;
ASSERT_EQ(e.code(), error_code_bad_allocator);
}
ASSERT(hit);
}
{
// string_view may be used to bridge strings with different allocators
ArenaString s(100, 'a', alloc);
std::string_view sv(s);
std::string s2(sv);
std::string_view sv2(s2);
ASSERT(sv == sv2);
}
return Void();
}
void forceLinkArenaStringTests() {}

View File

@ -311,8 +311,8 @@ void FlowKnobs::initialize(Randomize randomize, IsSimulated isSimulated) {
init( ENCRYPT_KEY_CACHE_LOGGING_INTERVAL, 5.0 );
init( ENCRYPT_KEY_CACHE_LOGGING_SKETCH_ACCURACY, 0.01 );
// Refer to EncryptUtil::EncryptAuthTokenAlgo for more details
init( ENCRYPT_HEADER_AUTH_TOKEN_ENABLED, true ); if ( randomize && BUGGIFY ) { ENCRYPT_HEADER_AUTH_TOKEN_ENABLED = !ENCRYPT_HEADER_AUTH_TOKEN_ENABLED; }
init( ENCRYPT_HEADER_AUTH_TOKEN_ALGO, 1 ); if ( randomize && BUGGIFY ) { ENCRYPT_HEADER_AUTH_TOKEN_ALGO = getRandomAuthTokenAlgo(); }
init( ENCRYPT_HEADER_AUTH_TOKEN_ENABLED, false ); if ( randomize && BUGGIFY ) { ENCRYPT_HEADER_AUTH_TOKEN_ENABLED = !ENCRYPT_HEADER_AUTH_TOKEN_ENABLED; }
init( ENCRYPT_HEADER_AUTH_TOKEN_ALGO, 0 ); if ( randomize && ENCRYPT_HEADER_AUTH_TOKEN_ENABLED ) { ENCRYPT_HEADER_AUTH_TOKEN_ALGO = getRandomAuthTokenAlgo(); }
// REST Client

View File

@ -448,6 +448,27 @@ void bindDeterministicRandomToOpenssl() {
#endif // OPENSSL_IS_BORINGSSL
}
int nChooseK(int n, int k) {
assert(n >= k && k >= 0);
if (k == 0) {
return 1;
}
if (k > n / 2) {
return nChooseK(n, n - k);
}
long ret = 1;
// To avoid integer overflow, we do n/1 * (n-1)/2 * (n-2)/3 * (n-i+1)/i, where i = k
for (int i = 1; i <= k; ++i) {
ret *= n - i + 1;
ret /= i;
}
ASSERT(ret <= INT_MAX);
return ret;
}
namespace {
// Simple message for flatbuffers unittests
struct Int {

View File

@ -0,0 +1,90 @@
/*
* ArenaAllocator.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef FLOW_ARENA_ALLOCATOR_H
#define FLOW_ARENA_ALLOCATOR_H
#pragma once
#include "flow/Arena.h"
#include "flow/Error.h"
#include "flow/FastRef.h"
#include <functional>
#include <type_traits>
#include <variant>
template <class T>
class ArenaAllocator {
Arena* arenaPtr;
Arena& arena() noexcept { return *arenaPtr; }
public:
using pointer = T*;
using const_pointer = const T*;
using reference = T&;
using const_reference = const T&;
using void_pointer = void*;
using const_void_pointer = const void*;
using self_type = ArenaAllocator<T>;
using size_type = size_t;
using value_type = T;
using difference_type = typename std::pointer_traits<pointer>::difference_type;
// Unfortunately this needs to exist due to STL's internal use of Allocator() in internal coding
ArenaAllocator() noexcept : arenaPtr(nullptr) {}
ArenaAllocator(Arena& arena) noexcept : arenaPtr(&arena) {}
ArenaAllocator(const self_type& other) noexcept = default;
// Rebind constructor does not modify
template <class U>
ArenaAllocator(const ArenaAllocator<U>& other) noexcept : arenaPtr(other.arenaPtr) {}
ArenaAllocator& operator=(const self_type& other) noexcept = default;
ArenaAllocator(self_type&& other) noexcept = default;
ArenaAllocator& operator=(self_type&& other) noexcept = default;
T* allocate(size_t n) {
if (!arenaPtr)
throw bad_allocator();
return new (arena()) T[n];
}
void deallocate(T*, size_t) noexcept {}
bool operator==(const self_type& other) const noexcept { return arenaPtr == other.arenaPtr; }
bool operator!=(const self_type& other) const noexcept { return !(*this == other); }
template <class U>
struct rebind {
using other = ArenaAllocator<U>;
};
using is_always_equal = std::false_type;
using propagate_on_container_copy_assignment = std::true_type;
using propagate_on_container_move_assignment = std::true_type;
using propagate_on_container_swap = std::true_type;
};
#endif /*FLOW_ARENA_ALLOCATOR_H*/

View File

@ -0,0 +1,30 @@
/*
* ArenaString.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef FLOW_ARENA_STRING_H
#define FLOW_ARENA_STRING_H
#pragma once
#include "flow/ArenaAllocator.h"
#include "flow/CustomAllocatorString.h"
using ArenaString = CustomAllocatorString<ArenaAllocator>;
#endif /*FLOW_ARENA_STRING_H*/

View File

@ -0,0 +1,30 @@
/*
* CustomAllocatorString.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef FLOW_CUSTOM_ALLOCATOR_STRING_H
#define FLOW_CUSTOM_ALLOCATOR_STRING_H
#pragma once
#include <string>
template <template <class> class Allocator>
using CustomAllocatorString = std::basic_string<char, std::char_traits<char>, Allocator<char>>;
#endif /*FLOW_CUSTOM_ALLOCATOR_STRING_H*/

View File

@ -108,6 +108,7 @@ ERROR( operation_cancelled, 1101, "Asynchronous operation cancelled" )
ERROR( future_released, 1102, "Future has been released" )
ERROR( connection_leaked, 1103, "Connection object leaked" )
ERROR( never_reply, 1104, "Never reply to the request" )
ERROR( retry, 1105, "Retry operation" )
ERROR( recruitment_failed, 1200, "Recruitment of a server failed" ) // Be careful, catching this will delete the data of a storage server or tlog permanently
ERROR( move_to_removed_server, 1201, "Attempt to move keys to a storage server that was removed" )
@ -140,6 +141,7 @@ ERROR( storage_quota_exceeded, 1225, "Exceeded the maximum storage quota allocat
ERROR( platform_error, 1500, "Platform error" )
ERROR( large_alloc_failed, 1501, "Large block allocation failed" )
ERROR( performance_counter_error, 1502, "QueryPerformanceCounter error" )
ERROR( bad_allocator, 1503, "Null allocator was used to allocate memory" )
ERROR( io_error, 1510, "Disk i/o operation failed" )
ERROR( file_not_found, 1511, "File not found" )

View File

@ -100,6 +100,9 @@ extern StringRef strinc(StringRef const& str, Arena& arena);
extern Standalone<StringRef> addVersionStampAtEnd(StringRef const& str);
extern StringRef addVersionStampAtEnd(StringRef const& str, Arena& arena);
// Return the number of combinations to choose k items out of n choices
int nChooseK(int n, int k);
template <typename Iter>
StringRef concatenate(Iter b, Iter const& e, Arena& arena) {
int rsize = 0;

View File

@ -344,10 +344,10 @@ Future<Void> storeOrThrow(T& out, Future<Optional<T>> what, Error e = key_not_fo
}
// Waits for a future to be ready, and then applies an asynchronous function to it.
ACTOR template <class T, class F, class U = decltype(std::declval<F>()(std::declval<T>()).getValue())>
Future<U> mapAsync(Future<T> what, F actorFunc) {
ACTOR template <class T, class F>
Future<decltype(std::declval<F>()(std::declval<T>()).getValue())> mapAsync(Future<T> what, F actorFunc) {
T val = wait(what);
U ret = wait(actorFunc(val));
decltype(std::declval<F>()(std::declval<T>()).getValue()) ret = wait(actorFunc(val));
return ret;
}

View File

@ -289,11 +289,9 @@ Future<Reference<IConnection>> INetworkConnections::connect(const std::string& h
// Wait for the endpoint to return, then wait for connect(endpoint) and return it.
// Template types are being provided explicitly because they can't be automatically deduced for some reason.
return mapAsync<NetworkAddress,
std::function<Future<Reference<IConnection>>(NetworkAddress const&)>,
Reference<IConnection>>(
pickEndpoint,
[=](NetworkAddress const& addr) -> Future<Reference<IConnection>> { return connectExternal(addr); });
return mapAsync(pickEndpoint, [=](NetworkAddress const& addr) -> Future<Reference<IConnection>> {
return connectExternal(addr);
});
}
IUDPSocket::~IUDPSocket() {}

View File

@ -141,6 +141,7 @@ if(WITH_PYTHON)
add_fdb_test(TEST_FILES fast/BlobGranuleVerifySmallClean.toml)
# TODO: test occasionally times out due to too many change feed shard parts
add_fdb_test(TEST_FILES fast/BlobGranuleMoveVerifyCycle.toml IGNORE)
add_fdb_test(TEST_FILES fast/BlobRestoreBasic.toml)
add_fdb_test(TEST_FILES fast/CacheTest.toml)
add_fdb_test(TEST_FILES fast/CloggedSideband.toml)
add_fdb_test(TEST_FILES fast/CompressionUtilsUnit.toml)
@ -438,7 +439,7 @@ if(WITH_PYTHON)
string(APPEND authz_venv_cmd "${Python3_EXECUTABLE} -m venv ${authz_venv_dir} ")
string(APPEND authz_venv_cmd "&& ${authz_venv_activate} ")
string(APPEND authz_venv_cmd "&& pip install --upgrade pip ")
string(APPEND authz_venv_cmd "&& pip install --upgrade -r ${CMAKE_SOURCE_DIR}/tests/authorization/requirements.txt ")
string(APPEND authz_venv_cmd "&& pip install -r ${CMAKE_SOURCE_DIR}/tests/authorization/requirements.txt ")
string(APPEND authz_venv_cmd "&& (cd ${CMAKE_BINARY_DIR}/bindings/python && python3 setup.py install) ")
string(APPEND authz_venv_cmd "&& touch ${authz_venv_stamp_file} ")
string(APPEND authz_venv_cmd "|| echo 'venv already set up'")

View File

@ -0,0 +1,31 @@
from authlib.jose import JsonWebKey, KeySet, jwt
from typing import List
import json
def private_key_gen(kty: str, kid: str):
assert kty == "EC" or kty == "RSA"
if kty == "EC":
return JsonWebKey.generate_key(kty=kty, crv_or_size="P-256", is_private=True, options={"kid": kid})
else:
return JsonWebKey.generate_key(kty=kty, crv_or_size=4096, is_private=True, options={"kid": kid})
def public_keyset_from_keys(keys: List):
keys = list(map(lambda key: key.as_dict(is_private=False, alg=alg_from_kty(key.kty)), keys))
return json.dumps({ "keys": keys })
def alg_from_kty(kty: str):
assert kty == "EC" or kty == "RSA"
if kty == "EC":
return "ES256"
else:
return "RS256"
def token_gen(private_key, claims, headers={}):
if not headers:
headers = {
"typ": "JWT",
"kty": private_key.kty,
"alg": alg_from_kty(private_key.kty),
"kid": private_key.kid,
}
return jwt.encode(headers, claims, private_key)

View File

@ -9,7 +9,7 @@ from urllib import request
import hashlib
from fdb_version import CURRENT_VERSION, FUTURE_VERSION
from local_cluster import random_secret_string
from test_util import random_alphanum_string
SUPPORTED_PLATFORMS = ["x86_64", "aarch64"]
FDB_DOWNLOAD_ROOT = "https://github.com/apple/foundationdb/releases/download/"
@ -85,7 +85,7 @@ class FdbBinaryDownloader:
# Download to a temporary file and then replace the target file atomically
# to avoid consistency errors in case of multiple tests are downloading the
# same file in parallel
local_file_tmp = Path("{}.{}".format(str(local_file), random_secret_string(8)))
local_file_tmp = Path("{}.{}".format(str(local_file), random_alphanum_string(8)))
self.download_dir.joinpath(version).mkdir(parents=True, exist_ok=True)
remote_file = "{}{}/{}".format(FDB_DOWNLOAD_ROOT, version, remote_bin_name)
remote_sha256 = "{}.sha256".format(remote_file)
@ -127,7 +127,7 @@ class FdbBinaryDownloader:
return
# Avoid race conditions in case of parallel test execution by first copying to a temporary file
# and then renaming it atomically
dest_file_tmp = Path("{}.{}".format(str(dest_lib_file), random_secret_string(8)))
dest_file_tmp = Path("{}.{}".format(str(dest_lib_file), random_alphanum_string(8)))
src_lib_file = self.local_binary_repo.joinpath(version, "lib", "libfdb_c-{}.so".format(version))
assert src_lib_file.exists(), "Missing file {} in the local old binaries repository".format(src_lib_file)
self.download_dir.joinpath(version).mkdir(parents=True, exist_ok=True)

View File

@ -4,7 +4,7 @@ import os
import shutil
import subprocess
import sys
from local_cluster import random_secret_string
from test_util import random_alphanum_string
from argparse import ArgumentParser, RawDescriptionHelpFormatter
from pathlib import Path
@ -14,7 +14,7 @@ class ClusterFileGenerator:
self.output_dir = Path(output_dir).resolve()
assert self.output_dir.exists(), "{} does not exist".format(output_dir)
assert self.output_dir.is_dir(), "{} is not a directory".format(output_dir)
self.tmp_dir = self.output_dir.joinpath("tmp", random_secret_string(16))
self.tmp_dir = self.output_dir.joinpath("tmp", random_alphanum_string(16))
self.tmp_dir.mkdir(parents=True)
self.cluster_file_path = self.tmp_dir.joinpath("fdb.cluster")

View File

@ -9,6 +9,8 @@ import time
import fcntl
import sys
import tempfile
from authz_util import private_key_gen, public_keyset_from_keys
from test_util import random_alphanum_string
CLUSTER_UPDATE_TIMEOUT_SEC = 10
EXCLUDE_SERVERS_TIMEOUT_SEC = 120
@ -61,10 +63,6 @@ class PortProvider:
pass
self._lock_files.clear()
valid_letters_for_secret = string.ascii_letters + string.digits
class TLSConfig:
# Passing a negative chain length generates expired leaf certificate
def __init__(
@ -77,11 +75,6 @@ class TLSConfig:
self.client_chain_len = client_chain_len
self.verify_peers = verify_peers
def random_secret_string(length):
return "".join(random.choice(valid_letters_for_secret) for _ in range(length))
class LocalCluster:
configuration_template = """
## foundationdb.conf
@ -145,7 +138,8 @@ logdir = {logdir}
tls_config: TLSConfig = None,
mkcert_binary: str = "",
custom_config: dict = {},
public_key_json_str: str = "",
authorization_kty: str = "",
authorization_keypair_id: str = "",
):
self.port_provider = PortProvider()
self.basedir = Path(basedir)
@ -179,8 +173,8 @@ logdir = {logdir}
self.server_ports = {server_id: self.__next_port() for server_id in range(self.process_number)}
self.server_by_port = {port: server_id for server_id, port in self.server_ports.items()}
self.next_server_id = self.process_number
self.cluster_desc = random_secret_string(8)
self.cluster_secret = random_secret_string(8)
self.cluster_desc = random_alphanum_string(8)
self.cluster_secret = random_alphanum_string(8)
self.env_vars = {}
self.running = False
self.process = None
@ -189,7 +183,12 @@ logdir = {logdir}
self.coordinators = set()
self.active_servers = set(self.server_ports.keys())
self.tls_config = tls_config
self.public_key_jwks_str = None
self.public_key_json_file = None
self.private_key = None
self.authorization_private_key_pem_file = None
self.authorization_keypair_id = authorization_keypair_id
self.authorization_kty = authorization_kty
self.mkcert_binary = Path(mkcert_binary)
self.server_cert_file = self.cert.joinpath("server_cert.pem")
self.client_cert_file = self.cert.joinpath("client_cert.pem")
@ -198,10 +197,17 @@ logdir = {logdir}
self.server_ca_file = self.cert.joinpath("server_ca.pem")
self.client_ca_file = self.cert.joinpath("client_ca.pem")
if public_key_json_str:
if self.authorization_kty:
assert self.authorization_keypair_id, "keypair ID must be set to enable authorization"
self.public_key_json_file = self.etc.joinpath("public_keys.json")
self.private_key = private_key_gen(
kty=self.authorization_kty, kid=self.authorization_keypair_id)
self.public_key_jwks_str = public_keyset_from_keys([self.private_key])
with open(self.public_key_json_file, "w") as pubkeyfile:
pubkeyfile.write(public_key_json_str)
pubkeyfile.write(self.public_key_jwks_str)
self.authorization_private_key_pem_file = self.etc.joinpath("authorization_private_key.pem")
with open(self.authorization_private_key_pem_file, "w") as privkeyfile:
privkeyfile.write(self.private_key.as_pem(is_private=True).decode("utf8"))
if create_config:
self.create_cluster_file()

View File

@ -0,0 +1,5 @@
Authlib==1.1.0
cffi==1.15.1
cryptography==38.0.3
pycparser==2.21
toml==0.10.2

View File

@ -0,0 +1,7 @@
import random
import string
alphanum_letters = string.ascii_letters + string.digits
def random_alphanum_string(length):
return "".join(random.choice(alphanum_letters) for _ in range(length))

View File

@ -5,7 +5,8 @@ import os
import shutil
import subprocess
import sys
from local_cluster import LocalCluster, TLSConfig, random_secret_string
from local_cluster import LocalCluster, TLSConfig
from test_util import random_alphanum_string
from argparse import ArgumentParser, RawDescriptionHelpFormatter
from pathlib import Path
@ -18,7 +19,8 @@ class TempCluster(LocalCluster):
port: str = None,
blob_granules_enabled: bool = False,
tls_config: TLSConfig = None,
public_key_json_str: str = None,
authorization_kty: str = "",
authorization_keypair_id: str = "",
remove_at_exit: bool = True,
custom_config: dict = {},
enable_tenants: bool = True,
@ -26,7 +28,7 @@ class TempCluster(LocalCluster):
self.build_dir = Path(build_dir).resolve()
assert self.build_dir.exists(), "{} does not exist".format(build_dir)
assert self.build_dir.is_dir(), "{} is not a directory".format(build_dir)
tmp_dir = self.build_dir.joinpath("tmp", random_secret_string(16))
tmp_dir = self.build_dir.joinpath("tmp", random_alphanum_string(16))
tmp_dir.mkdir(parents=True)
self.tmp_dir = tmp_dir
self.remove_at_exit = remove_at_exit
@ -41,7 +43,8 @@ class TempCluster(LocalCluster):
blob_granules_enabled=blob_granules_enabled,
tls_config=tls_config,
mkcert_binary=self.build_dir.joinpath("bin", "mkcert"),
public_key_json_str=public_key_json_str,
authorization_kty=authorization_kty,
authorization_keypair_id=authorization_keypair_id,
custom_config=custom_config,
)
@ -133,6 +136,25 @@ if __name__ == "__main__":
type=str,
default="Check.Valid=1",
)
parser.add_argument(
"--authorization-kty",
help="Public/Private key pair type to be used in signing and verifying authorization tokens. Must be either unset (empty string), EC or RSA. Unset argument (default) disables authorization",
type=str,
choices=["", "EC", "RSA"],
default="",
)
parser.add_argument(
"--authorization-keypair-id",
help="Name of the public/private key pair to be used in signing and verifying authorization tokens. Setting this argument takes effect only with authorization enabled.",
type=str,
default="",
)
parser.add_argument(
"--no-remove-at-exit",
help="whether to remove the cluster directory upon exit",
action="store_true",
default=False,
)
args = parser.parse_args()
if args.disable_tenants:
@ -151,43 +173,37 @@ if __name__ == "__main__":
blob_granules_enabled=args.blob_granules_enabled,
tls_config=tls_config,
enable_tenants=enable_tenants,
authorization_kty=args.authorization_kty,
authorization_keypair_id=args.authorization_keypair_id,
remove_at_exit=not args.no_remove_at_exit,
) as cluster:
print("log-dir: {}".format(cluster.log))
print("etc-dir: {}".format(cluster.etc))
print("data-dir: {}".format(cluster.data))
print("cluster-file: {}".format(cluster.cluster_file))
cmd_args = []
substitution_table = [
("@CLUSTER_FILE@", str(cluster.cluster_file)),
("@DATA_DIR@", str(cluster.data)),
("@LOG_DIR@", str(cluster.log)),
("@ETC_DIR@", str(cluster.etc)),
("@TMP_DIR@", str(cluster.tmp_dir)),
("@SERVER_CERT_FILE@", str(cluster.server_cert_file)),
("@SERVER_KEY_FILE@", str(cluster.server_key_file)),
("@SERVER_CA_FILE@", str(cluster.server_ca_file)),
("@CLIENT_CERT_FILE@", str(cluster.client_cert_file)),
("@CLIENT_KEY_FILE@", str(cluster.client_key_file)),
("@CLIENT_CA_FILE@", str(cluster.client_ca_file))]
for cmd in args.cmd:
if cmd == "@CLUSTER_FILE@":
cmd_args.append(str(cluster.cluster_file))
elif cmd == "@DATA_DIR@":
cmd_args.append(str(cluster.data))
elif cmd == "@LOG_DIR@":
cmd_args.append(str(cluster.log))
elif cmd == "@ETC_DIR@":
cmd_args.append(str(cluster.etc))
elif cmd == "@TMP_DIR@":
cmd_args.append(str(cluster.tmp_dir))
elif cmd == "@SERVER_CERT_FILE@":
cmd_args.append(str(cluster.server_cert_file))
elif cmd == "@SERVER_KEY_FILE@":
cmd_args.append(str(cluster.server_key_file))
elif cmd == "@SERVER_CA_FILE@":
cmd_args.append(str(cluster.server_ca_file))
elif cmd == "@CLIENT_CERT_FILE@":
cmd_args.append(str(cluster.client_cert_file))
elif cmd == "@CLIENT_KEY_FILE@":
cmd_args.append(str(cluster.client_key_file))
elif cmd == "@CLIENT_CA_FILE@":
cmd_args.append(str(cluster.client_ca_file))
elif cmd.startswith("@DATA_DIR@"):
cmd_args.append(str(cluster.data) + cmd[len("@DATA_DIR@") :])
else:
cmd_args.append(cmd)
for (placeholder, value) in substitution_table:
cmd = cmd.replace(placeholder, value)
cmd_args.append(cmd)
env = dict(**os.environ)
env["FDB_CLUSTER_FILE"] = env.get(
"FDB_CLUSTER_FILE", cluster.cluster_file
)
print("command: {}".format(cmd_args))
errcode = subprocess.run(
cmd_args, stdout=sys.stdout, stderr=sys.stderr, env=env
).returncode

View File

@ -13,7 +13,8 @@ import traceback
import time
from binary_download import FdbBinaryDownloader
from fdb_version import CURRENT_VERSION, FUTURE_VERSION
from local_cluster import LocalCluster, random_secret_string
from local_cluster import LocalCluster
from test_util import random_alphanum_string
TENANT_API_VERSION = 720
@ -56,7 +57,7 @@ class UpgradeTest:
assert self.tester_bin.exists(), "{} does not exist".format(self.tester_bin)
self.upgrade_path = args.upgrade_path
self.used_versions = set(self.upgrade_path).difference(set(CLUSTER_ACTIONS))
self.tmp_dir = self.build_dir.joinpath("tmp", random_secret_string(16))
self.tmp_dir = self.build_dir.joinpath("tmp", random_alphanum_string(16))
self.tmp_dir.mkdir(parents=True)
self.downloader = FdbBinaryDownloader(args.build_dir)
self.download_old_binaries()
@ -81,8 +82,8 @@ class UpgradeTest:
self.log = self.cluster.log
self.etc = self.cluster.etc
self.data = self.cluster.data
self.input_pipe_path = self.tmp_dir.joinpath("input.{}".format(random_secret_string(8)))
self.output_pipe_path = self.tmp_dir.joinpath("output.{}".format(random_secret_string(8)))
self.input_pipe_path = self.tmp_dir.joinpath("input.{}".format(random_alphanum_string(8)))
self.output_pipe_path = self.tmp_dir.joinpath("output.{}".format(random_alphanum_string(8)))
os.mkfifo(self.input_pipe_path)
os.mkfifo(self.output_pipe_path)
self.progress_event = Event()

View File

@ -29,7 +29,8 @@ import sys
import time
from multiprocessing import Process, Pipe
from typing import Union
from util import alg_from_kty, public_keyset_from_keys, random_alphanum_str, random_alphanum_bytes, to_str, to_bytes, KeyFileReverter, token_claim_1h, wait_until_tenant_tr_succeeds, wait_until_tenant_tr_fails
from authz_util import token_gen, private_key_gen, public_keyset_from_keys, alg_from_kty
from util import random_alphanum_str, random_alphanum_bytes, to_str, to_bytes, KeyFileReverter, token_claim_1h, wait_until_tenant_tr_succeeds, wait_until_tenant_tr_fails
special_key_ranges = [
("transaction description", b"/description", b"/description\x00"),
@ -42,8 +43,8 @@ special_key_ranges = [
("kill storage", b"/globals/killStorage", b"/globals/killStorage\x00"),
]
def test_simple_tenant_access(private_key, token_gen, default_tenant, tenant_tr_gen):
token = token_gen(private_key, token_claim_1h(default_tenant))
def test_simple_tenant_access(cluster, default_tenant, tenant_tr_gen):
token = token_gen(cluster.private_key, token_claim_1h(default_tenant))
tr = tenant_tr_gen(default_tenant)
tr.options.set_authorization_token(token)
tr[b"abc"] = b"def"
@ -52,16 +53,16 @@ def test_simple_tenant_access(private_key, token_gen, default_tenant, tenant_tr_
tr.options.set_authorization_token(token)
assert tr[b"abc"] == b"def", "tenant write transaction not visible"
def test_cross_tenant_access_disallowed(private_key, default_tenant, token_gen, tenant_gen, tenant_tr_gen):
def test_cross_tenant_access_disallowed(cluster, default_tenant, tenant_gen, tenant_tr_gen):
# use default tenant token with second tenant transaction and see it fail
second_tenant = random_alphanum_bytes(12)
tenant_gen(second_tenant)
token_second = token_gen(private_key, token_claim_1h(second_tenant))
token_second = token_gen(cluster.private_key, token_claim_1h(second_tenant))
tr_second = tenant_tr_gen(second_tenant)
tr_second.options.set_authorization_token(token_second)
tr_second[b"abc"] = b"def"
tr_second.commit().wait()
token_default = token_gen(private_key, token_claim_1h(default_tenant))
token_default = token_gen(cluster.private_key, token_claim_1h(default_tenant))
tr_second = tenant_tr_gen(second_tenant)
tr_second.options.set_authorization_token(token_default)
# test that read transaction fails
@ -80,7 +81,7 @@ def test_cross_tenant_access_disallowed(private_key, default_tenant, token_gen,
except fdb.FDBError as e:
assert e.code == 6000, f"expected permission_denied, got {e} instead"
def test_system_and_special_key_range_disallowed(db, tenant_tr_gen, token_gen):
def test_system_and_special_key_range_disallowed(db, tenant_tr_gen):
second_tenant = random_alphanum_bytes(12)
try:
fdb.tenant_management.create_tenant(db, second_tenant)
@ -135,18 +136,18 @@ def test_system_and_special_key_range_disallowed(db, tenant_tr_gen, token_gen):
assert e.code == 6000, f"expected permission_denied, got {e} instead"
def test_public_key_set_rollover(
kty, private_key_gen, private_key, public_key_refresh_interval,
cluster, default_tenant, token_gen, tenant_gen, tenant_tr_gen):
kty, public_key_refresh_interval,
cluster, default_tenant, tenant_gen, tenant_tr_gen):
new_kid = random_alphanum_str(12)
new_kty = "EC" if kty == "RSA" else "RSA"
new_key = private_key_gen(kty=new_kty, kid=new_kid)
token_default = token_gen(private_key, token_claim_1h(default_tenant))
token_default = token_gen(cluster.private_key, token_claim_1h(default_tenant))
second_tenant = random_alphanum_bytes(12)
tenant_gen(second_tenant)
token_second = token_gen(new_key, token_claim_1h(second_tenant))
interim_set = public_keyset_from_keys([new_key, private_key])
interim_set = public_keyset_from_keys([new_key, cluster.private_key])
max_repeat = 10
print(f"interim keyset: {interim_set}")
@ -159,79 +160,76 @@ def test_public_key_set_rollover(
with KeyFileReverter(cluster.public_key_json_file, old_key_json, delay):
with open(cluster.public_key_json_file, "w") as keyfile:
keyfile.write(interim_set)
wait_until_tenant_tr_succeeds(second_tenant, new_key, tenant_tr_gen, token_gen, max_repeat, delay)
wait_until_tenant_tr_succeeds(second_tenant, new_key, tenant_tr_gen, max_repeat, delay)
print("interim key set activated")
final_set = public_keyset_from_keys([new_key])
print(f"final keyset: {final_set}")
with open(cluster.public_key_json_file, "w") as keyfile:
keyfile.write(final_set)
wait_until_tenant_tr_fails(default_tenant, private_key, tenant_tr_gen, token_gen, max_repeat, delay)
wait_until_tenant_tr_fails(default_tenant, cluster.private_key, tenant_tr_gen, max_repeat, delay)
def test_public_key_set_broken_file_tolerance(
private_key, public_key_refresh_interval,
cluster, public_key_jwks_str, default_tenant, token_gen, tenant_tr_gen):
cluster, public_key_refresh_interval, default_tenant, tenant_tr_gen):
delay = public_key_refresh_interval
# retry limit in waiting for keyset file update to propagate to FDB server's internal keyset
max_repeat = 10
with KeyFileReverter(cluster.public_key_json_file, public_key_jwks_str, delay):
with KeyFileReverter(cluster.public_key_json_file, cluster.public_key_jwks_str, delay):
# key file update should take effect even after witnessing broken key file
with open(cluster.public_key_json_file, "w") as keyfile:
keyfile.write(public_key_jwks_str.strip()[:10]) # make the file partial, injecting parse error
keyfile.write(cluster.public_key_jwks_str.strip()[:10]) # make the file partial, injecting parse error
time.sleep(delay * 2)
# should still work; internal key set only clears with a valid, empty key set file
tr_default = tenant_tr_gen(default_tenant)
tr_default.options.set_authorization_token(token_gen(private_key, token_claim_1h(default_tenant)))
tr_default.options.set_authorization_token(token_gen(cluster.private_key, token_claim_1h(default_tenant)))
tr_default[b"abc"] = b"def"
tr_default.commit().wait()
with open(cluster.public_key_json_file, "w") as keyfile:
keyfile.write('{"keys":[]}')
# eventually internal key set will become empty and won't accept any new tokens
wait_until_tenant_tr_fails(default_tenant, private_key, tenant_tr_gen, token_gen, max_repeat, delay)
wait_until_tenant_tr_fails(default_tenant, cluster.private_key, tenant_tr_gen, max_repeat, delay)
def test_public_key_set_deletion_tolerance(
private_key, public_key_refresh_interval,
cluster, public_key_jwks_str, default_tenant, token_gen, tenant_tr_gen):
cluster, public_key_refresh_interval, default_tenant, tenant_tr_gen):
delay = public_key_refresh_interval
# retry limit in waiting for keyset file update to propagate to FDB server's internal keyset
max_repeat = 10
with KeyFileReverter(cluster.public_key_json_file, public_key_jwks_str, delay):
with KeyFileReverter(cluster.public_key_json_file, cluster.public_key_jwks_str, delay):
# key file update should take effect even after witnessing deletion of key file
with open(cluster.public_key_json_file, "w") as keyfile:
keyfile.write('{"keys":[]}')
time.sleep(delay)
wait_until_tenant_tr_fails(default_tenant, private_key, tenant_tr_gen, token_gen, max_repeat, delay)
wait_until_tenant_tr_fails(default_tenant, cluster.private_key, tenant_tr_gen, max_repeat, delay)
os.remove(cluster.public_key_json_file)
time.sleep(delay * 2)
with open(cluster.public_key_json_file, "w") as keyfile:
keyfile.write(public_key_jwks_str)
keyfile.write(cluster.public_key_jwks_str)
# eventually updated key set should take effect and transaction should be accepted
wait_until_tenant_tr_succeeds(default_tenant, private_key, tenant_tr_gen, token_gen, max_repeat, delay)
wait_until_tenant_tr_succeeds(default_tenant, cluster.private_key, tenant_tr_gen, max_repeat, delay)
def test_public_key_set_empty_file_tolerance(
private_key, public_key_refresh_interval,
cluster, public_key_jwks_str, default_tenant, token_gen, tenant_tr_gen):
cluster, public_key_refresh_interval, default_tenant, tenant_tr_gen):
delay = public_key_refresh_interval
# retry limit in waiting for keyset file update to propagate to FDB server's internal keyset
max_repeat = 10
with KeyFileReverter(cluster.public_key_json_file, public_key_jwks_str, delay):
with KeyFileReverter(cluster.public_key_json_file, cluster.public_key_jwks_str, delay):
# key file update should take effect even after witnessing an empty file
with open(cluster.public_key_json_file, "w") as keyfile:
keyfile.write('{"keys":[]}')
# eventually internal key set will become empty and won't accept any new tokens
wait_until_tenant_tr_fails(default_tenant, private_key, tenant_tr_gen, token_gen, max_repeat, delay)
wait_until_tenant_tr_fails(default_tenant, cluster.private_key, tenant_tr_gen, max_repeat, delay)
# empty the key file
with open(cluster.public_key_json_file, "w") as keyfile:
pass
time.sleep(delay * 2)
with open(cluster.public_key_json_file, "w") as keyfile:
keyfile.write(public_key_jwks_str)
keyfile.write(cluster.public_key_jwks_str)
# eventually key file should update and transactions should go through
wait_until_tenant_tr_succeeds(default_tenant, private_key, tenant_tr_gen, token_gen, max_repeat, delay)
wait_until_tenant_tr_succeeds(default_tenant, cluster.private_key, tenant_tr_gen, max_repeat, delay)
def test_bad_token(private_key, token_gen, default_tenant, tenant_tr_gen):
def test_bad_token(cluster, default_tenant, tenant_tr_gen):
def del_attr(d, attr):
del d[attr]
return d
@ -251,14 +249,14 @@ def test_bad_token(private_key, token_gen, default_tenant, tenant_tr_gen):
]
for case_name, mutation in claim_mutations:
tr = tenant_tr_gen(default_tenant)
tr.options.set_authorization_token(token_gen(private_key, mutation(token_claim_1h(default_tenant))))
tr.options.set_authorization_token(token_gen(cluster.private_key, mutation(token_claim_1h(default_tenant))))
try:
value = tr[b"abc"].value
assert False, f"expected permission_denied for case {case_name}, but read transaction went through"
except fdb.FDBError as e:
assert e.code == 6000, f"expected permission_denied for case {case_name}, got {e} instead"
tr = tenant_tr_gen(default_tenant)
tr.options.set_authorization_token(token_gen(private_key, mutation(token_claim_1h(default_tenant))))
tr.options.set_authorization_token(token_gen(cluster.private_key, mutation(token_claim_1h(default_tenant))))
tr[b"abc"] = b"def"
try:
tr.commit().wait()
@ -268,7 +266,7 @@ def test_bad_token(private_key, token_gen, default_tenant, tenant_tr_gen):
# unknown key case: override "kid" field in header
# first, update only the kid field of key with export-update-import
key_dict = private_key.as_dict(is_private=True)
key_dict = cluster.private_key.as_dict(is_private=True)
key_dict["kid"] = random_alphanum_str(10)
renamed_key = authlib.jose.JsonWebKey.import_key(key_dict)
unknown_key_token = token_gen(

View File

@ -22,11 +22,10 @@ import fdb
import pytest
import subprocess
import admin_server
from authlib.jose import JsonWebKey, KeySet, jwt
from local_cluster import TLSConfig
from tmp_cluster import TempCluster
from typing import Union
from util import alg_from_kty, public_keyset_from_keys, random_alphanum_str, random_alphanum_bytes, to_str, to_bytes
from util import random_alphanum_str, random_alphanum_bytes, to_str, to_bytes
fdb.api_version(720)
@ -66,47 +65,10 @@ def trusted_client(request):
def public_key_refresh_interval(request):
return request.config.option.public_key_refresh_interval
@pytest.fixture(scope="session")
def alg(kty):
if kty == "EC":
return "ES256"
else:
return "RS256"
@pytest.fixture(scope="session")
def kid():
return random_alphanum_str(12)
@pytest.fixture(scope="session")
def private_key_gen():
def fn(kty: str, kid: str):
if kty == "EC":
return JsonWebKey.generate_key(kty=kty, crv_or_size="P-256", is_private=True, options={"kid": kid})
else:
return JsonWebKey.generate_key(kty=kty, crv_or_size=4096, is_private=True, options={"kid": kid})
return fn
@pytest.fixture(scope="session")
def private_key(kty, kid, private_key_gen):
return private_key_gen(kty, kid)
@pytest.fixture(scope="session")
def public_key_jwks_str(private_key):
return public_keyset_from_keys([private_key])
@pytest.fixture(scope="session")
def token_gen():
def fn(private_key, claims, headers={}):
if not headers:
headers = {
"typ": "JWT",
"kty": private_key.kty,
"alg": alg_from_kty(private_key.kty),
"kid": private_key.kid,
}
return jwt.encode(headers, claims, private_key)
return fn
@pytest.fixture(scope=cluster_scope)
def admin_ipc():
server = admin_server.Server()
@ -115,11 +77,12 @@ def admin_ipc():
server.join()
@pytest.fixture(autouse=True, scope=cluster_scope)
def cluster(admin_ipc, build_dir, public_key_jwks_str, public_key_refresh_interval, trusted_client):
def cluster(admin_ipc, build_dir, public_key_refresh_interval, trusted_client):
with TempCluster(
build_dir=build_dir,
tls_config=TLSConfig(server_chain_len=3, client_chain_len=2),
public_key_json_str=public_key_jwks_str,
authorization_kty="EC",
authorization_keypair_id="authz-key",
remove_at_exit=True,
custom_config={
"knob-public-key-file-refresh-interval-seconds": public_key_refresh_interval,

Some files were not shown because too many files have changed in this diff Show More