Merge branch 'main' of https://github.com/apple/foundationdb into feature/main/wiggleDelay

This commit is contained in:
Xiaoxi Wang 2022-12-20 17:21:19 -08:00
commit f13453fe63
66 changed files with 2188 additions and 982 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)
@ -137,16 +147,16 @@ if(WIN32)
# I think depending on the cmake version this will cause weird warnings
find_package(Boost 1.72 COMPONENTS filesystem iostreams)
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

@ -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

@ -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

@ -1538,13 +1538,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

@ -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

@ -307,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

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

@ -269,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;

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 @@ 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

@ -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,58 @@ FDB_DEFINE_BOOLEAN_PARAM(InOverSizePhysicalShard);
FDB_DEFINE_BOOLEAN_PARAM(PhysicalShardAvailable);
FDB_DEFINE_BOOLEAN_PARAM(MoveKeyRangeOutPhysicalShard);
// Tracks storage metrics for `keys`. 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) {
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;
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::addRange(const KeyRange& newRange) {
if (g_network->isSimulated()) {
// Test that new range must not overlap with any existing range in this shard.
@ -1552,8 +1613,9 @@ void PhysicalShardCollection::PhysicalShard::addRange(const KeyRange& newRange)
}
}
// TODO(zhewu): add metrics tracking actor.
RangeData data;
data.stats = makeReference<AsyncVar<Optional<ShardMetrics>>>();
data.trackMetrics = trackKeyRangeInPhysicalShardMetrics(txnProcessor, newRange, data.stats);
rangeData.emplace(newRange, data);
}
@ -1569,8 +1631,9 @@ 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;
data.stats = makeReference<AsyncVar<Optional<ShardMetrics>>>();
data.trackMetrics = trackKeyRangeInPhysicalShardMetrics(txnProcessor, r, data.stats);
rangeData.emplace(r, data);
}
// Must erase last since `remainingRanges` uses data in `range`.
@ -1623,7 +1686,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();
@ -3110,6 +3124,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());
@ -3154,13 +3169,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;
}
@ -3172,13 +3187,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())
@ -3293,10 +3312,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()
@ -4067,6 +4091,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();
@ -4234,10 +4259,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;
}
@ -4526,13 +4554,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()) {
@ -4836,6 +4885,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;
}

View File

@ -199,7 +199,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>();

View File

@ -365,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;
}

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

@ -113,7 +113,7 @@ rocksdb::ColumnFamilyOptions getCFOptions() {
rocksdb::Options getOptions() {
rocksdb::Options options({}, getCFOptions());
options.create_if_missing = true;
options.db_log_dir = SERVER_KNOBS->LOG_DIRECTORY;
options.db_log_dir = g_network->isSimulated() ? "" : SERVER_KNOBS->LOG_DIRECTORY;
return options;
}

View File

@ -3124,6 +3124,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 +3507,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

@ -2657,16 +2657,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));
}

View File

@ -279,11 +279,12 @@ 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), teams(teams), whenCreated(whenCreated) {}
// Adds `newRange` to this physical shard and starts monitoring the shard.
void addRange(const KeyRange& newRange);
@ -293,13 +294,14 @@ 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
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.
Future<Void> trackMetrics;
Reference<AsyncVar<Optional<ShardMetrics>>>
stats; // TODO(zhewu): aggregate all metrics to a single physical shard metrics.
};

View File

@ -2410,7 +2410,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);

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

@ -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

@ -140,6 +140,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

@ -438,7 +438,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,

View File

@ -1,10 +1,10 @@
import base64
import fdb
import json
import random
import string
import time
from typing import Union, List
from typing import Union
from authz_util import token_gen
def to_str(s: Union[str, bytes]):
if isinstance(s, bytes):
@ -33,16 +33,6 @@ def cleanup_tenant(db, tenant_name):
else:
raise
def alg_from_kty(kty: str):
if kty == "EC":
return "ES256"
else:
return "RS256"
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 })
class KeyFileReverter(object):
def __init__(self, filename: str, content: str, refresh_delay: int):
self.filename = filename
@ -74,7 +64,7 @@ def token_claim_1h(tenant_name):
# repeat try-wait loop up to max_repeat times until both read and write tr fails for tenant with permission_denied
# important: only use this function if you don't have any data dependencies to key "abc"
def wait_until_tenant_tr_fails(tenant, private_key, tenant_tr_gen, token_gen, max_repeat, delay):
def wait_until_tenant_tr_fails(tenant, private_key, tenant_tr_gen, max_repeat, delay):
repeat = 0
read_blocked = False
write_blocked = False
@ -107,7 +97,7 @@ def wait_until_tenant_tr_fails(tenant, private_key, tenant_tr_gen, token_gen, ma
# repeat try-wait loop up to max_repeat times until both read and write tr succeeds for tenant
# important: only use this function if you don't have any data dependencies to key "abc"
def wait_until_tenant_tr_succeeds(tenant, private_key, tenant_tr_gen, token_gen, max_repeat, delay):
def wait_until_tenant_tr_succeeds(tenant, private_key, tenant_tr_gen, max_repeat, delay):
repeat = 0
token = token_gen(private_key, token_claim_1h(tenant))
while repeat < max_repeat: